From 17790ff3fdd361494ba49de63faac480360bdef5 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Thu, 30 May 2024 09:31:47 -0700 Subject: [PATCH 001/159] Add support for Kafka headers and timestamp in the Kafka Source (#4566) * Add support for Kafka headers and timestamp in the Kafka Source Signed-off-by: Krishna Kondaka * Fix the typo Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka * addressed review comments Signed-off-by: Krishna Kondaka * fixed checkstyle error Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../model/event/DefaultEventMetadata.java | 15 ++++- .../model/event/DefaultEventMetadataTest.java | 19 ++++++ .../kafka/source/KafkaSourceJsonTypeIT.java | 63 ++++++++++++++++++- .../kafka/consumer/KafkaCustomConsumer.java | 12 ++++ 4 files changed, 106 insertions(+), 3 deletions(-) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventMetadata.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventMetadata.java index fa53cd6ad1..a5726e0374 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventMetadata.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventMetadata.java @@ -100,8 +100,19 @@ public void setAttribute(final String key, final Object value) { public Object getAttribute(final String attributeKey) { String key = (attributeKey.charAt(0) == '/') ? attributeKey.substring(1) : attributeKey; - // Does not support recursive or inner-object lookups for now. - return attributes.get(key); + Map mapObject = attributes; + if (key.contains("/")) { + String[] keys = key.split("/"); + for (int i = 0; i < keys.length-1; i++) { + Object value = mapObject.get(keys[i]); + if (value == null || !(value instanceof Map)) { + return null; + } + mapObject = (Map)value; + key = keys[i+1]; + } + } + return mapObject.get(key); } @Override diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventMetadataTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventMetadataTest.java index c87bf1a101..057ce4d1a6 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventMetadataTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventMetadataTest.java @@ -134,6 +134,25 @@ public void testSetAttribute(String key, final Object value) { assertThat(eventMetadata.getAttribute(key), equalTo(value)); } + private static Stream getNestedAttributeTestInputs() { + return Stream.of(Arguments.of(Map.of("k1", "v1", "k2", Map.of("k3", "v3")), "k1", "v1"), + Arguments.of(Map.of("k1", "v1", "k2", Map.of("k3", "v3")), "k2/k3", "v3"), + Arguments.of(Map.of("k1", "v1", "k2", Map.of("k3", Map.of("k4", 4))), "k2/k3/k4", 4), + Arguments.of(Map.of("k1", "v1", "k2", Map.of("k3", 4)), "k2/k3/k4", null), + Arguments.of(Map.of("k1","v1"),"k1", "v1")); + } + + @ParameterizedTest + @MethodSource("getNestedAttributeTestInputs") + public void testNestedGetAttribute(Map attributes, final String key, final Object expectedValue) { + eventMetadata = DefaultEventMetadata.builder() + .withEventType(testEventType) + .withTimeReceived(testTimeReceived) + .withAttributes(attributes) + .build(); + assertThat(eventMetadata.getAttribute(key), equalTo(expectedValue)); + } + @Test public void test_with_ExternalOriginationTime() { Instant now = Instant.now(); 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 19e66e134e..53eeefdf25 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 @@ -13,6 +13,9 @@ import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.internals.RecordHeader; +import org.apache.kafka.common.header.internals.RecordHeaders; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -36,8 +39,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.nio.charset.StandardCharsets; import java.time.Duration; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -49,6 +54,7 @@ import static org.awaitility.Awaitility.await; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; @@ -56,6 +62,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; + public class KafkaSourceJsonTypeIT { private static final Logger LOG = LoggerFactory.getLogger(KafkaSourceJsonTypeIT.class); private static final int TEST_ID = 123456; @@ -98,6 +105,10 @@ public class KafkaSourceJsonTypeIT { private String testKey; private String testTopic; private String testGroup; + private String headerKey1; + private byte[] headerValue1; + private String headerKey2; + private byte[] headerValue2; public KafkaSource createObjectUnderTest() { return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription, kafkaClusterConfigSupplier, pluginConfigObservable); @@ -105,6 +116,10 @@ public KafkaSource createObjectUnderTest() { @BeforeEach public void setup() throws Throwable { + headerKey1 = RandomStringUtils.randomAlphabetic(6); + headerValue1 = RandomStringUtils.randomAlphabetic(10).getBytes(StandardCharsets.UTF_8); + headerKey2 = RandomStringUtils.randomAlphabetic(5); + headerValue2 = RandomStringUtils.randomAlphabetic(15).getBytes(StandardCharsets.UTF_8); sourceConfig = mock(KafkaSourceConfig.class); pluginMetrics = mock(PluginMetrics.class); counter = mock(Counter.class); @@ -209,6 +224,13 @@ public void TestJsonRecordsWithNullKey() throws Exception { assertThat(map.get("kafka_key"), equalTo(null)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(testTopic)); assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); + Map kafkaHeaders = (Map) metadata.getAttributes().get("kafka_headers"); + assertThat(kafkaHeaders.get(headerKey1), equalTo(headerValue1)); + assertThat(kafkaHeaders.get(headerKey2), equalTo(headerValue2)); + assertThat(metadata.getAttributes().get("kafka_timestamp"), not(equalTo(null))); + assertThat(metadata.getAttributes().get("kafka_timestamp_type"), equalTo("CreateTime")); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey1), equalTo(headerValue1)); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey2), equalTo(headerValue2)); } } @@ -240,6 +262,13 @@ public void TestJsonRecordsWithNegativeAcknowledgements() throws Exception { assertThat(map.get("status"), equalTo(true)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(testTopic)); assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); + Map kafkaHeaders = (Map) metadata.getAttributes().get("kafka_headers"); + assertThat(kafkaHeaders.get(headerKey1), equalTo(headerValue1)); + assertThat(kafkaHeaders.get(headerKey2), equalTo(headerValue2)); + assertThat(metadata.getAttributes().get("kafka_timestamp"), not(equalTo(null))); + assertThat(metadata.getAttributes().get("kafka_timestamp_type"), equalTo("CreateTime")); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey1), equalTo(headerValue1)); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey2), equalTo(headerValue2)); event.getEventHandle().release(false); } receivedRecords.clear(); @@ -258,6 +287,13 @@ public void TestJsonRecordsWithNegativeAcknowledgements() throws Exception { assertThat(map.get("status"), equalTo(true)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(testTopic)); assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); + Map kafkaHeaders = (Map) metadata.getAttributes().get("kafka_headers"); + assertThat(kafkaHeaders.get(headerKey1), equalTo(headerValue1)); + assertThat(kafkaHeaders.get(headerKey2), equalTo(headerValue2)); + assertThat(metadata.getAttributes().get("kafka_timestamp"), not(equalTo(null))); + assertThat(metadata.getAttributes().get("kafka_timestamp_type"), equalTo("CreateTime")); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey1), equalTo(headerValue1)); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey2), equalTo(headerValue2)); event.getEventHandle().release(true); } } @@ -289,6 +325,13 @@ public void TestJsonRecordsWithKafkaKeyModeDiscard() throws Exception { assertThat(map.get("status"), equalTo(true)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(testTopic)); assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); + Map kafkaHeaders = (Map) metadata.getAttributes().get("kafka_headers"); + assertThat(kafkaHeaders.get(headerKey1), equalTo(headerValue1)); + assertThat(kafkaHeaders.get(headerKey2), equalTo(headerValue2)); + assertThat(metadata.getAttributes().get("kafka_timestamp"), not(equalTo(null))); + assertThat(metadata.getAttributes().get("kafka_timestamp_type"), equalTo("CreateTime")); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey1), equalTo(headerValue1)); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey2), equalTo(headerValue2)); } } @@ -320,6 +363,13 @@ public void TestJsonRecordsWithKafkaKeyModeAsField() throws Exception { assertThat(map.get("kafka_key"), equalTo(testKey)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(testTopic)); assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); + Map kafkaHeaders = (Map) metadata.getAttributes().get("kafka_headers"); + assertThat(kafkaHeaders.get(headerKey1), equalTo(headerValue1)); + assertThat(kafkaHeaders.get(headerKey2), equalTo(headerValue2)); + assertThat(metadata.getAttributes().get("kafka_timestamp"), not(equalTo(null))); + assertThat(metadata.getAttributes().get("kafka_timestamp_type"), equalTo("CreateTime")); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey1), equalTo(headerValue1)); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey2), equalTo(headerValue2)); } } @@ -351,6 +401,13 @@ public void TestJsonRecordsWithKafkaKeyModeAsMetadata() throws Exception { assertThat(metadata.getAttributes().get("kafka_key"), equalTo(testKey)); assertThat(metadata.getAttributes().get("kafka_topic"), equalTo(testTopic)); assertThat(metadata.getAttributes().get("kafka_partition"), equalTo("0")); + Map kafkaHeaders = (Map) metadata.getAttributes().get("kafka_headers"); + assertThat(kafkaHeaders.get(headerKey1), equalTo(headerValue1)); + assertThat(kafkaHeaders.get(headerKey2), equalTo(headerValue2)); + assertThat(metadata.getAttributes().get("kafka_timestamp"), not(equalTo(null))); + assertThat(metadata.getAttributes().get("kafka_timestamp_type"), equalTo("CreateTime")); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey1), equalTo(headerValue1)); + assertThat(metadata.getAttribute("kafka_headers/"+headerKey2), equalTo(headerValue2)); } } @@ -364,8 +421,12 @@ public void produceJsonRecords(final String servers, final String topicName, fin KafkaProducer producer = new KafkaProducer(props); for (int i = 0; i < numRecords; i++) { String value = "{\"name\":\"testName" + i + "\", \"id\":" + (TEST_ID + i) + ", \"status\":true}"; + List
headers = Arrays.asList( + new RecordHeader(headerKey1, headerValue1), + new RecordHeader(headerKey2, headerValue2) + ); ProducerRecord record = - new ProducerRecord<>(topicName, testKey, value); + new ProducerRecord<>(topicName, null, testKey, value, new RecordHeaders(headers)); producer.send(record); try { Thread.sleep(100); diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java index f6024ef4e7..9e6a979d8e 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java @@ -17,6 +17,8 @@ import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.header.Header; +import org.apache.kafka.common.header.Headers; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.AuthenticationException; import org.apache.kafka.common.errors.RecordDeserializationException; @@ -426,6 +428,16 @@ private Record getRecord(ConsumerRecord consumerRecord, in if (kafkaKeyMode == KafkaKeyMode.INCLUDE_AS_METADATA) { eventMetadata.setAttribute("kafka_key", key); } + Headers headers = consumerRecord.headers(); + if (headers != null) { + Map headerData = new HashMap<>(); + for (Header header: headers) { + headerData.put(header.key(), header.value()); + } + eventMetadata.setAttribute("kafka_headers", headerData); + } + eventMetadata.setAttribute("kafka_timestamp", consumerRecord.timestamp()); + eventMetadata.setAttribute("kafka_timestamp_type", consumerRecord.timestampType().toString()); eventMetadata.setAttribute("kafka_topic", topicName); eventMetadata.setAttribute("kafka_partition", String.valueOf(partition)); eventMetadata.setExternalOriginationTime(Instant.ofEpochMilli(consumerRecord.timestamp())); From d2aa114f538da2f05d887b9c1ad4b77486267776 Mon Sep 17 00:00:00 2001 From: Dinu John <86094133+dinujoh@users.noreply.github.com> Date: Fri, 31 May 2024 09:46:28 -0500 Subject: [PATCH 002/159] Track the source of request for Kafka server (#4572) Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> --- .../kafka/buffer/BufferTopicConfig.java | 10 +++++++++ .../configuration/TopicConsumerConfig.java | 2 ++ .../consumer/KafkaCustomConsumerFactory.java | 11 +++++++--- .../plugins/kafka/source/KafkaSource.java | 21 ++----------------- .../kafka/source/SourceTopicConfig.java | 10 +++++++++ .../plugins/kafka/source/KafkaSourceTest.java | 15 +++++++++++++ 6 files changed, 47 insertions(+), 22 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/BufferTopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/BufferTopicConfig.java index ee0f6557de..56377c1f22 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/BufferTopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/BufferTopicConfig.java @@ -65,6 +65,11 @@ class BufferTopicConfig extends CommonTopicConfig implements TopicProducerConfig @Size(min = 1, max = 255, message = "size of group id should be between 1 and 255") private String groupId; + @JsonProperty("client_id") + @Valid + @Size(min = 1, max = 255, message = "size of client id should be between 1 and 255") + private String clientId; + @JsonProperty("workers") @Valid @Size(min = 1, max = 200, message = "Number of worker threads should lies between 1 and 200") @@ -135,6 +140,11 @@ public String getGroupId() { return groupId; } + @Override + public String getClientId() { + return clientId; + } + @Override public Duration getCommitInterval() { return commitInterval; diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConsumerConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConsumerConfig.java index 0ae2126cbe..0f8de7b458 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConsumerConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConsumerConfig.java @@ -16,6 +16,8 @@ public interface TopicConsumerConfig extends TopicConfig { String getGroupId(); + String getClientId(); + Boolean getAutoCommit(); String getAutoOffsetReset(); diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java index e4f0529ef8..d703538e42 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java @@ -134,14 +134,19 @@ private Properties getConsumerProperties(final KafkaConsumerConfig sourceConfig, break; } } - setConsumerTopicProperties(properties, topicConfig); + setConsumerTopicProperties(properties, topicConfig, topicConfig.getGroupId()); setSchemaRegistryProperties(sourceConfig, properties, topicConfig); LOG.debug("Starting consumer with the properties : {}", properties); return properties; } - private void setConsumerTopicProperties(final Properties properties, final TopicConsumerConfig topicConfig) { - properties.put(ConsumerConfig.GROUP_ID_CONFIG, topicConfig.getGroupId()); + + public static void setConsumerTopicProperties(final Properties properties, final TopicConsumerConfig topicConfig, + final String groupId) { + properties.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); + if (Objects.nonNull(topicConfig.getClientId())) { + properties.put(ConsumerConfig.CLIENT_ID_CONFIG, topicConfig.getClientId()); + } properties.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, (int)topicConfig.getMaxPartitionFetchBytes()); properties.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, ((Long)topicConfig.getRetryBackoff().toMillis()).intValue()); properties.put(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG, ((Long)topicConfig.getReconnectBackoff().toMillis()).intValue()); 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 3877350d3f..6a01a91bf0 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 @@ -37,6 +37,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaRegistryType; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConfig; import org.opensearch.dataprepper.plugins.kafka.consumer.KafkaCustomConsumer; +import org.opensearch.dataprepper.plugins.kafka.consumer.KafkaCustomConsumerFactory; import org.opensearch.dataprepper.plugins.kafka.consumer.PauseConsumePredicate; import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; import org.opensearch.dataprepper.plugins.kafka.util.ClientDNSLookupType; @@ -318,25 +319,7 @@ private void setPropertiesForSchemaType(Properties properties, TopicConfig topic } private void setConsumerTopicProperties(Properties properties, TopicConsumerConfig topicConfig) { - properties.put(ConsumerConfig.GROUP_ID_CONFIG, consumerGroupID); - properties.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, (int) topicConfig.getMaxPartitionFetchBytes()); - properties.put(ConsumerConfig.RETRY_BACKOFF_MS_CONFIG, ((Long) topicConfig.getRetryBackoff().toMillis()).intValue()); - properties.put(ConsumerConfig.RECONNECT_BACKOFF_MS_CONFIG, ((Long) topicConfig.getReconnectBackoff().toMillis()).intValue()); - properties.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, - topicConfig.getAutoCommit()); - properties.put(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG, - ((Long) topicConfig.getCommitInterval().toMillis()).intValue()); - properties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, - topicConfig.getAutoOffsetReset()); - properties.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, - topicConfig.getConsumerMaxPollRecords()); - properties.put(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, - ((Long) topicConfig.getMaxPollInterval().toMillis()).intValue()); - properties.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, ((Long) topicConfig.getSessionTimeOut().toMillis()).intValue()); - properties.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, ((Long) topicConfig.getHeartBeatInterval().toMillis()).intValue()); - properties.put(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, (int) topicConfig.getFetchMaxBytes()); - properties.put(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, topicConfig.getFetchMaxWait()); - properties.put(ConsumerConfig.FETCH_MIN_BYTES_CONFIG, (int) topicConfig.getFetchMinBytes()); + KafkaCustomConsumerFactory.setConsumerTopicProperties(properties, topicConfig, consumerGroupID); } private void setPropertiesForSchemaRegistryConnectivity(Properties properties) { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/SourceTopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/SourceTopicConfig.java index adcf030f1f..703fcded19 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/SourceTopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/source/SourceTopicConfig.java @@ -49,6 +49,11 @@ class SourceTopicConfig extends CommonTopicConfig implements TopicConsumerConfig @Size(min = 1, max = 255, message = "size of group id should be between 1 and 255") private String groupId; + @JsonProperty("client_id") + @Valid + @Size(min = 1, max = 255, message = "size of client id should be between 1 and 255") + private String clientId; + @JsonProperty("workers") @Valid @Size(min = 1, max = 200, message = "Number of worker threads should lies between 1 and 200") @@ -121,6 +126,11 @@ public String getGroupId() { return groupId; } + @Override + public String getClientId() { + return clientId; + } + @Override public MessageFormat getSerdeFormat() { return serdeFormat; 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 ab7b07c9b0..1503a7424d 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 @@ -82,6 +82,7 @@ class KafkaSourceTest { private PluginConfigObservable pluginConfigObservable; private static final String TEST_GROUP_ID = "testGroupId"; + private static final String TEST_CLIENT_ID = "testClientId"; public KafkaSource createObjectUnderTest() { return new KafkaSource( @@ -107,6 +108,8 @@ 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.getClientId()).thenReturn(TEST_CLIENT_ID); + when(topic2.getClientId()).thenReturn(TEST_CLIENT_ID); when(topic1.getMaxPollInterval()).thenReturn(Duration.ofSeconds(5)); when(topic2.getMaxPollInterval()).thenReturn(Duration.ofSeconds(5)); when(topic1.getHeartBeatInterval()).thenReturn(Duration.ofSeconds(5)); @@ -156,6 +159,18 @@ void test_kafkaSource_basicFunctionality() { assertTrue(Objects.nonNull(kafkaSource.getConsumer())); } + @Test + void test_kafkaSource_basicFunctionalityWithClientIdNull() { + when(topic1.getSessionTimeOut()).thenReturn(Duration.ofSeconds(15)); + when(topic2.getSessionTimeOut()).thenReturn(Duration.ofSeconds(15)); + when(topic1.getClientId()).thenReturn(null); + when(topic1.getClientId()).thenReturn(null); + kafkaSource = createObjectUnderTest(); + assertTrue(Objects.nonNull(kafkaSource)); + kafkaSource.start(buffer); + assertTrue(Objects.nonNull(kafkaSource.getConsumer())); + } + @Test void test_kafkaSource_retry_consumer_create() throws InterruptedException { when(topic1.getSessionTimeOut()).thenReturn(Duration.ofSeconds(15)); From 609b94c0d896672b508e83bc2c2fbfe615cb33e7 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Tue, 4 Jun 2024 15:19:35 -0500 Subject: [PATCH 003/159] Add Rds source config (#4573) * Add rds source config and some skeleton code Signed-off-by: Hai Yan * Add unit tests Signed-off-by: Hai Yan * Address comments Signed-off-by: Hai Yan * Add cluster and aurora options Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- data-prepper-plugins/rds-source/build.gradle | 26 ++++ .../plugins/source/rds/ClientFactory.java | 35 +++++ .../plugins/source/rds/RdsService.java | 71 +++++++++++ .../plugins/source/rds/RdsSource.java | 75 +++++++++++ .../plugins/source/rds/RdsSourceConfig.java | 120 ++++++++++++++++++ .../AwsAuthenticationConfig.java | 48 +++++++ .../source/rds/configuration/EngineType.java | 38 ++++++ .../rds/configuration/ExportConfig.java | 20 +++ .../source/rds/export/ExportScheduler.java | 48 +++++++ .../source/rds/leader/LeaderScheduler.java | 32 +++++ .../plugins/source/rds/RdsServiceTest.java | 89 +++++++++++++ .../source/rds/RdsSourceConfigTest.java | 51 ++++++++ .../plugins/source/rds/RdsSourceTest.java | 50 ++++++++ settings.gradle | 1 + 14 files changed, 704 insertions(+) create mode 100644 data-prepper-plugins/rds-source/build.gradle create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/ClientFactory.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/AwsAuthenticationConfig.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/ExportConfig.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfigTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java diff --git a/data-prepper-plugins/rds-source/build.gradle b/data-prepper-plugins/rds-source/build.gradle new file mode 100644 index 0000000000..8372276564 --- /dev/null +++ b/data-prepper-plugins/rds-source/build.gradle @@ -0,0 +1,26 @@ +plugins { + id 'java' +} + +dependencies { + implementation project(path: ':data-prepper-api') + implementation project(path: ':data-prepper-plugins:aws-plugin-api') + implementation project(path: ':data-prepper-plugins:buffer-common') + implementation project(path: ':data-prepper-plugins:http-common') + implementation project(path: ':data-prepper-plugins:common') + + implementation 'io.micrometer:micrometer-core' + + implementation 'software.amazon.awssdk:sts' + implementation 'software.amazon.awssdk:arns' + implementation 'software.amazon.awssdk:rds' + implementation 'software.amazon.awssdk:s3' + implementation 'software.amazon.awssdk:netty-nio-client' + + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + + testImplementation testLibs.mockito.inline + testImplementation project(path: ':data-prepper-test-common') + testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/ClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/ClientFactory.java new file mode 100644 index 0000000000..9cdb2bfa50 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/ClientFactory.java @@ -0,0 +1,35 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.source.rds.configuration.AwsAuthenticationConfig; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.services.rds.RdsClient; + +public class ClientFactory { + private final AwsCredentialsProvider awsCredentialsProvider; + private final AwsAuthenticationConfig awsAuthenticationConfig; + + public ClientFactory(final AwsCredentialsSupplier awsCredentialsSupplier, + final AwsAuthenticationConfig awsAuthenticationConfig) { + awsCredentialsProvider = awsCredentialsSupplier.getProvider(AwsCredentialsOptions.builder() + .withRegion(awsAuthenticationConfig.getAwsRegion()) + .withStsRoleArn(awsAuthenticationConfig.getAwsStsRoleArn()) + .withStsExternalId(awsAuthenticationConfig.getAwsStsExternalId()) + .withStsHeaderOverrides(awsAuthenticationConfig.getAwsStsHeaderOverrides()) + .build()); + this.awsAuthenticationConfig = awsAuthenticationConfig; + } + + public RdsClient buildRdsClient() { + return RdsClient.builder() + .region(awsAuthenticationConfig.getAwsRegion()) + .credentialsProvider(awsCredentialsProvider) + .build(); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java new file mode 100644 index 0000000000..0e8a92e31d --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -0,0 +1,71 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; +import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.rds.RdsClient; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class RdsService { + private static final Logger LOG = LoggerFactory.getLogger(RdsService.class); + + private final RdsClient rdsClient; + private final EnhancedSourceCoordinator sourceCoordinator; + private final PluginMetrics pluginMetrics; + private final RdsSourceConfig sourceConfig; + private ExecutorService executor; + + public RdsService(final EnhancedSourceCoordinator sourceCoordinator, + final RdsSourceConfig sourceConfig, + final ClientFactory clientFactory, + final PluginMetrics pluginMetrics) { + this.sourceCoordinator = sourceCoordinator; + this.pluginMetrics = pluginMetrics; + this.sourceConfig = sourceConfig; + + rdsClient = clientFactory.buildRdsClient(); + } + + /** + * This service start three long-running threads (scheduler) + * Each thread is responsible for one type of job. + * The data will be guaranteed to be sent to {@link Buffer} in order. + * + * @param buffer Data Prepper Buffer + */ + public void start(Buffer> buffer) { + LOG.info("Start running RDS service"); + final List runnableList = new ArrayList<>(); + runnableList.add(new LeaderScheduler(sourceCoordinator, sourceConfig)); + runnableList.add(new ExportScheduler(sourceCoordinator, rdsClient, pluginMetrics)); + + executor = Executors.newFixedThreadPool(runnableList.size()); + runnableList.forEach(executor::submit); + } + + /** + * Interrupt the running of schedulers. + * Each scheduler must implement logic for gracefully shutdown. + */ + public void shutdown() { + if (executor != null) { + LOG.info("shutdown RDS schedulers"); + executor.shutdownNow(); + } + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java new file mode 100644 index 0000000000..cc4bd23ca0 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java @@ -0,0 +1,75 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.UsesEnhancedSourceCoordination; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.function.Function; + +@DataPrepperPlugin(name = "rds", pluginType = Source.class, pluginConfigurationType = RdsSourceConfig.class) +public class RdsSource implements Source>, UsesEnhancedSourceCoordination { + + private static final Logger LOG = LoggerFactory.getLogger(RdsSource.class); + + private final ClientFactory clientFactory; + private final PluginMetrics pluginMetrics; + private final RdsSourceConfig sourceConfig; + private EnhancedSourceCoordinator sourceCoordinator; + private RdsService rdsService; + + @DataPrepperPluginConstructor + public RdsSource(final PluginMetrics pluginMetrics, + final RdsSourceConfig sourceConfig, + final AwsCredentialsSupplier awsCredentialsSupplier) { + this.pluginMetrics = pluginMetrics; + this.sourceConfig = sourceConfig; + + clientFactory = new ClientFactory(awsCredentialsSupplier, sourceConfig.getAwsAuthenticationConfig()); + } + + @Override + public void start(Buffer> buffer) { + Objects.requireNonNull(sourceCoordinator); + + rdsService = new RdsService(sourceCoordinator, sourceConfig, clientFactory, pluginMetrics); + + LOG.info("Start RDS service"); + rdsService.start(buffer); + } + + @Override + public void stop() { + LOG.info("Stop RDS service"); + if (Objects.nonNull(rdsService)) { + rdsService.shutdown(); + } + } + + @Override + public void setEnhancedSourceCoordinator(EnhancedSourceCoordinator sourceCoordinator) { + this.sourceCoordinator = sourceCoordinator; + this.sourceCoordinator.initialize(); + } + + @Override + public Function getPartitionFactory() { + return null; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java new file mode 100644 index 0000000000..cb8e41513a --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java @@ -0,0 +1,120 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.plugins.source.rds.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; +import org.opensearch.dataprepper.plugins.source.rds.configuration.ExportConfig; + +import java.util.List; + +/** + * Configuration for RDS Source + */ +public class RdsSourceConfig { + + /** + * Identifier for RDS instance/cluster or Aurora cluster + */ + @JsonProperty("db_identifier") + private String dbIdentifier; + + /** + * Whether the db_identifier refers to a cluster or an instance + */ + @JsonProperty("cluster") + private boolean isCluster = false; + + @JsonProperty("engine") + private EngineType engine = EngineType.MYSQL; + + /** + * Whether the source is an Aurora cluster + */ + @JsonProperty("aurora") + private boolean isAurora = false; + + /** + * The table name is in the format of `database.table` for MySQL engine + */ + @JsonProperty("table_names") + private List tableNames; + + @JsonProperty("aws") + @NotNull + @Valid + private AwsAuthenticationConfig awsAuthenticationConfig; + + @JsonProperty("acknowledgments") + private boolean acknowledgments = false; + + /** + * S3 bucket for holding both export and stream data + */ + @JsonProperty("s3_bucket") + private String s3Bucket; + + @JsonProperty("s3_prefix") + private String s3Prefix; + + @JsonProperty("s3_region") + private String s3Region; + + @JsonProperty("export") + @Valid + private ExportConfig exportConfig; + + public String getDbIdentifier() { + return dbIdentifier; + } + + public boolean isCluster() { + return isCluster; + } + + public EngineType getEngine() { + return engine; + } + + public boolean isAurora() { + return isAurora; + } + + public List getTableNames() { + return tableNames; + } + + public AwsAuthenticationConfig getAwsAuthenticationConfig() { + return awsAuthenticationConfig; + } + + public boolean isAcknowledgmentsEnabled() { + return acknowledgments; + } + + public String getS3Bucket() { + return s3Bucket; + } + + public String getS3Prefix() { + return s3Prefix; + } + + public String getS3Region() { + return s3Region; + } + + public ExportConfig getExport() { + return exportConfig; + } + + public boolean isExportEnabled() { + return exportConfig != null; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/AwsAuthenticationConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/AwsAuthenticationConfig.java new file mode 100644 index 0000000000..1684d08a06 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/AwsAuthenticationConfig.java @@ -0,0 +1,48 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.regions.Region; + +import java.util.Map; + +public class AwsAuthenticationConfig { + + @JsonProperty("region") + @Size(min = 1, message = "Region cannot be empty string") + private String awsRegion; + + @JsonProperty("sts_role_arn") + @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") + private String awsStsRoleArn; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } +} + diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java new file mode 100644 index 0000000000..f75ec32bfe --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java @@ -0,0 +1,38 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.configuration; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +public enum EngineType { + + MYSQL("mysql"); + + private static final Map ENGINE_TYPE_MAP = Arrays.stream(EngineType.values()) + .collect(Collectors.toMap( + value -> value.engine, + value -> value + )); + private final String engine; + + EngineType(String engine) { + this.engine = engine; + } + + @Override + public String toString() { + return engine; + } + + @JsonCreator + public static EngineType fromOptionValue(final String option) { + return ENGINE_TYPE_MAP.get(option); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/ExportConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/ExportConfig.java new file mode 100644 index 0000000000..b3bdedcaef --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/ExportConfig.java @@ -0,0 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; + +public class ExportConfig { + + @JsonProperty("kms_key_id") + @NotNull + private String kmsKeyId; + + public String getKmsKeyId() { + return kmsKeyId; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java new file mode 100644 index 0000000000..9c83643c68 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -0,0 +1,48 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.rds.RdsClient; + +import java.time.Duration; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class ExportScheduler implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(ExportScheduler.class); + + private static final int DEFAULT_TAKE_LEASE_INTERVAL_MILLIS = 60_000; + private static final Duration DEFAULT_CLOSE_DURATION = Duration.ofMinutes(10); + private static final int DEFAULT_MAX_CLOSE_COUNT = 36; + private static final int DEFAULT_CHECKPOINT_INTERVAL_MILLS = 5 * 60_000; + private static final int DEFAULT_CHECK_STATUS_INTERVAL_MILLS = 30 * 1000; + + private final RdsClient rdsClient; + + private final PluginMetrics pluginMetrics; + + private final EnhancedSourceCoordinator sourceCoordinator; + + private final ExecutorService executor; + + public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, + final RdsClient rdsClient, + final PluginMetrics pluginMetrics) { + this.pluginMetrics = pluginMetrics; + this.sourceCoordinator = sourceCoordinator; + this.rdsClient = rdsClient; + this.executor = Executors.newCachedThreadPool(); + } + + @Override + public void run() { + + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java new file mode 100644 index 0000000000..ca99a7c8f1 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -0,0 +1,32 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.leader; + +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; + +public class LeaderScheduler implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(LeaderScheduler.class); + private static final int DEFAULT_EXTEND_LEASE_MINUTES = 3; + private static final Duration DEFAULT_LEASE_INTERVAL = Duration.ofMinutes(1); + private final EnhancedSourceCoordinator sourceCoordinator; + private final RdsSourceConfig sourceConfig; + + public LeaderScheduler(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig) { + this.sourceCoordinator = sourceCoordinator; + this.sourceConfig = sourceConfig; + } + + @Override + public void run() { + + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java new file mode 100644 index 0000000000..218c23d121 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java @@ -0,0 +1,89 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; +import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; +import software.amazon.awssdk.services.rds.RdsClient; + +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class RdsServiceTest { + + @Mock + private RdsClient rdsClient; + + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private RdsSourceConfig sourceConfig; + + @Mock + private ExecutorService executor; + + @Mock + private ClientFactory clientFactory; + + @Mock + private Buffer> buffer; + + @BeforeEach + void setUp() { + when(clientFactory.buildRdsClient()).thenReturn(rdsClient); + + } + + @Test + void test_normal_service_start() { + RdsService rdsService = createObjectUnderTest(); + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { + executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); + rdsService.start(buffer); + } + + verify(executor).submit(any(LeaderScheduler.class)); + verify(executor).submit(any(ExportScheduler.class)); + } + + @Test + void test_service_shutdown_calls_executor_shutdownNow() { + RdsService rdsService = createObjectUnderTest(); + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { + executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); + rdsService.start(buffer); + } + rdsService.shutdown(); + + verify(executor).shutdownNow(); + } + + private RdsService createObjectUnderTest() { + return new RdsService(sourceCoordinator, sourceConfig, clientFactory, pluginMetrics); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfigTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfigTest.java new file mode 100644 index 0000000000..a23dccde4c --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfigTest.java @@ -0,0 +1,51 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds; + +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; +import org.opensearch.dataprepper.plugins.source.rds.configuration.ExportConfig; + +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; + +class RdsSourceConfigTest { + + @Test + void test_default_RdsSrouceConfig_returns_default_values() { + final RdsSourceConfig objectUnderTest = createObjectUnderTest(); + + assertThat(objectUnderTest.getEngine(), equalTo(EngineType.MYSQL)); + assertThat(objectUnderTest.isCluster(), equalTo(false)); + assertThat(objectUnderTest.isAurora(), equalTo(false)); + assertThat(objectUnderTest.isAcknowledgmentsEnabled(), equalTo(false)); + } + + @Test + void test_when_export_is_not_configured_then_isExportEnabled_returns_false() { + final RdsSourceConfig objectUnderTest = createObjectUnderTest(); + + assertThat(objectUnderTest.isExportEnabled(), equalTo(false)); + } + + @Test + void test_when_export_is_configured_then_isExportEnabled_returns_true() throws NoSuchFieldException, IllegalAccessException { + final RdsSourceConfig objectUnderTest = createObjectUnderTest(); + + ExportConfig exportConfig = new ExportConfig(); + setField(ExportConfig.class, exportConfig, "kmsKeyId", UUID.randomUUID().toString()); + setField(RdsSourceConfig.class, objectUnderTest, "exportConfig", exportConfig); + + assertThat(objectUnderTest.isExportEnabled(), equalTo(true)); + } + + private RdsSourceConfig createObjectUnderTest() { + return new RdsSourceConfig(); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java new file mode 100644 index 0000000000..edd409e5e4 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java @@ -0,0 +1,50 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds; + +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.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.plugins.source.rds.configuration.AwsAuthenticationConfig; + +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.when; + + +@ExtendWith(MockitoExtension.class) +class RdsSourceTest { + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private RdsSourceConfig sourceConfig; + + @Mock + AwsCredentialsSupplier awsCredentialsSupplier; + + @Mock + private AwsAuthenticationConfig awsAuthenticationConfig; + + @BeforeEach + void setUp() { + when(sourceConfig.getAwsAuthenticationConfig()).thenReturn(awsAuthenticationConfig); + } + + @Test + void test_when_buffer_is_null_then_start_throws_exception() { + RdsSource objectUnderTest = createObjectUnderTest(); + assertThrows(NullPointerException.class, () -> objectUnderTest.start(null)); + } + + private RdsSource createObjectUnderTest() { + return new RdsSource(pluginMetrics, sourceConfig, awsCredentialsSupplier); + } +} \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index ef718504de..3d07f3114b 100644 --- a/settings.gradle +++ b/settings.gradle @@ -172,3 +172,4 @@ include 'data-prepper-plugins:split-event-processor' include 'data-prepper-plugins:http-common' include 'data-prepper-plugins:flatten-processor' include 'data-prepper-plugins:mongodb' +include 'data-prepper-plugins:rds-source' From 7d15115c281687aab50e5c471fd210cb1ef90fc5 Mon Sep 17 00:00:00 2001 From: Santhosh Gandhe <1909520+san81@users.noreply.github.com> Date: Tue, 4 Jun 2024 13:40:46 -0700 Subject: [PATCH 004/159] Introduced BigDecimalConverter (#4557) * Introduced BigDecimalConverter that users can use as part of convert_entry_type processor that currently exists. Optionally, users can also specify required scaling needed on the converted Signed-off-by: Santhosh Gandhe * Added Test case for the newly introduced class. Removed * imports as per the review comment Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * Avoiding using a deprecated method. Added additional test cases Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * Additional tests to increase the coverage Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * removed "scale" being the state of BigDecimal converter. We are now passing the scale while converting the instance only when the instance is BigDecimalConverter Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * test case fix to be inline with the previous commit Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * test case fix to be inline with the previous commit Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * addressing review comments Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * renaming bigdecimal to big_decimal Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * Introduced ConverterArguments as a way to pass additional arguments to the converter and avoided conditional statement for calling converter methods Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * Added additional override convert method to reduce the changes across the code Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * additional Test cases to increase the coverage Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * added additional tests for converter cases Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> --------- Signed-off-by: Santhosh Gandhe Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> --- .../dataprepper/model/event/DataType.java | 38 +++--- .../dataprepper/model/event/JacksonEvent.java | 9 +- .../source/coordinator/SourceCoordinator.java | 1 + .../typeconverter/BigDecimalConverter.java | 58 ++++++++++ .../typeconverter/BooleanConverter.java | 15 ++- .../typeconverter/ConverterArguments.java | 10 ++ .../typeconverter/DoubleConverter.java | 7 +- .../typeconverter/IntegerConverter.java | 5 + .../typeconverter/LongConverter.java | 5 + .../typeconverter/StringConverter.java | 5 + .../typeconverter/TypeConverter.java | 1 + .../dataprepper/model/event/DataTypeTest.java | 8 +- .../model/event/DefaultEventMetadataTest.java | 48 ++++++++ .../model/event/JacksonEventTest.java | 42 ++++++- .../BigDecimalConverterTests.java | 109 ++++++++++++++++++ .../typeconverter/BooleanConverterTests.java | 12 +- .../typeconverter/DoubleConverterTests.java | 17 +-- .../typeconverter/IntegerConverterTests.java | 23 ++-- .../typeconverter/LongConverterTests.java | 9 +- .../typeconverter/StringConverterTests.java | 1 + ...DataPrepperScalarTypeDeserializerTest.java | 4 +- .../plugin/VariableExpanderTest.java | 3 + .../ConvertEntryTypeProcessor.java | 12 +- .../ConvertEntryTypeProcessorConfig.java | 16 ++- .../processor/mutateevent/TargetType.java | 14 ++- .../ConvertEntryTypeProcessorTests.java | 70 +++++++++-- .../processor/mutateevent/TargetTypeTest.java | 3 +- 27 files changed, 470 insertions(+), 75 deletions(-) create mode 100644 data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/BigDecimalConverter.java create mode 100644 data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/ConverterArguments.java create mode 100644 data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/BigDecimalConverterTests.java diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DataType.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DataType.java index 44852a36ab..c6e899a6f4 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DataType.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DataType.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; +import java.math.BigDecimal; import java.util.Arrays; import java.util.ArrayList; import java.util.Map; @@ -49,6 +50,13 @@ public enum DataType { */ DOUBLE("double"), + /** + * Type of BigDecimal. No precision loss possible type. Compatible with the Java BigDecimal primitive data type. + * + * @since 2.8 + */ + BIG_DECIMAL("big_decimal"), + /** * Type of map. Compatible with the Java map primitive data type. * @@ -96,20 +104,22 @@ public static boolean isSameType(final Object object, final String option) { if (type == null) throw new IllegalArgumentException("Unknown DataType"); switch (type) { - case MAP: - return (object instanceof Map); - case ARRAY: - return (object instanceof ArrayList || object.getClass().isArray()); - case DOUBLE: - return (object instanceof Double); - case BOOLEAN: - return (object instanceof Boolean); - case INTEGER: - return (object instanceof Integer); - case LONG: - return (object instanceof Long); - default: // STRING - return (object instanceof String); + case MAP: + return (object instanceof Map); + case ARRAY: + return (object instanceof ArrayList || object.getClass().isArray()); + case DOUBLE: + return (object instanceof Double); + case BOOLEAN: + return (object instanceof Boolean); + case INTEGER: + return (object instanceof Integer); + case LONG: + return (object instanceof Long); + case BIG_DECIMAL: + return (object instanceof BigDecimal); + default: // STRING + return (object instanceof String); } } } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java index e135f7e9db..9ef34bb82c 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java @@ -11,6 +11,8 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectReader; +import com.fasterxml.jackson.databind.cfg.JsonNodeFeature; +import com.fasterxml.jackson.databind.json.JsonMapper; import com.fasterxml.jackson.databind.node.ArrayNode; import com.fasterxml.jackson.databind.node.ObjectNode; import com.fasterxml.jackson.databind.type.TypeFactory; @@ -61,11 +63,14 @@ public class JacksonEvent implements Event { private static final String SEPARATOR = "/"; - private static final ObjectMapper mapper = new ObjectMapper() + private static final ObjectMapper mapper = JsonMapper.builder() + .disable(JsonNodeFeature.STRIP_TRAILING_BIGDECIMAL_ZEROES) + .build() .registerModule(new JavaTimeModule()) .registerModule(new Jdk8Module()); // required for using Optional with Jackson. Ref: https://github.com/FasterXML/jackson-modules-java8 - private static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference>() { + + private static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference<>() { }; private final EventMetadata eventMetadata; diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/source/coordinator/SourceCoordinator.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/source/coordinator/SourceCoordinator.java index fa78880925..71c0232669 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/source/coordinator/SourceCoordinator.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/source/coordinator/SourceCoordinator.java @@ -122,6 +122,7 @@ public interface SourceCoordinator { /** * Should be called by the source when it is shutting down to indicate that it will no longer be able to perform work on partitions, * or can be called to give up ownership of its partitions in order to pick up new ones with {@link #getNextPartition(Function)} ()}. + * @param partitionKey - Key used as the partition key. * @param priorityTimestamp - A timestamp that will determine the order that UNASSIGNED partitions are acquired after they are given up. * @throws org.opensearch.dataprepper.model.source.coordinator.exceptions.PartitionUpdateException if the partition could not be given up due to some failure * @since 2.8 diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/BigDecimalConverter.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/BigDecimalConverter.java new file mode 100644 index 0000000000..8935e55e71 --- /dev/null +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/BigDecimalConverter.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + + package org.opensearch.dataprepper.typeconverter; + +import java.math.BigDecimal; +import java.math.RoundingMode; + +/** + * Converter class for BigDecimal data type. By default, it applies zero scaling keeping the original value as it is. + * If required, the scale can be set using the setScale method. + */ +public class BigDecimalConverter implements TypeConverter { + + public BigDecimal convert(Object source) throws IllegalArgumentException { + return this.convert(source, 0); + } + + public BigDecimal convert(Object source, ConverterArguments arguments) throws IllegalArgumentException { + return this.convert(source, arguments.getScale()); + } + + public BigDecimal convert(Object source, int scale) throws IllegalArgumentException { + BigDecimal result = null; + if (source instanceof String) { + result = new BigDecimal((String)source); + } + else if (source instanceof Float) { + result = BigDecimal.valueOf((Float)source); + } + else if (source instanceof Double) { + result = BigDecimal.valueOf((Double)source); + } + else if (source instanceof Boolean) { + result = ((Boolean)source) ? BigDecimal.valueOf(1L) : BigDecimal.valueOf(0L); + } + else if (source instanceof Integer) { + result = BigDecimal.valueOf((Integer)source); + } + else if (source instanceof Long) { + result = BigDecimal.valueOf((Long)source); + } + else if (source instanceof BigDecimal) { + result = ((BigDecimal)source); + } + + if(result!=null) { + if(scale!=0) { + result = result.setScale(scale, RoundingMode.HALF_EVEN); + } + return result; + } + throw new IllegalArgumentException("Unsupported type conversion. From Source class: " + source.getClass() + " to BigDecimal"); + } +} + \ No newline at end of file diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/BooleanConverter.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/BooleanConverter.java index a24c0bdc64..3da4669624 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/BooleanConverter.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/BooleanConverter.java @@ -6,18 +6,23 @@ package org.opensearch.dataprepper.typeconverter; public class BooleanConverter implements TypeConverter { + + public Boolean convert(Object source, ConverterArguments arguments) throws IllegalArgumentException { + return this.convert(source); + } + public Boolean convert(Object source) throws IllegalArgumentException { if (source instanceof String) { return Boolean.parseBoolean((String)source); } if (source instanceof Number) { Number number = (Number)source; - return ((number.intValue() != 0) || - (number.longValue() != 0) || + return ((number instanceof Integer && number.intValue() != 0) || + (number instanceof Long && number.longValue() != 0) || + (number instanceof Short && number.shortValue() != 0) || + (number instanceof Byte && number.byteValue() != 0)) || (number.floatValue() != 0) || - (number.doubleValue() != 0) || - (number.shortValue() != 0) || - (number.byteValue() != 0)) ? true : false; + (number.doubleValue() != 0); } if (source instanceof Boolean) { return (Boolean)source; diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/ConverterArguments.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/ConverterArguments.java new file mode 100644 index 0000000000..3fad9e3a0e --- /dev/null +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/ConverterArguments.java @@ -0,0 +1,10 @@ +package org.opensearch.dataprepper.typeconverter; + +/** + * Interface for arguments passed to the {@link TypeConverter} + * + * @since 1.2 + */ +public interface ConverterArguments { + int getScale(); +} diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/DoubleConverter.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/DoubleConverter.java index db57bbe9eb..aa80c1360e 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/DoubleConverter.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/DoubleConverter.java @@ -6,6 +6,11 @@ package org.opensearch.dataprepper.typeconverter; public class DoubleConverter implements TypeConverter { + + public Double convert(Object source, ConverterArguments arguments) throws IllegalArgumentException { + return convert(source); + } + public Double convert(Object source) throws IllegalArgumentException { if (source instanceof String) { return Double.parseDouble((String)source); @@ -17,7 +22,7 @@ public Double convert(Object source) throws IllegalArgumentException { return (((Number)source).doubleValue()); } if (source instanceof Boolean) { - return (double)(((Boolean)source) ? 1.0 : 0.0); + return ((Boolean)source) ? 1.0 : 0.0; } throw new IllegalArgumentException("Unsupported type conversion. Source class: " + source.getClass()); } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/IntegerConverter.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/IntegerConverter.java index 664c7f463f..41420b9dde 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/IntegerConverter.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/IntegerConverter.java @@ -7,6 +7,11 @@ import java.math.BigDecimal; public class IntegerConverter implements TypeConverter { + + public Integer convert(Object source, ConverterArguments arguments) throws IllegalArgumentException { + return convert(source); + } + public Integer convert(Object source) throws IllegalArgumentException { if (source instanceof String) { return Integer.parseInt((String)source); diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/LongConverter.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/LongConverter.java index 809bec9280..e3c1ee2b08 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/LongConverter.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/LongConverter.java @@ -7,6 +7,11 @@ import java.math.BigDecimal; public class LongConverter implements TypeConverter { + + public Long convert(Object source, ConverterArguments arguments) throws IllegalArgumentException { + return this.convert(source); + } + public Long convert(Object source) throws IllegalArgumentException { if (source instanceof String) { return Long.parseLong((String)source); diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/StringConverter.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/StringConverter.java index c2ad1a1eb6..acc9d08fce 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/StringConverter.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/StringConverter.java @@ -6,6 +6,11 @@ package org.opensearch.dataprepper.typeconverter; public class StringConverter implements TypeConverter { + + public String convert(Object source, ConverterArguments arguments) throws IllegalArgumentException { + return this.convert(source); + } + public String convert(Object source) throws IllegalArgumentException { if (source instanceof Number || source instanceof Boolean || source instanceof String) { return source.toString(); diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/TypeConverter.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/TypeConverter.java index 9d597d38a9..4256b8571f 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/TypeConverter.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/typeconverter/TypeConverter.java @@ -7,4 +7,5 @@ public interface TypeConverter { T convert(Object source); + T convert(Object source, ConverterArguments arguments); } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DataTypeTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DataTypeTest.java index 015fb91873..ac7a5bf613 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DataTypeTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DataTypeTest.java @@ -15,6 +15,9 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.List; import java.util.Map; import java.util.stream.Stream; @@ -37,15 +40,18 @@ void test_isSameType(Object object, String type, boolean expectedResult) { } private static Stream getSameTypeTestData() { - int testArray[] = {1,2}; + int[] testArray = {1,2}; + List testList = new ArrayList<>(); return Stream.of( Arguments.of(2, "integer", true), Arguments.of("testString", "string", true), Arguments.of(2L, "long", true), Arguments.of(2.0, "double", true), + Arguments.of(BigDecimal.valueOf(2.34567), "big_decimal", true), Arguments.of(true, "boolean", true), Arguments.of(Map.of("k","v"), "map", true), Arguments.of(testArray, "array", true), + Arguments.of(testList, "array", true), Arguments.of(2.0, "integer", false), Arguments.of(2, "string", false), Arguments.of("testString", "long", false), diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventMetadataTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventMetadataTest.java index 057ce4d1a6..7e91a86eb3 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventMetadataTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventMetadataTest.java @@ -176,6 +176,20 @@ public void testAttributes_without_attributes_is_empty() { } + @Test + public void testAttributes_with_attributes_of_not_map_type() { + Object v1 = new Object(); + eventMetadata = DefaultEventMetadata.builder() + .withEventType(testEventType) + .withTimeReceived(testTimeReceived) + .withAttributes(Map.of("key1", v1)) + .build(); + assertThat(eventMetadata.getAttribute("key1"), equalTo(v1)); + assertThat(eventMetadata.getAttribute("key1/key2/"), equalTo(null)); + assertThat(eventMetadata.getAttribute("key3"), equalTo(null)); + + } + @Test public void testBuild_withoutTimeReceived() { @@ -303,6 +317,40 @@ void setUp() { @Test void equals_returns_false_for_null() { assertThat(event.equals(null), equalTo(false)); + assertThat(event.equals(new Object()), equalTo(false)); + } + + @Test + void equals_returns_false_when_timeinstance_not_match() { + DefaultEventMetadata newEvent = DefaultEventMetadata.builder() + .withEventType(eventType) + .withTimeReceived(Instant.now()) + .withAttributes(Collections.singletonMap(attributeKey, attributeValue)) + .build(); + assertThat(event.equals(newEvent), equalTo(false)); + } + + @Test + void equals_returns_false_when_attributes_not_match() { + String newAttributeKey = UUID.randomUUID().toString(); + String newAttributeValue = UUID.randomUUID().toString(); + DefaultEventMetadata newEvent = DefaultEventMetadata.builder() + .withEventType(eventType) + .withTimeReceived(timeReceived) + .withAttributes(Collections.singletonMap(newAttributeKey, newAttributeValue)) + .build(); + assertThat(event.equals(newEvent), equalTo(false)); + } + + @Test + void equals_returns_false_when_tags_not_match() { + DefaultEventMetadata newEvent = DefaultEventMetadata.builder() + .withEventType(eventType) + .withTimeReceived(timeReceived) + .withAttributes(Collections.singletonMap(attributeKey, attributeValue)) + .withTags(Set.of("some","new","tag")) + .build(); + assertThat(event.equals(newEvent), equalTo(false)); } @Test diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java index e6fdceecfc..1a7efb7467 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java @@ -9,11 +9,14 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.CsvSource; +import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.model.event.exceptions.EventKeyNotFoundException; +import java.math.BigDecimal; import java.time.Instant; import java.util.Arrays; import java.util.Collections; @@ -22,6 +25,7 @@ import java.util.Map; import java.util.Random; import java.util.UUID; +import java.util.stream.Stream; import static org.hamcrest.CoreMatchers.containsStringIgnoringCase; import static org.hamcrest.CoreMatchers.equalTo; @@ -776,6 +780,13 @@ void testJsonStringBuilderWithIncludeKeys() { .getThis() .build(); + // Include Keys must start with / and also ordered, This is pre-processed in SinkModel + List includeNullKey = null; + assertThat(event.jsonBuilder().rootKey(null).includeKeys(includeNullKey).toJsonString(), equalTo(jsonString)); + + List includeEmptyKey = List.of(); + assertThat(event.jsonBuilder().rootKey(null).includeKeys(includeEmptyKey).toJsonString(), equalTo(jsonString)); + // Include Keys must start with / and also ordered, This is pre-processed in SinkModel List includeKeys1 = Arrays.asList("foo", "info"); final String expectedJsonString1 = "{\"foo\":\"bar\",\"info\":{\"name\":\"hello\",\"foo\":\"bar\"}}"; @@ -861,7 +872,16 @@ void testJsonStringBuilderWithExcludeKeys() { } @ParameterizedTest - @CsvSource(value = {"test_key, true", "/test_key, true", "inv(alid, false", "getMetadata(\"test_key\"), false"}) + @CsvSource(value = {"test_key, true", + "/test_key, true", + "inv(alid, false", + "getMetadata(\"test_key\"), false", + "key.with.dot, true", + "key-with-hyphen, true", + "key_with_underscore, true", + "key@with@at, true", + "key[with]brackets, true" + }) void isValidEventKey_returns_expected_result(final String key, final boolean isValid) { assertThat(JacksonEvent.isValidEventKey(key), equalTo(isValid)); } @@ -882,4 +902,24 @@ private static Map createComplexDataMap() { return dataObject; } + @ParameterizedTest + @MethodSource("getBigDecimalPutTestData") + void testPutAndGet_withBigDecimal(final String value) { + final String key = "bigDecimalKey"; + event.put(key, new BigDecimal(value)); + final Object result = event.get(key, Object.class); + assertThat(result, is(notNullValue())); + assertThat(result.toString(), is(equalTo(value))); + } + + private static Stream getBigDecimalPutTestData() { + return Stream.of( + Arguments.of("702062202420"), + Arguments.of("1.23345E+9"), + Arguments.of("1.2345E+60"), + Arguments.of("1.2345E+6"), + Arguments.of("1.000") + ); + } + } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/BigDecimalConverterTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/BigDecimalConverterTests.java new file mode 100644 index 0000000000..5aebb426a2 --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/BigDecimalConverterTests.java @@ -0,0 +1,109 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.typeconverter; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.util.Collections; +import java.util.Map; +import java.util.stream.Stream; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class BigDecimalConverterTests { + @Test + void testStringToBigDecimalConversion() { + BigDecimalConverter converter = new BigDecimalConverter(); + final String stringConstant = "12345678912.12345"; + assertThat(converter.convert(stringConstant), equalTo(new BigDecimal(stringConstant))); + assertThat(converter.convert(stringConstant, () -> 0), equalTo(new BigDecimal(stringConstant))); + } + + @Test + void testIntegerToBigDecimalConversion() { + BigDecimalConverter converter = new BigDecimalConverter(); + final int intConstant = 12345; + assertThat(converter.convert(intConstant), equalTo(BigDecimal.valueOf(intConstant))); + assertThat(converter.convert(intConstant, () -> 0), equalTo(new BigDecimal(intConstant))); + } + + @Test + void testLongToBigDecimalConversion() { + BigDecimalConverter converter = new BigDecimalConverter(); + final long longConstant = 123456789012L; + assertThat(converter.convert(longConstant).longValue(), equalTo(longConstant)); + assertThat(converter.convert(longConstant, () -> 0).longValue(), equalTo(longConstant)); + } + + @Test + void testBooleanToBigDecimalConversion() { + BigDecimalConverter converter = new BigDecimalConverter(); + final Boolean boolFalseConstant = false; + assertThat(converter.convert(boolFalseConstant), equalTo(BigDecimal.valueOf(0))); + final Boolean boolTrueConstant = true; + assertThat(converter.convert(boolTrueConstant), equalTo(BigDecimal.valueOf(1))); + assertThat(converter.convert(boolTrueConstant, () -> 0), equalTo(BigDecimal.valueOf(1))); + } + + @Test + void testFloatToBigDecimalConversion() { + BigDecimalConverter converter = new BigDecimalConverter(); + final float fval = 12345.6789f; + assertThat(converter.convert(fval).floatValue(), equalTo(fval)); + assertThat(converter.convert(fval, () -> 0).floatValue(), equalTo(fval)); + } + + @Test + void testBigDecimalToBigDecimalConversion() { + BigDecimalConverter converter = new BigDecimalConverter(); + BigDecimal bigDecimal = new BigDecimal("12345.6789"); + assertThat(converter.convert(bigDecimal), equalTo(bigDecimal)); + assertThat(converter.convert(bigDecimal, () -> 0), equalTo(bigDecimal)); + } + + @ParameterizedTest + @MethodSource("decimalToBigDecimalValueProvider") + void testDoubleToBigDecimalConversion(BigDecimal expectedBigDecimal, double actualValue, int scale) { + BigDecimalConverter converter = new BigDecimalConverter(); + if(scale!=0) { + expectedBigDecimal = expectedBigDecimal.setScale(scale, RoundingMode.HALF_EVEN); + } + assertThat(converter.convert(actualValue, scale), equalTo(expectedBigDecimal)); + assertThat(converter.convert(actualValue, () -> scale), equalTo(expectedBigDecimal)); + } + + private static Stream decimalToBigDecimalValueProvider() { + return Stream.of( + Arguments.of(new BigDecimal ("0.0"), 0, 1), + Arguments.of(new BigDecimal ("0.0"), 0.0, 1), + Arguments.of(new BigDecimal ("0.00000000000000000000000"), 0.00000000000000000000000, 1), + Arguments.of(BigDecimal.ZERO, BigDecimal.ZERO.doubleValue(), 1), + Arguments.of(new BigDecimal ("1"), (double)1, 1), + Arguments.of(new BigDecimal ("1703908514.045833"), 1703908514.045833, 6), + Arguments.of(new BigDecimal ("1.00000000000000000000000"), 1.00000000000000000000000, 1), + Arguments.of(new BigDecimal ("-12345678912.12345"), -12345678912.12345, 1), + Arguments.of(BigDecimal.ONE, BigDecimal.ONE.doubleValue(), 1), + Arguments.of(new BigDecimal("1.7976931348623157E+308"), 1.7976931348623157E+308, 0), + Arguments.of(new BigDecimal("1702062202420"), 1.70206220242E+12, 12), + Arguments.of(BigDecimal.valueOf(Double.MAX_VALUE), Double.MAX_VALUE, 0), + Arguments.of(BigDecimal.valueOf(Double.MIN_VALUE), Double.MIN_VALUE, 0) + ); + } + + @Test + void testInvalidBigDecimalConversion() { + BigDecimalConverter converter = new BigDecimalConverter(); + final Map map = Collections.emptyMap(); + assertThrows(IllegalArgumentException.class, () -> converter.convert(map)); + } +} diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/BooleanConverterTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/BooleanConverterTests.java index c29bcddb0c..354f7a610f 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/BooleanConverterTests.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/BooleanConverterTests.java @@ -21,11 +21,19 @@ import java.util.stream.Stream; public class BooleanConverterTests { + + @Test + void testStringToBooleanConversionWithArguments() { + BooleanConverter converter = new BooleanConverter(); + final String stringConstant = "12345678912.12345"; + assertThat(converter.convert(stringConstant, () -> 0), equalTo(false)); + } + @Test void testStringToBooleanConversion() { BooleanConverter converter = new BooleanConverter(); final String stringConstant = "12345678912.12345"; - assertThat(converter.convert(stringConstant), equalTo(Boolean.parseBoolean(stringConstant))); + assertThat(converter.convert(stringConstant), equalTo(false)); } @Test void testIntegerToBooleanConversion() { @@ -46,7 +54,7 @@ void testDoubleToBooleanConversion() { @Test void testLongToBooleanConversion() { BooleanConverter converter = new BooleanConverter(); - final Long longTrueConstant = (long)1234578912; + final Long longTrueConstant = 1234578912345L; assertThat(converter.convert(longTrueConstant), equalTo(true)); final Long longFalseConstant = (long)0; assertThat(converter.convert(longFalseConstant), equalTo(false)); diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/DoubleConverterTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/DoubleConverterTests.java index e10121e2c6..8008bfc226 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/DoubleConverterTests.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/DoubleConverterTests.java @@ -25,6 +25,7 @@ void testStringToDoubleConversion() { DoubleConverter converter = new DoubleConverter(); final String stringConstant = "12345678912.12345"; assertThat(converter.convert(stringConstant), equalTo(Double.parseDouble(stringConstant))); + assertThat(converter.convert(stringConstant, () -> 0), equalTo(Double.parseDouble(stringConstant))); } @Test void testIntegerToDoubleConversion() { @@ -43,7 +44,7 @@ void testBooleanToDoubleConversion() { @Test void testDoubleToDoubleConversion() { DoubleConverter converter = new DoubleConverter(); - final Double doubleConstant = (double)12345.123; + final Double doubleConstant = 12345.123; assertThat(converter.convert(doubleConstant), equalTo(doubleConstant)); } @ParameterizedTest @@ -56,16 +57,16 @@ private static Stream BigDecimalValueProvider() { return Stream.of( Arguments.of(new BigDecimal ("0"), (double)0), Arguments.of(new BigDecimal ("0.0"), (double)0), - Arguments.of(new BigDecimal ("0.00000000000000000000000"), (double)0.00000000000000000000000), + Arguments.of(new BigDecimal ("0.00000000000000000000000"), 0.00000000000000000000000), Arguments.of(BigDecimal.ZERO, BigDecimal.ZERO.doubleValue()), Arguments.of(new BigDecimal ("1"), (double)1), - Arguments.of(new BigDecimal ("1703908514.045833"), (double)1703908514.045833), - Arguments.of(new BigDecimal ("1.00000000000000000000000"), (double)1.00000000000000000000000), - Arguments.of(new BigDecimal ("-12345678912.12345"), (double)-12345678912.12345), + Arguments.of(new BigDecimal ("1703908514.045833"), 1703908514.045833), + Arguments.of(new BigDecimal ("1.00000000000000000000000"), 1.00000000000000000000000), + Arguments.of(new BigDecimal ("-12345678912.12345"), -12345678912.12345), Arguments.of(BigDecimal.ONE, BigDecimal.ONE.doubleValue()), - Arguments.of(new BigDecimal("1.7976931348623157E+308"), (double)1.7976931348623157E+308), - Arguments.of(new BigDecimal(Double.MAX_VALUE), (double)Double.MAX_VALUE), - Arguments.of(new BigDecimal(Double.MIN_VALUE), (double)Double.MIN_VALUE) + Arguments.of(new BigDecimal("1.7976931348623157E+308"), 1.7976931348623157E+308), + Arguments.of(new BigDecimal(Double.MAX_VALUE), Double.MAX_VALUE), + Arguments.of(new BigDecimal(Double.MIN_VALUE), Double.MIN_VALUE) ); } @Test diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/IntegerConverterTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/IntegerConverterTests.java index 958c24b8d5..ed993a9ba0 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/IntegerConverterTests.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/IntegerConverterTests.java @@ -23,6 +23,7 @@ void testStringToIntegerConversion() { IntegerConverter converter = new IntegerConverter(); final String stringConstant = "1234"; assertThat(converter.convert(stringConstant), equalTo(Integer.parseInt(stringConstant))); + assertThat(converter.convert(stringConstant, () -> 0), equalTo(Integer.parseInt(stringConstant))); } @Test void testFloatToIntegerConversion() { @@ -41,7 +42,7 @@ void testBooleanToIntegerConversion() { @Test void testIntegerToIntegerConversion() { IntegerConverter converter = new IntegerConverter(); - final Integer intConstant = (int)1234; + final Integer intConstant = 1234; assertThat(converter.convert(intConstant), equalTo(intConstant)); } @ParameterizedTest @@ -52,18 +53,18 @@ void testBigDecimalToIntegerConversion(BigDecimal bigDecimalConstant, int expect } private static Stream BigDecimalValueProvider() { return Stream.of( - Arguments.of(new BigDecimal ("0"), (int)0), - Arguments.of(new BigDecimal ("0.0"), (int)0), - Arguments.of(new BigDecimal ("0.00000000000000000000000"), (int)0), + Arguments.of(new BigDecimal ("0"), 0), + Arguments.of(new BigDecimal ("0.0"), 0), + Arguments.of(new BigDecimal ("0.00000000000000000000000"), 0), Arguments.of(BigDecimal.ZERO, BigDecimal.ZERO.intValue()), - Arguments.of(new BigDecimal ("1"), (int)1), - Arguments.of(new BigDecimal ("1703908514.045833"), (int)1703908514), - Arguments.of(new BigDecimal ("1.00000000000000000000000"), (int)1), - Arguments.of(new BigDecimal ("-12345678.12345"), (int)-12345678), + Arguments.of(new BigDecimal ("1"), 1), + Arguments.of(new BigDecimal ("1703908514.045833"), 1703908514), + Arguments.of(new BigDecimal ("1.00000000000000000000000"), 1), + Arguments.of(new BigDecimal ("-12345678.12345"), -12345678), Arguments.of(BigDecimal.ONE, BigDecimal.ONE.intValue()), - Arguments.of(new BigDecimal("1.7976931348623157E+308"), (int)0), - Arguments.of(new BigDecimal(Integer.MAX_VALUE), (int)Integer.MAX_VALUE), - Arguments.of(new BigDecimal(Integer.MIN_VALUE), (int)Integer.MIN_VALUE) + Arguments.of(new BigDecimal("1.7976931348623157E+308"), 0), + Arguments.of(new BigDecimal(Integer.MAX_VALUE), Integer.MAX_VALUE), + Arguments.of(new BigDecimal(Integer.MIN_VALUE), Integer.MIN_VALUE) ); } @Test diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/LongConverterTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/LongConverterTests.java index 5b72fdf0d9..e1e8ebc424 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/LongConverterTests.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/LongConverterTests.java @@ -24,18 +24,19 @@ public class LongConverterTests { void testStringToLongConversion(String stringValue) { LongConverter converter = new LongConverter(); assertThat(converter.convert(stringValue), equalTo(Long.parseLong(stringValue))); + assertThat(converter.convert(stringValue, () -> 0), equalTo(Long.parseLong(stringValue))); } @ParameterizedTest @ValueSource(floats = {(float)1234.56789, Float.MAX_VALUE, Float.MIN_VALUE}) void testfloatToLongConversion(float floatValue) { LongConverter converter = new LongConverter(); - assertThat(converter.convert(floatValue), equalTo((long)(float)floatValue)); + assertThat(converter.convert(floatValue), equalTo((long) floatValue)); } @ParameterizedTest @ValueSource(doubles = {12345678.12345678, 2.0 * Integer.MAX_VALUE, Double.MAX_VALUE, Double.MIN_VALUE}) void testDoubleToLongConversion(double doubleValue) { LongConverter converter = new LongConverter(); - assertThat(converter.convert(doubleValue), equalTo((long)(double)doubleValue)); + assertThat(converter.convert(doubleValue), equalTo((long) doubleValue)); } @ParameterizedTest @ValueSource(booleans = {false,true}) @@ -77,8 +78,8 @@ private static Stream BigDecimalValueProvider() { Arguments.of(new BigDecimal("1.7976931348623157E+308"), (long)0), Arguments.of(new BigDecimal(Integer.MAX_VALUE), (long)Integer.MAX_VALUE), Arguments.of(new BigDecimal(Integer.MIN_VALUE), (long)Integer.MIN_VALUE), - Arguments.of(new BigDecimal(Long.MAX_VALUE), (long)Long.MAX_VALUE), - Arguments.of(new BigDecimal(Long.MIN_VALUE), (long)Long.MIN_VALUE), + Arguments.of(new BigDecimal(Long.MAX_VALUE), Long.MAX_VALUE), + Arguments.of(new BigDecimal(Long.MIN_VALUE), Long.MIN_VALUE), Arguments.of(new BigDecimal("267694723"), (long)267694723) ); diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/StringConverterTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/StringConverterTests.java index b9ad5c8f7d..b67648976a 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/StringConverterTests.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/typeconverter/StringConverterTests.java @@ -25,6 +25,7 @@ void testLongToStringConversion() { StringConverter converter = new StringConverter(); final Long longConstant = (long)100000000 * (long)10000000; assertThat(converter.convert(longConstant), equalTo(longConstant.toString())); + assertThat(converter.convert(longConstant, () -> 0), equalTo(longConstant.toString())); } @Test void testDoubleToStringConversion() { diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/DataPrepperScalarTypeDeserializerTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/DataPrepperScalarTypeDeserializerTest.java index d0edd69049..8756e97134 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/DataPrepperScalarTypeDeserializerTest.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/DataPrepperScalarTypeDeserializerTest.java @@ -16,6 +16,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import java.io.IOException; +import java.math.BigDecimal; import java.time.Duration; import java.util.stream.Stream; @@ -53,6 +54,7 @@ private static Stream getScalarTypeArguments() { Arguments.of(Long.class, 200L), Arguments.of(Double.class, 1.23d), Arguments.of(Float.class, 2.15f), - Arguments.of(Character.class, 'c')); + Arguments.of(Character.class, 'c'), + Arguments.of(BigDecimal.class, 1.2345E+5)); } } \ No newline at end of file diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/VariableExpanderTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/VariableExpanderTest.java index ab5a125321..386b7fd826 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/VariableExpanderTest.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/VariableExpanderTest.java @@ -21,6 +21,7 @@ import org.opensearch.dataprepper.model.plugin.PluginConfigValueTranslator; import java.io.IOException; +import java.math.BigDecimal; import java.time.Duration; import java.util.Collections; import java.util.Map; @@ -113,6 +114,7 @@ private static Stream getNonStringTypeArguments() { Arguments.of(Long.class, "200", 200L), Arguments.of(Double.class, "1.23", 1.23d), Arguments.of(Float.class, "2.15", 2.15f), + Arguments.of(BigDecimal.class, "2.15", BigDecimal.valueOf(2.15)), Arguments.of(Map.class, "{}", Collections.emptyMap())); } @@ -127,6 +129,7 @@ private static Stream getStringTypeArguments() { Arguments.of(Long.class, "\"200\"", 200L), Arguments.of(Double.class, "\"1.23\"", 1.23d), Arguments.of(Float.class, "\"2.15\"", 2.15f), + Arguments.of(BigDecimal.class, "\"2.15\"", BigDecimal.valueOf(2.15)), Arguments.of(Character.class, "\"c\"", 'c')); } } \ No newline at end of file diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java index 13c28d3345..24f56ef2ba 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java @@ -13,6 +13,7 @@ import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.typeconverter.ConverterArguments; import org.opensearch.dataprepper.typeconverter.TypeConverter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,17 +34,22 @@ public class ConvertEntryTypeProcessor extends AbstractProcessor, private final List nullValues; private final String type; private final List tagsOnFailure; + private int scale = 0; private final ExpressionEvaluator expressionEvaluator; + private final ConverterArguments converterArguments; @DataPrepperPluginConstructor public ConvertEntryTypeProcessor(final PluginMetrics pluginMetrics, final ConvertEntryTypeProcessorConfig convertEntryTypeProcessorConfig, final ExpressionEvaluator expressionEvaluator) { super(pluginMetrics); + this.converterArguments = convertEntryTypeProcessorConfig; this.convertEntryKeys = getKeysToConvert(convertEntryTypeProcessorConfig); - this.type = convertEntryTypeProcessorConfig.getType().name(); - this.converter = convertEntryTypeProcessorConfig.getType().getTargetConverter(); + TargetType targetType = convertEntryTypeProcessorConfig.getType(); + this.type = targetType.name(); + this.converter = targetType.getTargetConverter(); + this.scale = convertEntryTypeProcessorConfig.getScale(); this.convertWhen = convertEntryTypeProcessorConfig.getConvertWhen(); this.nullValues = convertEntryTypeProcessorConfig.getNullValues() .orElse(List.of()); @@ -67,7 +73,7 @@ public Collection> doExecute(final Collection> recor if (keyVal != null) { if (!nullValues.contains(keyVal.toString())) { try { - recordEvent.put(key, converter.convert(keyVal)); + recordEvent.put(key, converter.convert(keyVal, converterArguments)); } catch (final RuntimeException e) { LOG.error(EVENT, "Unable to convert key: {} with value: {} to {}", key, keyVal, type, e); recordEvent.getMetadata().addTags(tagsOnFailure); diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java index 07183f7bcf..448d9bb0a4 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java @@ -6,11 +6,12 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import org.opensearch.dataprepper.typeconverter.ConverterArguments; import java.util.List; import java.util.Optional; -public class ConvertEntryTypeProcessorConfig { +public class ConvertEntryTypeProcessorConfig implements ConverterArguments { @JsonProperty("key") private String key; @@ -20,6 +21,12 @@ public class ConvertEntryTypeProcessorConfig { @JsonProperty("type") private TargetType type = TargetType.INTEGER; + /** + * Optional scale value used only in the case of BigDecimal converter + */ + @JsonProperty("scale") + private int scale = 0; + @JsonProperty("convert_when") private String convertWhen; @@ -35,9 +42,10 @@ public String getKey() { public List getKeys() { return keys; } - public TargetType getType() { - return type; - } + public TargetType getType() { return type; } + + @Override + public int getScale() { return scale; } public String getConvertWhen() { return convertWhen; } diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/TargetType.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/TargetType.java index 74e516f27c..acf67e8702 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/TargetType.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/TargetType.java @@ -7,15 +7,16 @@ import com.fasterxml.jackson.annotation.JsonCreator; import org.opensearch.dataprepper.model.event.DataType; -import org.opensearch.dataprepper.typeconverter.TypeConverter; -import org.opensearch.dataprepper.typeconverter.IntegerConverter; -import org.opensearch.dataprepper.typeconverter.StringConverter; -import org.opensearch.dataprepper.typeconverter.DoubleConverter; +import org.opensearch.dataprepper.typeconverter.BigDecimalConverter; import org.opensearch.dataprepper.typeconverter.BooleanConverter; +import org.opensearch.dataprepper.typeconverter.DoubleConverter; +import org.opensearch.dataprepper.typeconverter.IntegerConverter; import org.opensearch.dataprepper.typeconverter.LongConverter; +import org.opensearch.dataprepper.typeconverter.StringConverter; +import org.opensearch.dataprepper.typeconverter.TypeConverter; -import java.util.Map; import java.util.Arrays; +import java.util.Map; import java.util.stream.Collectors; public enum TargetType { @@ -23,7 +24,8 @@ public enum TargetType { STRING(DataType.STRING, new StringConverter()), DOUBLE(DataType.DOUBLE, new DoubleConverter()), BOOLEAN(DataType.BOOLEAN, new BooleanConverter()), - LONG(DataType.LONG, new LongConverter()); + LONG(DataType.LONG, new LongConverter()), + BIG_DECIMAL(DataType.BIG_DECIMAL, new BigDecimalConverter()); private static final Map OPTIONS_MAP = Arrays.stream(TargetType.values()) .collect(Collectors.toMap( diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java index 0e9ffd9502..5f8b66a6a6 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java @@ -8,6 +8,9 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.expression.ExpressionEvaluator; @@ -16,17 +19,18 @@ import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; +import java.math.BigDecimal; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.UUID; -import java.math.BigDecimal; +import java.util.stream.Stream; -import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @@ -53,21 +57,21 @@ static Record buildRecordWithEvent(final Map data) { } @BeforeEach - private void setup() { + void setup() { lenient().when(mockConfig.getKey()).thenReturn(TEST_KEY); lenient().when(mockConfig.getKeys()).thenReturn(null); lenient().when(mockConfig.getConvertWhen()).thenReturn(null); } - private Record getMessage(String message, String key, Object value) { - final Map testData = new HashMap(); + private Record getMessage(String message, Object value) { + final Map testData = new HashMap<>(); testData.put("message", message); - testData.put(key, value); + testData.put(ConvertEntryTypeProcessorTests.TEST_KEY, value); return buildRecordWithEvent(testData); } private Event executeAndGetProcessedEvent(final Object testValue) { - final Record record = getMessage(UUID.randomUUID().toString(), TEST_KEY, testValue); + final Record record = getMessage(UUID.randomUUID().toString(), testValue); final List> processedRecords = (List>) typeConversionProcessor.doExecute(Collections.singletonList(record)); assertThat(processedRecords.size(), equalTo(1)); assertThat(processedRecords.get(0), notNullValue()); @@ -103,6 +107,50 @@ void testBigDecimalToIntegerConvertEntryTypeProcessor() { assertThat(event.get(TEST_KEY, Integer.class), equalTo(testValue.intValue())); } + @Test + void testDecimalToBigDecimalConvertEntryTypeProcessor() { + BigDecimal testValue = new BigDecimal(Integer.MAX_VALUE); + when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("big_decimal")); + typeConversionProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); + Event event = executeAndGetProcessedEvent(testValue.toString()); + assertThat(event.get(TEST_KEY, BigDecimal.class), equalTo(testValue)); + } + + @Test + void testDecimalToBigDecimalWithScaleConvertEntryTypeProcessor() { + String testValue = "2147483647"; + TargetType bigdecimalTargetType = TargetType.fromOptionValue("big_decimal"); + when(mockConfig.getType()).thenReturn(bigdecimalTargetType); + when(mockConfig.getScale()).thenReturn(5); + typeConversionProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); + Event event = executeAndGetProcessedEvent(testValue); + //As we set the scale to 5, we expect to see 5 positions filled with zeros + assertThat(event.get(TEST_KEY, BigDecimal.class), equalTo(new BigDecimal(testValue+".00000"))); + } + + @ParameterizedTest + @MethodSource("decimalFormatKeysArgumentProvider") + void testDecimalToBigDecimalWithRoundingConvertEntryTypeProcessor(String source, String target) { + + TargetType bigdecimalTargetType = TargetType.fromOptionValue("big_decimal"); + when(mockConfig.getType()).thenReturn(bigdecimalTargetType); + when(mockConfig.getScale()).thenReturn(5); + typeConversionProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); + //Default HALF_ROUND_UP applied for all the conversions + Event event1 = executeAndGetProcessedEvent(source); + assertThat(event1.get(TEST_KEY, BigDecimal.class), equalTo(new BigDecimal(target))); + } + + private static Stream decimalFormatKeysArgumentProvider() { + //Default HALF_ROUND_UP applied for all the conversions + return Stream.of( + Arguments.of("1703908412.707011", "1703908412.70701"), + Arguments.of("1703908412.707016", "1703908412.70702"), + Arguments.of("1703908412.707015", "1703908412.70702"), + Arguments.of("1703908412.707014", "1703908412.70701") + ); + } + @Test void testBooleanToIntegerConvertEntryTypeProcessor() { int testValue = 1; @@ -203,7 +251,7 @@ void testBigDecimalToStringConvertEntryTypeProcessor() { @Test void testDoubleToStringConvertEntryTypeProcessor() { - Double testValue = (double)123.456; + Double testValue = 123.456; String expectedValue = testValue.toString(); when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("string")); typeConversionProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); @@ -242,7 +290,7 @@ void testNoConversionWhenConvertWhenIsFalse() { when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("integer")); when(mockConfig.getConvertWhen()).thenReturn(convertWhen); - final Record record = getMessage(UUID.randomUUID().toString(), TEST_KEY, testValue); + final Record record = getMessage(UUID.randomUUID().toString(), testValue); when(expressionEvaluator.evaluateConditional(convertWhen, record.getData())).thenReturn(false); typeConversionProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); Event event = executeAndGetProcessedEvent(record); @@ -258,11 +306,11 @@ void testMultipleKeysConvertEntryTypeProcessor() { when(mockConfig.getKey()).thenReturn(null); when(mockConfig.getKeys()).thenReturn(List.of(testKey1, testKey2)); when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("string")); - final Map testData = new HashMap(); + final Map testData = new HashMap<>(); testData.put("message", "testMessage"); testData.put(testKey1, testValue); testData.put(testKey2, testValue); - Record record = buildRecordWithEvent(testData); + Record record = buildRecordWithEvent(testData); typeConversionProcessor = new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator); Event event = executeAndGetProcessedEvent(record); assertThat(event.get(testKey1, String.class), equalTo(expectedValue)); diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/TargetTypeTest.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/TargetTypeTest.java index e2565f7e04..0b653fc766 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/TargetTypeTest.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/TargetTypeTest.java @@ -39,7 +39,8 @@ public Stream provideArguments(final ExtensionContext exten arguments(DataType.BOOLEAN.getTypeName(), TargetType.BOOLEAN), arguments(DataType.INTEGER.getTypeName(), TargetType.INTEGER), arguments(DataType.LONG.getTypeName(), TargetType.LONG), - arguments(DataType.DOUBLE.getTypeName(), TargetType.DOUBLE) + arguments(DataType.DOUBLE.getTypeName(), TargetType.DOUBLE), + arguments(DataType.BIG_DECIMAL.getTypeName(), TargetType.BIG_DECIMAL) ); } } From 530be53f58fcb803581f6be272862c939c38e7c8 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Tue, 4 Jun 2024 14:19:48 -0700 Subject: [PATCH 005/159] Modify Key Value processor to support string literal grouping (#4599) * Key Value Processor fixes Signed-off-by: Krishna Kondaka * New options to KV processor Signed-off-by: Krishna Kondaka * Add string literal support Signed-off-by: Krishna Kondaka * Remove unnecessary changes Signed-off-by: Krishna Kondaka * Remove unnecessary changes Signed-off-by: Krishna Kondaka * Fixed tests Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../processor/keyvalue/KeyValueProcessor.java | 94 +++++++++++++++---- .../keyvalue/KeyValueProcessorConfig.java | 26 +++++ .../keyvalue/KeyValueProcessorTests.java | 53 ++++++++++- 3 files changed, 153 insertions(+), 20 deletions(-) diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java index 4af8684729..9f1cd56f20 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java @@ -65,6 +65,7 @@ public class KeyValueProcessor extends AbstractProcessor, Record bracketSet = Set.of('[', ']', '(', ')', '<', '>'); private final List tagsOnFailure; + private final Character stringLiteralCharacter; @DataPrepperPluginConstructor public KeyValueProcessor(final PluginMetrics pluginMetrics, @@ -73,6 +74,8 @@ public KeyValueProcessor(final PluginMetrics pluginMetrics, super(pluginMetrics); this.keyValueProcessorConfig = keyValueProcessorConfig; + this.stringLiteralCharacter = keyValueProcessorConfig.getStringLiteralCharacter(); + tagsOnFailure = keyValueProcessorConfig.getTagsOnFailure(); if (keyValueProcessorConfig.getFieldDelimiterRegex() != null @@ -163,7 +166,7 @@ public KeyValueProcessor(final PluginMetrics pluginMetrics, } validateKeySets(includeKeysSet, excludeKeysSet, defaultValuesSet); - + if (!validTransformOptionSet.contains(keyValueProcessorConfig.getTransformKey())) { throw new IllegalArgumentException(String.format("The transform_key value: %s is not a valid option", keyValueProcessorConfig.getTransformKey())); } @@ -261,11 +264,14 @@ public int skipGroup(final String str, int idx, final Character endChar) { i++; continue; } else if (str.charAt(i) == endChar) { - return i-1; + return i; } else i++; } - throw new RuntimeException("Bad Input, no end character found in "+str+" after index " + idx +", expected end char = "+endChar); + if (keyValueProcessorConfig.isStrictGroupingEnabled()) { + throw new RuntimeException("Bad Input, no end character found in "+str+" after index " + idx +", expected end char = "+endChar); + } + return i-1; } private void addPart(List parts, final String str, final int start, final int end) { @@ -279,7 +285,8 @@ public int findInStartGroup(final String str, int idx) { for (int j = 0; j < startGroupStrings.length; j++) { try { if (startGroupStrings[j].equals(str.substring(idx, idx+startGroupStrings[j].length()))) { - if (j <= 1 && idx > 0 && str.charAt(idx-1) != '\\') { + // For " and ', make sure, it's not escaped + if (j <= 1 && (idx == 0 || str.charAt(idx-1) != '\\')) { return j; } else if (j > 1) { return j; @@ -291,7 +298,7 @@ public int findInStartGroup(final String str, int idx) { } return -1; } - + private List parseWithValueGrouping(String str) { String fieldDelimiter = keyValueProcessorConfig.getFieldSplitCharacters(); Set fieldDelimiterSet = new HashSet<>(); @@ -308,10 +315,22 @@ private List parseWithValueGrouping(String str) { i++; continue; } + int groupIndex = findInStartGroup(str, i); + boolean skippedGroup = false; if (groupIndex >= 0) { - i = skipGroup(str, i+1, endGroupChars[groupIndex])+2; - } else if (fieldDelimiterSet.contains(str.charAt(i))) { + String[] s = keyValueDelimiterPattern.split(str.substring(start,i+1)); + // Only handle Grouping patterns in the values, not keys + if (s.length > 1 || startGroupStrings[groupIndex].charAt(0) == stringLiteralCharacter) { + i = skipGroup(str, i+1, endGroupChars[groupIndex]); + skippedGroup = true; + } + } + if (fieldDelimiterSet.contains(str.charAt(i))) { + // If end of group character is same as field delimiter, then include that in the value if value grouping is done + if (skippedGroup) { + i++; + } addPart(parts, str, start, i); i++; start = i; @@ -322,7 +341,7 @@ private List parseWithValueGrouping(String str) { if (start != i) { addPart(parts, str, start, i); } - + return parts; } @@ -460,7 +479,7 @@ private ObjectNode recurse(final String input, final ObjectMapper mapper) { valueEnd = pair.length() - 1; valueString = pair.substring(valueStart, valueEnd).stripLeading(); JsonNode child = ((ObjectNode) root).put(keyString, recurse(valueString, mapper)); - } + } } else { valueString = pair.substring(valueStart).stripLeading(); ObjectNode child = ((ObjectNode)root).put(keyString, valueString); @@ -484,15 +503,30 @@ private Map createRecursedMap(JsonNode node, ObjectMapper mapper return mapper.convertValue(node, new TypeReference>() {}); } + private boolean isIgnoredGroup(String group) { + // If a group starts and ends with stringLiteralCharacter, + // treat the entire group as key with null as the value + return stringLiteralCharacter != null && + group.charAt(0) == stringLiteralCharacter && + group.charAt(group.length()-1) == stringLiteralCharacter; + } + private Map createNonRecursedMap(String[] groups) { Map nonRecursedMap = new LinkedHashMap<>(); List valueList; for(final String group : groups) { + if (isIgnoredGroup(group)) { + if (validKeyAndValue(group, null)) { + nonRecursedMap.put(group, null); + } + continue; + } + final String[] terms = keyValueDelimiterPattern.split(group, 2); String key = terms[0]; Object value; - + if (terms.length == 2) { value = terms[1]; } else { @@ -508,7 +542,9 @@ private Map createNonRecursedMap(String[] groups) { } else { valueList = new ArrayList(); valueList.add(existingValue); - nonRecursedMap.put(key, valueList); + if (validKeyAndValue(key, valueList)) { + nonRecursedMap.put(key, valueList); + } } if (keyValueProcessorConfig.getSkipDuplicateValues()) { @@ -519,7 +555,9 @@ private Map createNonRecursedMap(String[] groups) { valueList.add(value); } } else { - nonRecursedMap.put(key, value); + if (validKeyAndValue(key, value)) { + nonRecursedMap.put(key, value); + } } } @@ -581,12 +619,11 @@ private Map executeConfigs(Map map) { LOG.debug("Skipping already included default key: '{}'", pair.getKey()); continue; } - processed.put(pair.getKey(), pair.getValue()); + if (validKeyAndValue(pair.getKey(), pair.getValue())) { + processed.put(pair.getKey(), pair.getValue()); + } } - if (keyValueProcessorConfig.getDropKeysWithNoValue()) { - processed.entrySet().removeIf(entry -> entry.getValue() == null); - } return processed; } @@ -594,7 +631,7 @@ private String[] trimWhitespace(String key, Object value) { String[] arr = {key.stripTrailing(), value.toString().stripLeading()}; return arr; } - + private String transformKey(String key) { if (keyValueProcessorConfig.getTransformKey().equals(lowercaseKey)) { key = key.toLowerCase(); @@ -606,9 +643,24 @@ private String transformKey(String key) { return key; } + private boolean validKeyAndValue(String key, Object value) { + if (key == null || key.isEmpty()) { + return false; + } + + if (keyValueProcessorConfig.getDropKeysWithNoValue() && value == null) { + return false; + } + return true; + } + private void addKeyValueToMap(final Map parsedMap, final String key, Object value) { Object processedValue = value; + if (!validKeyAndValue(key, value)) { + return; + } + if (value instanceof List) { List valueAsList = (List) value; if (valueAsList.size() == 1) { @@ -646,8 +698,12 @@ private void addKeyValueToMap(final Map parsedMap, final String private void writeToRoot(final Event event, final Map parsedJson) { for (Map.Entry entry : parsedJson.entrySet()) { - if (keyValueProcessorConfig.getOverwriteIfDestinationExists() || !event.containsKey(entry.getKey())) { - event.put(entry.getKey(), entry.getValue()); + try { + if (keyValueProcessorConfig.getOverwriteIfDestinationExists() || !event.containsKey(entry.getKey())) { + event.put(entry.getKey(), entry.getValue()); + } + } catch (IllegalArgumentException e) { + LOG.warn("Failed to put key: "+entry.getKey()+" value : "+entry.getValue()+" into event. ", e); } } } diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java index 68d09408f3..84cdb868e9 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java @@ -9,6 +9,7 @@ import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.AssertTrue; +import jakarta.validation.constraints.Size; import java.util.ArrayList; import java.util.List; @@ -111,15 +112,40 @@ public class KeyValueProcessorConfig { @JsonProperty("key_value_when") private String keyValueWhen; + @JsonProperty("strict_grouping") + private boolean strictGrouping = false; + + @JsonProperty("string_literal_character") + @Size(min = 0, max = 1, message = "string_literal_character may only have character") + private String stringLiteralCharacter = null; + @AssertTrue(message = "Invalid Configuration. value_grouping option and field_delimiter_regex are mutually exclusive") boolean isValidValueGroupingAndFieldDelimiterRegex() { return (!valueGrouping || fieldDelimiterRegex == null); } + @AssertTrue(message = "Invalid Configuration. String literal character config is valid only when value_grouping is enabled, and only double quote (\") and single quote are (') are valid string literal characters.") + boolean isValidStringLiteralConfig() { + if (stringLiteralCharacter == null) + return true; + if ((!stringLiteralCharacter.equals("\"") && + (!stringLiteralCharacter.equals("'")))) + return false; + return valueGrouping; + } + public String getSource() { return source; } + public Character getStringLiteralCharacter() { + return stringLiteralCharacter == null ? null : stringLiteralCharacter.charAt(0); + } + + public boolean isStrictGroupingEnabled() { + return strictGrouping; + } + public String getDestination() { return destination; } diff --git a/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java b/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java index 019bbb7fab..2adfc37884 100644 --- a/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java +++ b/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java @@ -70,6 +70,7 @@ static Record buildRecordWithEvent(final Map data) { void setup() { final KeyValueProcessorConfig defaultConfig = new KeyValueProcessorConfig(); lenient().when(mockConfig.getSource()).thenReturn(defaultConfig.getSource()); + lenient().when(mockConfig.getStringLiteralCharacter()).thenReturn(null); lenient().when(mockConfig.getDestination()).thenReturn(defaultConfig.getDestination()); lenient().when(mockConfig.getFieldDelimiterRegex()).thenReturn(defaultConfig.getFieldDelimiterRegex()); lenient().when(mockConfig.getFieldSplitCharacters()).thenReturn(defaultConfig.getFieldSplitCharacters()); @@ -187,6 +188,7 @@ void testDropKeysWithNoValue() { @MethodSource("getKeyValueGroupingTestdata") void testMultipleKvToObjectKeyValueProcessorWithValueGrouping(String fieldDelimiters, String input, Map expectedResultMap) { lenient().when(mockConfig.getValueGrouping()).thenReturn(true); + lenient().when(mockConfig.getStringLiteralCharacter()).thenReturn('\"'); lenient().when(mockConfig.getDropKeysWithNoValue()).thenReturn(true); lenient().when(mockConfig.getFieldSplitCharacters()).thenReturn(fieldDelimiters); final KeyValueProcessor objectUnderTest = createObjectUnderTest(); @@ -208,17 +210,66 @@ private static Stream getKeyValueGroupingTestdata() { Arguments.of(", ", "key1=value1, key2=value2", Map.of("key1", "value1", "key2", "value2")), Arguments.of(", ", "key1=It\\'sValue1, key2=value2", Map.of("key1", "It\\'sValue1", "key2", "value2")), Arguments.of(", ", "text1 text2 key1=value1, key2=value2 text3 text4", Map.of("key1", "value1", "key2", "value2")), - Arguments.of(", ", "text1 text2 foo key1=value1 url=http://foo.com?bar=text,text&foo=zoo bar k2=\"http://bar.com?a=b&c=foo bar\" barr", Map.of("key1", "value1", "url", "http://foo.com?bar=text,text&foo=zoo", "k2", "\"http://bar.com?a=b&c=foo bar\"")), + Arguments.of(", ", "text1 text2 foo key1=value1 url=http://foo.com?bar=text,text&foo=zoo bar k2=\"http://bar.com?a=b&c=foo bar\" barr", Map.of("key1", "value1", "url", "http://foo.com?bar=text,text&foo=zoo", "k2", "\"http://bar.com?a=b&c=foo bar\"")), Arguments.of(", ", "vendorMessage=VendorMessage(uid=1847060493-1712778523223, feedValue=https://syosetu.org/novel/147705/15.html, bundleId=, linkType=URL, vendor=DOUBLEVERIFY, platform=DESKTOP, deviceTypeId=1, bidCount=6, appStoreTld=, feedSource=DSP, regions=[APAC], timestamp=1712778523223, externalId=)", Map.of("vendorMessage", "VendorMessage(uid=1847060493-1712778523223, feedValue=https://syosetu.org/novel/147705/15.html, bundleId=, linkType=URL, vendor=DOUBLEVERIFY, platform=DESKTOP, deviceTypeId=1, bidCount=6, appStoreTld=, feedSource=DSP, regions=[APAC], timestamp=1712778523223, externalId=)")), + Arguments.of(", ()", "foo bar(key1=value1, key2=value2, key3=)", Map.of("key1", "value1", "key2", "value2", "key3","")), + Arguments.of(", ", "foo bar(key1=value1, key2=value2, key3=)", Map.of("bar(key1", "value1", "key2", "value2", "key3",")")), + Arguments.of(", ", "foo bar[key1=value1, key2=value2, key3=]", Map.of("bar[key1", "value1", "key2", "value2", "key3","]")), + Arguments.of(", ", "foo bar{key1=value1, key2=value2, key3=}", Map.of("bar{key1", "value1", "key2", "value2", "key3","}")), + Arguments.of(", ", "key1 \"key2=val2\" key3=\"value3,value4\"", Map.of("key3", "\"value3,value4\"")), Arguments.of(", ", "key1=[value1,value2], key3=value3", Map.of("key1", "[value1,value2]", "key3", "value3")), Arguments.of(", ", "key1=(value1, value2), key3=value3", Map.of("key1", "(value1, value2)", "key3", "value3")), Arguments.of(", ", "key1=, key3=value3", Map.of("key1", "", "key3", "value3")), Arguments.of(", ", "key1={value1,value2}, key3=value3", Map.of("key1", "{value1,value2}", "key3", "value3")), Arguments.of(", ", "key1='value1,value2', key3=value3", Map.of("key1", "'value1,value2'", "key3", "value3")), + Arguments.of(", ", "foo key1=val1, key2=val2,key3=val3 bar", Map.of("key1", "val1", "key2", "val2", "key3", "val3")), + Arguments.of(", ", "foo,key1=(val1,key2=val2,val3),key4=val4 bar", Map.of("key1", "(val1,key2=val2,val3)", "key4", "val4")), + Arguments.of(", ", "foo,key1=(val1,key2=val2,val3,key4=val4 bar", Map.of("key1", "(val1,key2=val2,val3,key4=val4 bar")), + + Arguments.of(", ", "foo,key1=[val1,key2=val2,val3],key4=val4 bar", Map.of("key1", "[val1,key2=val2,val3]", "key4", "val4")), + Arguments.of(", ", "foo,key1=[val1,key2=val2,val3,key4=val4 bar", Map.of("key1", "[val1,key2=val2,val3,key4=val4 bar")), + + Arguments.of(", ", "foo,key1={val1,key2=val2,val3},key4=val4 bar", Map.of("key1", "{val1,key2=val2,val3}", "key4", "val4")), + Arguments.of(", ", "foo,key1={val1,key2=val2,val3,key4=val4 bar", Map.of("key1", "{val1,key2=val2,val3,key4=val4 bar")), + + Arguments.of(", ", "foo,key1=,key4=val4 bar", Map.of("key1", "", "key4", "val4")), + Arguments.of(", ", "foo,key1= record = getMessage(message); + keyValueProcessor = createObjectUnderTest(); + final List> editedRecords = (List>) keyValueProcessor.doExecute(Collections.singletonList(record)); + + final Event event = editedRecords.get(0).getData(); + assertThat(event.containsKey("parsed_message"), is(false)); + + assertThat(event.containsKey("key1"), is(true)); + assertThat(event.containsKey("key2"), is(true)); + assertThat(event.get("key1", Object.class), is("value1")); + } + @Test void testWriteToRoot() { when(mockConfig.getDestination()).thenReturn(null); From 2180a69327e2a63fc8e8c84f375b1e4aeb457827 Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Tue, 4 Jun 2024 19:49:07 -0400 Subject: [PATCH 006/159] Refactor http source functionality for supporting a new OpenSearch API source in DataPrepper (#4570) Refactor http source configuration to a separate http source common package. Signed-off-by: Souvik Bose --- .../http-source-common/build.gradle | 29 ++ .../http/BaseHttpServerConfig.java | 260 +++++++++++++ .../dataprepper/http/HttpServerConfig.java | 72 ++++ .../http}/LogThrottlingRejectHandler.java | 2 +- .../http}/LogThrottlingStrategy.java | 2 +- .../CertificateProviderFactory.java | 32 +- .../dataprepper/http}/codec/Codec.java | 2 +- .../dataprepper/http}/codec/JsonCodec.java | 2 +- .../http/BaseHttpServerConfigTest.java | 368 ++++++++++++++++++ .../http}/LogThrottlingRejectHandlerTest.java | 2 +- .../http}/LogThrottlingStrategyTest.java | 2 +- .../CertificateProviderFactoryTest.java | 34 +- .../http}/codec/JsonCodecTest.java | 2 +- .../org.mockito.plugins.MockMaker | 3 + .../src/test/resources/test_cert.crt | 14 + .../src/test/resources/test_decrypted_key.key | 15 + data-prepper-plugins/http-source/build.gradle | 4 +- .../plugins/source/loghttp/HTTPSource.java | 7 +- .../source/loghttp/HTTPSourceConfig.java | 222 +---------- .../source/loghttp/LogHTTPService.java | 2 +- .../source/loghttp/HTTPSourceConfigTest.java | 272 +------------ .../source/loghttp/HTTPSourceTest.java | 1 + settings.gradle | 1 + 23 files changed, 823 insertions(+), 527 deletions(-) create mode 100644 data-prepper-plugins/http-source-common/build.gradle create mode 100644 data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/BaseHttpServerConfig.java create mode 100644 data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/HttpServerConfig.java rename data-prepper-plugins/{http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp => http-source-common/src/main/java/org/opensearch/dataprepper/http}/LogThrottlingRejectHandler.java (96%) rename data-prepper-plugins/{http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp => http-source-common/src/main/java/org/opensearch/dataprepper/http}/LogThrottlingStrategy.java (97%) rename data-prepper-plugins/{http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp => http-source-common/src/main/java/org/opensearch/dataprepper/http}/certificate/CertificateProviderFactory.java (81%) rename data-prepper-plugins/{http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp => http-source-common/src/main/java/org/opensearch/dataprepper/http}/codec/Codec.java (90%) rename data-prepper-plugins/{http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp => http-source-common/src/main/java/org/opensearch/dataprepper/http}/codec/JsonCodec.java (97%) create mode 100644 data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/BaseHttpServerConfigTest.java rename data-prepper-plugins/{http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp => http-source-common/src/test/java/org/opensearch/dataprepper/http}/LogThrottlingRejectHandlerTest.java (97%) rename data-prepper-plugins/{http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp => http-source-common/src/test/java/org/opensearch/dataprepper/http}/LogThrottlingStrategyTest.java (97%) rename data-prepper-plugins/{http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp => http-source-common/src/test/java/org/opensearch/dataprepper/http}/certificate/CertificateProviderFactoryTest.java (70%) rename data-prepper-plugins/{http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp => http-source-common/src/test/java/org/opensearch/dataprepper/http}/codec/JsonCodecTest.java (97%) create mode 100644 data-prepper-plugins/http-source-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker create mode 100644 data-prepper-plugins/http-source-common/src/test/resources/test_cert.crt create mode 100644 data-prepper-plugins/http-source-common/src/test/resources/test_decrypted_key.key diff --git a/data-prepper-plugins/http-source-common/build.gradle b/data-prepper-plugins/http-source-common/build.gradle new file mode 100644 index 0000000000..49b282a1f2 --- /dev/null +++ b/data-prepper-plugins/http-source-common/build.gradle @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +plugins { + id 'java' +} + +dependencies { + implementation project(':data-prepper-plugins:common') + implementation libs.armeria.core + implementation 'software.amazon.awssdk:acm' + implementation 'software.amazon.awssdk:s3' + implementation 'software.amazon.awssdk:apache-client' + testImplementation 'org.assertj:assertj-core:3.25.3' + testImplementation project(':data-prepper-test-common') +} + +jacocoTestCoverageVerification { + dependsOn jacocoTestReport + violationRules { + rule { //in addition to core projects rule + limit { + minimum = 0.90 + } + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/BaseHttpServerConfig.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/BaseHttpServerConfig.java new file mode 100644 index 0000000000..8ad972379a --- /dev/null +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/BaseHttpServerConfig.java @@ -0,0 +1,260 @@ +package org.opensearch.dataprepper.http; + +import com.fasterxml.jackson.annotation.JsonProperty; +import io.micrometer.core.instrument.util.StringUtils; +import jakarta.validation.constraints.AssertTrue; +import jakarta.validation.constraints.Max; +import jakarta.validation.constraints.Min; +import jakarta.validation.constraints.Size; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.codec.CompressionOption; + +/** + * BaseHttpServerConfig class holds the common Http related configurations defined in the customer's source configuration along with default set of configuration values. +*/ +public class BaseHttpServerConfig implements HttpServerConfig { + static final String COMPRESSION = "compression"; + static final String SSL = "ssl"; + static final String SSL_CERTIFICATE_FILE = "ssl_certificate_file"; + static final String SSL_KEY_FILE = "ssl_key_file"; + static final boolean DEFAULT_USE_ACM_CERTIFICATE_FOR_SSL = false; + static final int DEFAULT_ACM_CERTIFICATE_TIMEOUT_MILLIS = 120000; + static final int DEFAULT_REQUEST_TIMEOUT_MS = 10000; + static final double BUFFER_TIMEOUT_FRACTION = 0.8; + static final int DEFAULT_THREAD_COUNT = 200; + static final int DEFAULT_MAX_CONNECTION_COUNT = 500; + static final int DEFAULT_MAX_PENDING_REQUESTS = 1024; + static final boolean DEFAULT_HEALTH_CHECK = false; + static final String HEALTH_CHECK_SERVICE = "health_check_service"; + static final String UNAUTHENTICATED_HEALTH_CHECK = "unauthenticated_health_check"; + static final String S3_PREFIX = "s3://"; + + @JsonProperty("port") + @Min(0) + @Max(65535) + private int port = getDefaultPort(); + + @Override + public int getDefaultPort() { + return 0; + } + + @JsonProperty("path") + @Size(min = 1, message = "path length should be at least 1") + private String path = getDefaultPath(); + + @Override + public String getDefaultPath() { + return ""; + } + + @JsonProperty("request_timeout") + @Min(2) + private int requestTimeoutInMillis = DEFAULT_REQUEST_TIMEOUT_MS; + + @JsonProperty("thread_count") + @Min(0) + private int threadCount = DEFAULT_THREAD_COUNT; + + @JsonProperty("max_connection_count") + @Min(0) + private int maxConnectionCount = DEFAULT_MAX_CONNECTION_COUNT; + + @JsonProperty("max_pending_requests") + @Min(0) + private int maxPendingRequests = DEFAULT_MAX_PENDING_REQUESTS; + + @JsonProperty(SSL) + private boolean ssl; + + @JsonProperty(SSL_CERTIFICATE_FILE) + private String sslCertificateFile; + + @JsonProperty(SSL_KEY_FILE) + private String sslKeyFile; + + @JsonProperty("use_acm_certificate_for_ssl") + private boolean useAcmCertificateForSsl = DEFAULT_USE_ACM_CERTIFICATE_FOR_SSL; + + @JsonProperty("acm_certificate_arn") + private String acmCertificateArn; + + @JsonProperty("acm_private_key_password") + private String acmPrivateKeyPassword; + + @JsonProperty("acm_certificate_timeout_millis") + @Min(0) + private Integer acmCertificateTimeoutMillis = DEFAULT_ACM_CERTIFICATE_TIMEOUT_MILLIS; + + @JsonProperty("aws_region") + private String awsRegion; + + @JsonProperty(HEALTH_CHECK_SERVICE) + private boolean healthCheckService = DEFAULT_HEALTH_CHECK; + + @JsonProperty(UNAUTHENTICATED_HEALTH_CHECK) + private boolean unauthenticatedHealthCheck = false; + + @JsonProperty("max_request_length") + private ByteCount maxRequestLength; + + private PluginModel authentication; + + @JsonProperty(COMPRESSION) + private CompressionOption compression = CompressionOption.NONE; + + @Override + @AssertTrue(message = "path should start with /") + public boolean isPathValid() { + return path.startsWith("/"); + } + + @Override + public int getPort() { + return port; + } + + @Override + public String getPath() { + return path; + } + + @Override + public CompressionOption getCompression() { + return compression; + } + + @Override + public boolean isSslCertAndKeyFileInS3() { + return ssl && sslCertificateFile.toLowerCase().startsWith(S3_PREFIX) && + sslKeyFile.toLowerCase().startsWith(S3_PREFIX); + } + + @AssertTrue(message = "ssl_certificate_file cannot be a empty or null when ssl is enabled") + @Override + public boolean isSslCertificateFileValid() { + if (ssl && !useAcmCertificateForSsl) { + return StringUtils.isNotEmpty(sslCertificateFile); + } + else { + return true; + } + } + + @AssertTrue(message = "ssl_key_file cannot be a empty or null when ssl is enabled") + @Override + public boolean isSslKeyFileValid() { + if (ssl && !useAcmCertificateForSsl) { + return StringUtils.isNotEmpty(sslKeyFile); + } + else { + return true; + } + } + + @AssertTrue(message = "acm_certificate_arn cannot be a empty or null when ACM is used for ssl") + @Override + public boolean isAcmCertificateArnValid() { + if (ssl && useAcmCertificateForSsl) { + return StringUtils.isNotEmpty(acmCertificateArn); + } + else { + return true; + } + } + + @AssertTrue(message = "aws_region cannot be a empty or null when ACM / S3 is used for ssl") + @Override + public boolean isAwsRegionValid() { + if (ssl && (useAcmCertificateForSsl || isSslCertAndKeyFileInS3())) { + return StringUtils.isNotEmpty(awsRegion); + } + return true; + } + + @Override + public int getRequestTimeoutInMillis() { + return requestTimeoutInMillis; + } + + @Override + public int getBufferTimeoutInMillis() { + return (int)(BUFFER_TIMEOUT_FRACTION * requestTimeoutInMillis); + } + + @Override + public int getThreadCount() { + return threadCount; + } + + @Override + public int getMaxConnectionCount() { + return maxConnectionCount; + } + + @Override + public int getMaxPendingRequests() { + return maxPendingRequests; + } + + @Override + public boolean isSsl() { + return ssl; + } + + @Override + public String getSslCertificateFile() { + return sslCertificateFile; + } + + @Override + public String getSslKeyFile() { + return sslKeyFile; + } + + @Override + public boolean isUseAcmCertificateForSsl() { + return useAcmCertificateForSsl; + } + + @Override + public String getAcmCertificateArn() { + return acmCertificateArn; + } + + @Override + public String getAcmPrivateKeyPassword() { + return acmPrivateKeyPassword; + } + + @Override + public int getAcmCertificateTimeoutMillis() { + return acmCertificateTimeoutMillis; + } + + @Override + public String getAwsRegion() { + return awsRegion; + } + + @Override + public PluginModel getAuthentication() { + return authentication; + } + + @Override + public boolean hasHealthCheckService() { + return healthCheckService; + } + + @Override + public boolean isUnauthenticatedHealthCheck() { + return unauthenticatedHealthCheck; + } + + @Override + public ByteCount getMaxRequestLength() { + return maxRequestLength; + } +} diff --git a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/HttpServerConfig.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/HttpServerConfig.java new file mode 100644 index 0000000000..f62c116631 --- /dev/null +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/HttpServerConfig.java @@ -0,0 +1,72 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.http; + +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.plugins.codec.CompressionOption; + +/** + * HttpServerConfig is an interface for the Http based source configurations to be shared across different types of Http based sources +*/ +public interface HttpServerConfig { + + int getDefaultPort(); + + String getDefaultPath(); + + boolean isPathValid(); + + int getPort(); + + String getPath(); + + CompressionOption getCompression(); + + boolean isSslCertAndKeyFileInS3(); + + boolean isSslCertificateFileValid(); + + boolean isSslKeyFileValid(); + + boolean isAcmCertificateArnValid(); + + boolean isAwsRegionValid(); + + int getRequestTimeoutInMillis(); + + int getBufferTimeoutInMillis(); + + int getThreadCount(); + + int getMaxConnectionCount(); + + int getMaxPendingRequests(); + + boolean isSsl(); + + String getSslCertificateFile(); + + String getSslKeyFile(); + + boolean isUseAcmCertificateForSsl(); + + String getAcmCertificateArn(); + + String getAcmPrivateKeyPassword(); + + int getAcmCertificateTimeoutMillis(); + + String getAwsRegion(); + + PluginModel getAuthentication(); + + boolean hasHealthCheckService(); + + boolean isUnauthenticatedHealthCheck(); + + ByteCount getMaxRequestLength(); +} diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingRejectHandler.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/LogThrottlingRejectHandler.java similarity index 96% rename from data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingRejectHandler.java rename to data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/LogThrottlingRejectHandler.java index 2ae46a837f..2df60da3d2 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingRejectHandler.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/LogThrottlingRejectHandler.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.loghttp; +package org.opensearch.dataprepper.http; import org.opensearch.dataprepper.metrics.PluginMetrics; import com.linecorp.armeria.common.HttpRequest; diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingStrategy.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/LogThrottlingStrategy.java similarity index 97% rename from data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingStrategy.java rename to data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/LogThrottlingStrategy.java index bceb7c41a3..32782034a4 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingStrategy.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/LogThrottlingStrategy.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.loghttp; +package org.opensearch.dataprepper.http; import com.linecorp.armeria.common.HttpRequest; import com.linecorp.armeria.common.util.UnmodifiableFuture; diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/certificate/CertificateProviderFactory.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/certificate/CertificateProviderFactory.java similarity index 81% rename from data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/certificate/CertificateProviderFactory.java rename to data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/certificate/CertificateProviderFactory.java index a546766a49..49adaca1ac 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/certificate/CertificateProviderFactory.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/certificate/CertificateProviderFactory.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.loghttp.certificate; +package org.opensearch.dataprepper.http.certificate; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.plugins.certificate.CertificateProvider; @@ -11,7 +11,7 @@ import org.opensearch.dataprepper.plugins.certificate.file.FileCertificateProvider; import org.opensearch.dataprepper.plugins.certificate.s3.S3CertificateProvider; import org.opensearch.dataprepper.plugins.metricpublisher.MicrometerMetricPublisher; -import org.opensearch.dataprepper.plugins.source.loghttp.HTTPSourceConfig; +import org.opensearch.dataprepper.http.HttpServerConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; @@ -36,13 +36,13 @@ public class CertificateProviderFactory { private static final long ACM_CLIENT_BASE_BACKOFF_MILLIS = 1000l; private static final long ACM_CLIENT_MAX_BACKOFF_MILLIS = 60000l; - final HTTPSourceConfig httpSourceConfig; - public CertificateProviderFactory(final HTTPSourceConfig httpSourceConfig) { - this.httpSourceConfig = httpSourceConfig; + final HttpServerConfig httpServerConfig; + public CertificateProviderFactory(final HttpServerConfig httpServerConfig) { + this.httpServerConfig = httpServerConfig; } public CertificateProvider getCertificateProvider() { - if (httpSourceConfig.isUseAcmCertificateForSsl()) { + if (httpServerConfig.isUseAcmCertificateForSsl()) { LOG.info("Using ACM certificate and private key for SSL/TLS."); final AwsCredentialsProvider credentialsProvider = AwsCredentialsProviderChain.builder() .addCredentialsProvider(DefaultCredentialsProvider.create()) @@ -65,7 +65,7 @@ public CertificateProvider getCertificateProvider() { final PluginMetrics awsSdkMetrics = PluginMetrics.fromNames("sdk", "aws"); final AcmClient awsCertificateManager = AcmClient.builder() - .region(Region.of(httpSourceConfig.getAwsRegion())) + .region(Region.of(httpServerConfig.getAwsRegion())) .credentialsProvider(credentialsProvider) .overrideConfiguration(clientConfig) .httpClientBuilder(ApacheHttpClient.builder()) @@ -73,30 +73,30 @@ public CertificateProvider getCertificateProvider() { .build(); return new ACMCertificateProvider(awsCertificateManager, - httpSourceConfig.getAcmCertificateArn(), - httpSourceConfig.getAcmCertificateTimeoutMillis(), - httpSourceConfig.getAcmPrivateKeyPassword()); - } else if (httpSourceConfig.isSslCertAndKeyFileInS3()) { + httpServerConfig.getAcmCertificateArn(), + httpServerConfig.getAcmCertificateTimeoutMillis(), + httpServerConfig.getAcmPrivateKeyPassword()); + } else if (httpServerConfig.isSslCertAndKeyFileInS3()) { LOG.info("Using S3 to fetch certificate and private key for SSL/TLS."); final AwsCredentialsProvider credentialsProvider = AwsCredentialsProviderChain.builder() .addCredentialsProvider(DefaultCredentialsProvider.create()).build(); final S3Client s3Client = S3Client.builder() - .region(Region.of(httpSourceConfig.getAwsRegion())) + .region(Region.of(httpServerConfig.getAwsRegion())) .credentialsProvider(credentialsProvider) .httpClientBuilder(ApacheHttpClient.builder()) .build(); return new S3CertificateProvider( s3Client, - httpSourceConfig.getSslCertificateFile(), - httpSourceConfig.getSslKeyFile() + httpServerConfig.getSslCertificateFile(), + httpServerConfig.getSslKeyFile() ); } else { LOG.info("Using local file system to get certificate and private key for SSL/TLS."); return new FileCertificateProvider( - httpSourceConfig.getSslCertificateFile(), - httpSourceConfig.getSslKeyFile() + httpServerConfig.getSslCertificateFile(), + httpServerConfig.getSslKeyFile() ); } } diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/codec/Codec.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/Codec.java similarity index 90% rename from data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/codec/Codec.java rename to data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/Codec.java index 709727bada..3a71abbd3d 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/codec/Codec.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/Codec.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.loghttp.codec; +package org.opensearch.dataprepper.http.codec; import com.linecorp.armeria.common.HttpData; diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/codec/JsonCodec.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java similarity index 97% rename from data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/codec/JsonCodec.java rename to data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java index b00fc121a2..fc25193a9d 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/codec/JsonCodec.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.loghttp.codec; +package org.opensearch.dataprepper.http.codec; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/BaseHttpServerConfigTest.java b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/BaseHttpServerConfigTest.java new file mode 100644 index 0000000000..d80f107807 --- /dev/null +++ b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/BaseHttpServerConfigTest.java @@ -0,0 +1,368 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.http; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.codec.CompressionOption; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; + +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Stream; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.opensearch.dataprepper.http.BaseHttpServerConfig.S3_PREFIX; + +public class BaseHttpServerConfigTest { + + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String PLUGIN_NAME = "http"; + private static final String USERNAME = "test_user"; + private static final String PASSWORD = "test_password"; + + private static Stream provideCompressionOption() { + return Stream.of(Arguments.of(CompressionOption.GZIP)); + } + + @Test + void testDefault() { + // Prepare + final HttpServerConfig sourceConfig = new BaseHttpServerConfig(); + + // When/Then + assertEquals(BaseHttpServerConfig.DEFAULT_REQUEST_TIMEOUT_MS, sourceConfig.getRequestTimeoutInMillis()); + assertEquals(BaseHttpServerConfig.DEFAULT_THREAD_COUNT, sourceConfig.getThreadCount()); + assertEquals(BaseHttpServerConfig.DEFAULT_MAX_CONNECTION_COUNT, sourceConfig.getMaxConnectionCount()); + assertEquals(BaseHttpServerConfig.DEFAULT_MAX_PENDING_REQUESTS, sourceConfig.getMaxPendingRequests()); + assertEquals(BaseHttpServerConfig.DEFAULT_USE_ACM_CERTIFICATE_FOR_SSL, sourceConfig.isUseAcmCertificateForSsl()); + assertEquals(BaseHttpServerConfig.DEFAULT_ACM_CERTIFICATE_TIMEOUT_MILLIS, sourceConfig.getAcmCertificateTimeoutMillis()); + assertEquals((int)(BaseHttpServerConfig.DEFAULT_REQUEST_TIMEOUT_MS * BaseHttpServerConfig.BUFFER_TIMEOUT_FRACTION), + sourceConfig.getBufferTimeoutInMillis()); + } + + @Test + void getPath_should_return_correct_path() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "path", "/my/custom/path"); + + assertThat(objectUnderTest.isPathValid(), equalTo(true)); + assertThat(objectUnderTest.getPath(), equalTo("/my/custom/path")); + } + + @Test + void isPathValid_should_return_false_for_invalid_path() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "path", "my/custom/path"); + + assertThat(objectUnderTest.isPathValid(), equalTo(false)); + } + + @Test + void testValidPort() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "port", 2021); + + assertThat(objectUnderTest.getPort(), equalTo(2021)); + } + + @Test + void testValidAWSRegion() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "awsRegion", "us-east-1"); + + assertThat(objectUnderTest.getAwsRegion(), equalTo("us-east-1")); + } + + @Test + void testMaxRequestLength() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "maxRequestLength", ByteCount.ofBytes(4)); + + assertThat(objectUnderTest.getMaxRequestLength(), equalTo(ByteCount.ofBytes(4))); + } + + @Test + void testHealthCheckService() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "healthCheckService", true); + + assertEquals(objectUnderTest.hasHealthCheckService(), true); + } + + @Test + void testUnauthenticatedHealthCheck() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "unauthenticatedHealthCheck", true); + + assertThat(objectUnderTest.isUnauthenticatedHealthCheck(), equalTo(true)); + } + + @ParameterizedTest + @MethodSource("provideCompressionOption") + void testValidCompression(final CompressionOption compressionOption) { + // Prepare + final Map settings = new HashMap<>(); + settings.put(BaseHttpServerConfig.COMPRESSION, compressionOption.name()); + + final PluginSetting pluginSetting = new PluginSetting(PLUGIN_NAME, settings); + final BaseHttpServerConfig httpSourceConfig = OBJECT_MAPPER.convertValue( + pluginSetting.getSettings(), BaseHttpServerConfig.class); + + // When/Then + assertEquals(compressionOption, httpSourceConfig.getCompression()); + } + + @Test + void testAuthentication() throws NoSuchFieldException, IllegalAccessException { + PluginModel authentication = new PluginModel("http_basic", + Map.of( + "username", USERNAME, + "password", PASSWORD + )); + + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "authentication", authentication); + + assertThat(objectUnderTest.getAuthentication(), equalTo(authentication)); + } + + @Nested + class SslValidationWithFile { + @Test + void isSslCertificateFileValidation_should_return_true_if_ssl_is_false() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", false); + + assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSsl(), equalTo(false)); + } + + @Test + void isSslCertificateFileValidation_should_return_false_if_ssl_is_true_and_sslCertificateFile_is_null() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + + assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(false)); + assertThat(objectUnderTest.isSsl(), equalTo(true)); + } + + @Test + void isSslCertificateFileValidation_should_return_true_if_ssl_is_true_and_sslCertificateFile_is_a_valid_file() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + + final String sslCertificateFile = UUID.randomUUID().toString(); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslCertificateFile", sslCertificateFile); + + assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); + assertThat(objectUnderTest.getSslCertificateFile(), equalTo(sslCertificateFile)); + } + + @Test + void isSslKeyFileValidation_should_return_true_if_ssl_is_false() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", false); + + assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); + } + + @Test + void isSslKeyFileValidation_should_return_false_if_ssl_is_true_and_sslKeyFile_is_null() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + + assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(false)); + } + + @Test + void isSslKeyFileValidation_should_return_true_if_ssl_is_true_and_sslKeyFile_is_a_valid_file() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + + final String sslKeyFile = UUID.randomUUID().toString(); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslKeyFile", sslKeyFile); + + assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); + assertThat(objectUnderTest.getSslKeyFile(), equalTo(sslKeyFile)); + } + + } + + @Nested + class SslValidationWithS3 { + @Test + void isSslCertAndKeyFileInS3_should_return_true_if_ssl_is_true_and_KeyFile_and_certFile_are_s3_paths() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslCertificateFile", getS3FilePath()); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslKeyFile", getS3FilePath()); + + assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(true)); + } + + @Test + void isSslCertAndKeyFileInS3_should_return_false_if_ssl_is_true_and_KeyFile_and_certFile_are_not_s3_paths() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslCertificateFile", UUID.randomUUID().toString()); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslKeyFile", UUID.randomUUID().toString()); + + assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(false)); + } + + @Test + void isAwsRegionValid_should_return_true_if_ssl_is_true_and_aws_region_is_null_without_s3_paths() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslCertificateFile", UUID.randomUUID().toString()); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslKeyFile", UUID.randomUUID().toString()); + + assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(false)); + assertThat(objectUnderTest.isAwsRegionValid(), equalTo(true)); + } + + @Test + void isAwsRegionValid_should_return_false_if_ssl_is_true_and_aws_region_is_null_with_s3_paths() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslCertificateFile", getS3FilePath()); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslKeyFile", getS3FilePath()); + + assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(true)); + assertThat(objectUnderTest.isAwsRegionValid(), equalTo(false)); + } + + @Test + void isAwsRegionValid_should_return_true_if_ssl_is_true_and_aws_region_is_not_null_with_s3_paths() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslCertificateFile", getS3FilePath()); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslKeyFile", getS3FilePath()); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "awsRegion", UUID.randomUUID().toString()); + + assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(true)); + assertThat(objectUnderTest.isAwsRegionValid(), equalTo(true)); + } + + @Test + void isAwsRegionValid_should_return_false_if_ssl_is_true_and_aws_region_is_not_null_with_s3_paths() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslCertificateFile", getS3FilePath()); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "sslKeyFile", getS3FilePath()); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "awsRegion", UUID.randomUUID().toString()); + + assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); + assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(true)); + assertThat(objectUnderTest.isAwsRegionValid(), equalTo(true)); + } + } + + @Nested + class SslValidationWithAcm { + @Test + void isAwsRegionValid_should_return_false_if_ssl_is_true_and_aws_region_is_null_with_acm() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "useAcmCertificateForSsl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "acmCertificateArn", "acm-certificate-arn"); + + assertThat(objectUnderTest.isAwsRegionValid(), equalTo(false)); + assertThat(objectUnderTest.getAcmCertificateArn(), equalTo("acm-certificate-arn")); + } + + @Test + void isAwsRegionValid_should_return_true_if_ssl_is_true_and_aws_region_is_not_null_with_acm() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "useAcmCertificateForSsl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "awsRegion", UUID.randomUUID().toString()); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "acmCertificateArn", "acm-certificate-arn"); + + assertThat(objectUnderTest.isAwsRegionValid(), equalTo(true)); + assertThat(objectUnderTest.getAcmCertificateArn(), equalTo("acm-certificate-arn")); + } + + @Test + void isAcmCertificateArnValid_should_return_false_if_ssl_is_true_and_acm_is_true_and_arn_is_null() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "useAcmCertificateForSsl", true); + + assertThat(objectUnderTest.isAcmCertificateArnValid(), equalTo(false)); + } + + @Test + void isAcmCertificateArnValid_should_return_true_if_ssl_is_false_and_acm_is_true_and_arn_is_null() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", false); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "useAcmCertificateForSsl", true); + + assertThat(objectUnderTest.isAcmCertificateArnValid(), equalTo(true)); + } + + @Test + void isAcmCertificateArnValid_should_return_true_if_ssl_is_true_and_acm_is_true_and_arn_is_not_null() throws NoSuchFieldException, IllegalAccessException { + final BaseHttpServerConfig objectUnderTest = new BaseHttpServerConfig(); + + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "ssl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "useAcmCertificateForSsl", true); + ReflectivelySetField.setField(BaseHttpServerConfig.class, objectUnderTest, "acmCertificateArn", UUID.randomUUID().toString()); + + assertThat(objectUnderTest.isAcmCertificateArnValid(), equalTo(true)); + } + } + + private String getS3FilePath() { + return S3_PREFIX.concat(UUID.randomUUID().toString()); + } +} diff --git a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingRejectHandlerTest.java b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/LogThrottlingRejectHandlerTest.java similarity index 97% rename from data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingRejectHandlerTest.java rename to data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/LogThrottlingRejectHandlerTest.java index d0b7823fd3..cc6cbd8b28 100644 --- a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingRejectHandlerTest.java +++ b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/LogThrottlingRejectHandlerTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.loghttp; +package org.opensearch.dataprepper.http; import org.opensearch.dataprepper.metrics.PluginMetrics; import com.linecorp.armeria.common.AggregatedHttpResponse; diff --git a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingStrategyTest.java b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/LogThrottlingStrategyTest.java similarity index 97% rename from data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingStrategyTest.java rename to data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/LogThrottlingStrategyTest.java index 0a1d3028ef..2edbedaeec 100644 --- a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogThrottlingStrategyTest.java +++ b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/LogThrottlingStrategyTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.loghttp; +package org.opensearch.dataprepper.http; import com.linecorp.armeria.common.HttpRequest; import com.linecorp.armeria.common.util.UnmodifiableFuture; diff --git a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/certificate/CertificateProviderFactoryTest.java b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/certificate/CertificateProviderFactoryTest.java similarity index 70% rename from data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/certificate/CertificateProviderFactoryTest.java rename to data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/certificate/CertificateProviderFactoryTest.java index 8b1122aa07..f309cec2b7 100644 --- a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/certificate/CertificateProviderFactoryTest.java +++ b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/certificate/CertificateProviderFactoryTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.loghttp.certificate; +package org.opensearch.dataprepper.http.certificate; import org.hamcrest.core.IsInstanceOf; import org.junit.jupiter.api.BeforeEach; @@ -12,7 +12,7 @@ import org.opensearch.dataprepper.plugins.certificate.acm.ACMCertificateProvider; import org.opensearch.dataprepper.plugins.certificate.file.FileCertificateProvider; import org.opensearch.dataprepper.plugins.certificate.s3.S3CertificateProvider; -import org.opensearch.dataprepper.plugins.source.loghttp.HTTPSourceConfig; +import org.opensearch.dataprepper.http.HttpServerConfig; import static org.hamcrest.MatcherAssert.assertThat; import static org.mockito.Mockito.mock; @@ -22,21 +22,21 @@ class CertificateProviderFactoryTest { private final String TEST_SSL_CERTIFICATE_FILE = getClass().getClassLoader().getResource("test_cert.crt").getFile(); private final String TEST_SSL_KEY_FILE = getClass().getClassLoader().getResource("test_decrypted_key.key").getFile(); - private HTTPSourceConfig httpSourceConfig; + private HttpServerConfig httpServerConfig; private CertificateProviderFactory certificateProviderFactory; @BeforeEach void setUp() { - httpSourceConfig = mock(HTTPSourceConfig.class); + httpServerConfig = mock(HttpServerConfig.class); } @Test void getCertificateProviderFileCertificateProviderSuccess() { - when(httpSourceConfig.isSsl()).thenReturn(true); - when(httpSourceConfig.getSslCertificateFile()).thenReturn(TEST_SSL_CERTIFICATE_FILE); - when(httpSourceConfig.getSslKeyFile()).thenReturn(TEST_SSL_KEY_FILE); + when(httpServerConfig.isSsl()).thenReturn(true); + when(httpServerConfig.getSslCertificateFile()).thenReturn(TEST_SSL_CERTIFICATE_FILE); + when(httpServerConfig.getSslKeyFile()).thenReturn(TEST_SSL_KEY_FILE); - certificateProviderFactory = new CertificateProviderFactory(httpSourceConfig); + certificateProviderFactory = new CertificateProviderFactory(httpServerConfig); final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); assertThat(certificateProvider, IsInstanceOf.instanceOf(FileCertificateProvider.class)); @@ -44,12 +44,12 @@ void getCertificateProviderFileCertificateProviderSuccess() { @Test void getCertificateProviderS3ProviderSuccess() { - when(httpSourceConfig.isSslCertAndKeyFileInS3()).thenReturn(true); - when(httpSourceConfig.getAwsRegion()).thenReturn("us-east-1"); - when(httpSourceConfig.getSslCertificateFile()).thenReturn("s3://data/certificate/test_cert.crt"); - when(httpSourceConfig.getSslKeyFile()).thenReturn("s3://data/certificate/test_decrypted_key.key"); + when(httpServerConfig.isSslCertAndKeyFileInS3()).thenReturn(true); + when(httpServerConfig.getAwsRegion()).thenReturn("us-east-1"); + when(httpServerConfig.getSslCertificateFile()).thenReturn("s3://data/certificate/test_cert.crt"); + when(httpServerConfig.getSslKeyFile()).thenReturn("s3://data/certificate/test_decrypted_key.key"); - certificateProviderFactory = new CertificateProviderFactory(httpSourceConfig); + certificateProviderFactory = new CertificateProviderFactory(httpServerConfig); final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); assertThat(certificateProvider, IsInstanceOf.instanceOf(S3CertificateProvider.class)); @@ -57,11 +57,11 @@ void getCertificateProviderS3ProviderSuccess() { @Test void getCertificateProviderAcmProviderSuccess() { - when(httpSourceConfig.isUseAcmCertificateForSsl()).thenReturn(true); - when(httpSourceConfig.getAwsRegion()).thenReturn("us-east-1"); - when(httpSourceConfig.getAcmCertificateArn()).thenReturn("arn:aws:acm:us-east-1:account:certificate/1234-567-856456"); + when(httpServerConfig.isUseAcmCertificateForSsl()).thenReturn(true); + when(httpServerConfig.getAwsRegion()).thenReturn("us-east-1"); + when(httpServerConfig.getAcmCertificateArn()).thenReturn("arn:aws:acm:us-east-1:account:certificate/1234-567-856456"); - certificateProviderFactory = new CertificateProviderFactory(httpSourceConfig); + certificateProviderFactory = new CertificateProviderFactory(httpServerConfig); final CertificateProvider certificateProvider = certificateProviderFactory.getCertificateProvider(); assertThat(certificateProvider, IsInstanceOf.instanceOf(ACMCertificateProvider.class)); diff --git a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/codec/JsonCodecTest.java b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java similarity index 97% rename from data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/codec/JsonCodecTest.java rename to data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java index 12b3400906..4863667bc0 100644 --- a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/codec/JsonCodecTest.java +++ b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.loghttp.codec; +package org.opensearch.dataprepper.http.codec; import com.linecorp.armeria.common.HttpData; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/http-source-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/http-source-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 0000000000..78ccc25012 --- /dev/null +++ b/data-prepper-plugins/http-source-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,3 @@ +# To enable mocking of final classes with vanilla Mockito +# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods +mock-maker-inline \ No newline at end of file diff --git a/data-prepper-plugins/http-source-common/src/test/resources/test_cert.crt b/data-prepper-plugins/http-source-common/src/test/resources/test_cert.crt new file mode 100644 index 0000000000..26c78d1411 --- /dev/null +++ b/data-prepper-plugins/http-source-common/src/test/resources/test_cert.crt @@ -0,0 +1,14 @@ +-----BEGIN CERTIFICATE----- +MIICHTCCAYYCCQD4hqYeYDQZADANBgkqhkiG9w0BAQUFADBSMQswCQYDVQQGEwJV +UzELMAkGA1UECAwCVFgxDzANBgNVBAcMBkF1c3RpbjEPMA0GA1UECgwGQW1hem9u +MRQwEgYDVQQLDAtEYXRhcHJlcHBlcjAgFw0yMTA2MjUxOTIzMTBaGA8yMTIxMDYw +MTE5MjMxMFowUjELMAkGA1UEBhMCVVMxCzAJBgNVBAgMAlRYMQ8wDQYDVQQHDAZB +dXN0aW4xDzANBgNVBAoMBkFtYXpvbjEUMBIGA1UECwwLRGF0YXByZXBwZXIwgZ8w +DQYJKoZIhvcNAQEBBQADgY0AMIGJAoGBAKrb3YhdKbQ5PtLHall10iLZC9ZdDVrq +HOvqVSM8NHlL8f82gJ3l0n9k7hYc5eKisutaS9eDTmJ+Dnn8xn/qPSKTIq9Wh+OZ +O+e9YEEpI/G4F9KpGULgMyRg9sJK0GlZdEt9o5GJNJIJUkptJU5eiLuE0IV+jyJo +Nvm8OE6EJPqxAgMBAAEwDQYJKoZIhvcNAQEFBQADgYEAjgnX5n/Tt7eo9uakIGAb +uBhvYdR8JqKXqF9rjFJ/MIK7FdQSF/gCdjnvBhzLlZFK/Nb6MGKoSKm5Lcr75LgC +FyhIwp3WlqQksiMFnOypYVY71vqDgj6UKdMaOBgthsYhngj8lC+wsVzWqQvkJ2Qg +/GAIzJwiZfXiaevQHRk79qI= +-----END CERTIFICATE----- diff --git a/data-prepper-plugins/http-source-common/src/test/resources/test_decrypted_key.key b/data-prepper-plugins/http-source-common/src/test/resources/test_decrypted_key.key new file mode 100644 index 0000000000..479b877131 --- /dev/null +++ b/data-prepper-plugins/http-source-common/src/test/resources/test_decrypted_key.key @@ -0,0 +1,15 @@ +-----BEGIN RSA PRIVATE KEY----- +MIICXAIBAAKBgQCq292IXSm0OT7Sx2pZddIi2QvWXQ1a6hzr6lUjPDR5S/H/NoCd +5dJ/ZO4WHOXiorLrWkvXg05ifg55/MZ/6j0ikyKvVofjmTvnvWBBKSPxuBfSqRlC +4DMkYPbCStBpWXRLfaORiTSSCVJKbSVOXoi7hNCFfo8iaDb5vDhOhCT6sQIDAQAB +AoGANrrhFqpJDpr7vcb1ER0Fp/YArbT27zVo+EUC6puBb41dQlQyFOImcHpjLaAq +H1PgnjU5cBp2hGQ+vOK0rwrYc/HNl6vfh6N3NbDptMiuoBafRJA9JzYourAM09BU +zmXyr61Yn3KHzx1PRwWe37icX93oXP3P0qHb3dI1ZF4jG0ECQQDU5N/a7ogoz2zn +ZssD6FvUOUQDsdBWdXmhUvg+YdZrV44e4xk+FVzwEONoRktEYKz9MFXlsgNHr445 +KRguHWcJAkEAzXQkwOkN8WID1wrwoobUIMbZSGAZzofwkKXgTTnllnT1qOQXuRbS +aCMejFEymBBef4aXP6N4+va2FKW/MF34aQJAO2oMl1sOoOUSrZngepy0VAwPUUCk +thxe74jqQu6nGpn6zd/vQYZQw6bS8Fz90H1yic6dilcd1znFZWp0lxoZkQJBALeI +xoBycRsuFQIYasi1q3AwUtBd0Q/3zkZZeBtk2hzjFMUwJaUZpxKSNOrialD/ZnuD +jz+xWBTRKe0d98JMX+kCQCmsJEj/HYQAC1GamZ7JQWogRSRF2KTgTWRaDXDxy0d4 +yUQgwHB+HZLFcbi1JEK6eIixCsX8iifrrkteh+1npJ0= +-----END RSA PRIVATE KEY----- diff --git a/data-prepper-plugins/http-source/build.gradle b/data-prepper-plugins/http-source/build.gradle index 459513f6c7..7d54d5f177 100644 --- a/data-prepper-plugins/http-source/build.gradle +++ b/data-prepper-plugins/http-source/build.gradle @@ -10,6 +10,7 @@ plugins { dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:blocking-buffer') + implementation project(':data-prepper-plugins:http-source-common') implementation project(':data-prepper-plugins:common') implementation project(':data-prepper-plugins:armeria-common') implementation libs.armeria.core @@ -17,8 +18,9 @@ dependencies { implementation 'software.amazon.awssdk:acm' implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:apache-client' - testImplementation project(':data-prepper-api').sourceSets.test.output testImplementation 'org.assertj:assertj-core:3.25.3' + testImplementation project(':data-prepper-api').sourceSets.test.output + testImplementation project(':data-prepper-test-common') } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSource.java b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSource.java index 5f7ad0ecfa..cea9e252f6 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSource.java +++ b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSource.java @@ -13,6 +13,9 @@ import com.linecorp.armeria.server.throttling.ThrottlingService; import org.opensearch.dataprepper.HttpRequestExceptionHandler; import org.opensearch.dataprepper.armeria.authentication.ArmeriaHttpAuthenticationProvider; +import org.opensearch.dataprepper.http.HttpServerConfig; +import org.opensearch.dataprepper.http.LogThrottlingRejectHandler; +import org.opensearch.dataprepper.http.LogThrottlingStrategy; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; @@ -29,7 +32,7 @@ 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.source.loghttp.certificate.CertificateProviderFactory; +import org.opensearch.dataprepper.http.certificate.CertificateProviderFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +52,7 @@ public class HTTPSource implements Source> { public static final String REGEX_HEALTH = "regex:^/(?!health$).*$"; static final String SERVER_CONNECTIONS = "serverConnections"; - private final HTTPSourceConfig sourceConfig; + private final HttpServerConfig sourceConfig; private final CertificateProviderFactory certificateProviderFactory; private final ArmeriaHttpAuthenticationProvider authenticationProvider; private final HttpRequestExceptionHandler httpRequestExceptionHandler; diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceConfig.java b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceConfig.java index c8ad8397d0..9ab52afce0 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceConfig.java +++ b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceConfig.java @@ -5,224 +5,20 @@ package org.opensearch.dataprepper.plugins.source.loghttp; -import jakarta.validation.constraints.Size; -import org.opensearch.dataprepper.model.types.ByteCount; -import org.opensearch.dataprepper.plugins.codec.CompressionOption; -import org.opensearch.dataprepper.model.configuration.PluginModel; -import com.fasterxml.jackson.annotation.JsonProperty; -import io.micrometer.core.instrument.util.StringUtils; -import jakarta.validation.constraints.AssertTrue; -import jakarta.validation.constraints.Max; -import jakarta.validation.constraints.Min; +import org.opensearch.dataprepper.http.BaseHttpServerConfig; + +public class HTTPSourceConfig extends BaseHttpServerConfig { -public class HTTPSourceConfig { static final String DEFAULT_LOG_INGEST_URI = "/log/ingest"; - static final String SSL = "ssl"; - static final String SSL_CERTIFICATE_FILE = "ssl_certificate_file"; - static final String SSL_KEY_FILE = "ssl_key_file"; - static final String COMPRESSION = "compression"; - static final boolean DEFAULT_USE_ACM_CERTIFICATE_FOR_SSL = false; - static final int DEFAULT_ACM_CERTIFICATE_TIMEOUT_MILLIS = 120000; static final int DEFAULT_PORT = 2021; - static final int DEFAULT_REQUEST_TIMEOUT_MS = 10000; - static final double BUFFER_TIMEOUT_FRACTION = 0.8; - static final int DEFAULT_THREAD_COUNT = 200; - static final int DEFAULT_MAX_CONNECTION_COUNT = 500; - static final int DEFAULT_MAX_PENDING_REQUESTS = 1024; - static final boolean DEFAULT_HEALTH_CHECK = false; - static final String HEALTH_CHECK_SERVICE = "health_check_service"; - static final String UNAUTHENTICATED_HEALTH_CHECK = "unauthenticated_health_check"; - static final String S3_PREFIX = "s3://"; - - @JsonProperty("port") - @Min(0) - @Max(65535) - private int port = DEFAULT_PORT; - - @JsonProperty("path") - @Size(min = 1, message = "path length should be at least 1") - private String path = DEFAULT_LOG_INGEST_URI; - - @JsonProperty("request_timeout") - @Min(2) - private int requestTimeoutInMillis = DEFAULT_REQUEST_TIMEOUT_MS; - - @JsonProperty("thread_count") - @Min(0) - private int threadCount = DEFAULT_THREAD_COUNT; - - @JsonProperty("max_connection_count") - @Min(0) - private int maxConnectionCount = DEFAULT_MAX_CONNECTION_COUNT; - - @JsonProperty("max_pending_requests") - @Min(0) - private int maxPendingRequests = DEFAULT_MAX_PENDING_REQUESTS; - - @JsonProperty(SSL) - private boolean ssl; - - @JsonProperty(SSL_CERTIFICATE_FILE) - private String sslCertificateFile; - - @JsonProperty(SSL_KEY_FILE) - private String sslKeyFile; - - @JsonProperty("use_acm_certificate_for_ssl") - private boolean useAcmCertificateForSsl = DEFAULT_USE_ACM_CERTIFICATE_FOR_SSL; - - @JsonProperty("acm_certificate_arn") - private String acmCertificateArn; - - @JsonProperty("acm_private_key_password") - private String acmPrivateKeyPassword; - - @JsonProperty("acm_certificate_timeout_millis") - @Min(0) - private Integer acmCertificateTimeoutMillis = DEFAULT_ACM_CERTIFICATE_TIMEOUT_MILLIS; - - @JsonProperty("aws_region") - private String awsRegion; - - @JsonProperty(HEALTH_CHECK_SERVICE) - private boolean healthCheckService = DEFAULT_HEALTH_CHECK; - - @JsonProperty(UNAUTHENTICATED_HEALTH_CHECK) - private boolean unauthenticatedHealthCheck = false; - - @JsonProperty(COMPRESSION) - private CompressionOption compression = CompressionOption.NONE; - - @JsonProperty("max_request_length") - private ByteCount maxRequestLength; - - private PluginModel authentication; - - public boolean isSslCertAndKeyFileInS3() { - return ssl && sslCertificateFile.toLowerCase().startsWith(S3_PREFIX) && - sslKeyFile.toLowerCase().startsWith(S3_PREFIX); - } - - @AssertTrue(message = "path should start with /") - boolean isPathValid() { - return path.startsWith("/"); - } - - @AssertTrue(message = "ssl_certificate_file cannot be a empty or null when ssl is enabled") - boolean isSslCertificateFileValid() { - if (ssl && !useAcmCertificateForSsl) { - return StringUtils.isNotEmpty(sslCertificateFile); - } - else { - return true; - } - } - - @AssertTrue(message = "ssl_key_file cannot be a empty or null when ssl is enabled") - boolean isSslKeyFileValid() { - if (ssl && !useAcmCertificateForSsl) { - return StringUtils.isNotEmpty(sslKeyFile); - } - else { - return true; - } - } - - @AssertTrue(message = "acm_certificate_arn cannot be a empty or null when ACM is used for ssl") - boolean isAcmCertificateArnValid() { - if (ssl && useAcmCertificateForSsl) { - return StringUtils.isNotEmpty(acmCertificateArn); - } - else { - return true; - } - } - - @AssertTrue(message = "aws_region cannot be a empty or null when ACM / S3 is used for ssl") - boolean isAwsRegionValid() { - if (ssl && (useAcmCertificateForSsl || isSslCertAndKeyFileInS3())) { - return StringUtils.isNotEmpty(awsRegion); - } - return true; - } - - public int getPort() { - return port; - } - - public String getPath() { - return path; - } - - public int getRequestTimeoutInMillis() { - return requestTimeoutInMillis; - } - - public int getBufferTimeoutInMillis() { - return (int)(BUFFER_TIMEOUT_FRACTION * requestTimeoutInMillis); - } - - public int getThreadCount() { - return threadCount; - } - - public int getMaxConnectionCount() { - return maxConnectionCount; - } - - public int getMaxPendingRequests() { - return maxPendingRequests; - } - - public boolean isSsl() { - return ssl; - } - - public String getSslCertificateFile() { - return sslCertificateFile; - } - - public String getSslKeyFile() { - return sslKeyFile; - } - - public boolean isUseAcmCertificateForSsl() { - return useAcmCertificateForSsl; - } - - public String getAcmCertificateArn() { - return acmCertificateArn; - } - - public String getAcmPrivateKeyPassword() { - return acmPrivateKeyPassword; - } - - public int getAcmCertificateTimeoutMillis() { - return acmCertificateTimeoutMillis; - } - - public String getAwsRegion() { - return awsRegion; - } - - public PluginModel getAuthentication() { - return authentication; - } - - public boolean hasHealthCheckService() { - return healthCheckService; - } - - public boolean isUnauthenticatedHealthCheck() { - return unauthenticatedHealthCheck; - } - public CompressionOption getCompression() { - return compression; + @Override + public int getDefaultPort() { + return DEFAULT_PORT; } - public ByteCount getMaxRequestLength() { - return maxRequestLength; + @Override + public String getDefaultPath() { + return DEFAULT_LOG_INGEST_URI; } } diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java index 5b7ec1d152..8384315aa4 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java +++ b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java @@ -21,7 +21,7 @@ import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.DistributionSummary; import io.micrometer.core.instrument.Timer; -import org.opensearch.dataprepper.plugins.source.loghttp.codec.JsonCodec; +import org.opensearch.dataprepper.http.codec.JsonCodec; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceConfigTest.java b/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceConfigTest.java index c1671f5ac0..bf05e6b6b2 100644 --- a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceConfigTest.java +++ b/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceConfigTest.java @@ -5,33 +5,11 @@ package org.opensearch.dataprepper.plugins.source.loghttp; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; -import org.opensearch.dataprepper.plugins.codec.CompressionOption; -import org.opensearch.dataprepper.model.configuration.PluginSetting; -import java.lang.reflect.Field; -import java.util.HashMap; -import java.util.Map; -import java.util.UUID; -import java.util.stream.Stream; - -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.opensearch.dataprepper.plugins.source.loghttp.HTTPSourceConfig.S3_PREFIX; public class HTTPSourceConfigTest { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final String PLUGIN_NAME = "http"; - - private static Stream provideCompressionOption() { - return Stream.of(Arguments.of(CompressionOption.GZIP)); - } @Test void testDefault() { @@ -41,254 +19,8 @@ void testDefault() { // When/Then assertEquals(HTTPSourceConfig.DEFAULT_PORT, sourceConfig.getPort()); assertEquals(HTTPSourceConfig.DEFAULT_LOG_INGEST_URI, sourceConfig.getPath()); - assertEquals(HTTPSourceConfig.DEFAULT_REQUEST_TIMEOUT_MS, sourceConfig.getRequestTimeoutInMillis()); - assertEquals(HTTPSourceConfig.DEFAULT_THREAD_COUNT, sourceConfig.getThreadCount()); - assertEquals(HTTPSourceConfig.DEFAULT_MAX_CONNECTION_COUNT, sourceConfig.getMaxConnectionCount()); - assertEquals(HTTPSourceConfig.DEFAULT_MAX_PENDING_REQUESTS, sourceConfig.getMaxPendingRequests()); - assertEquals(HTTPSourceConfig.DEFAULT_USE_ACM_CERTIFICATE_FOR_SSL, sourceConfig.isUseAcmCertificateForSsl()); - assertEquals(HTTPSourceConfig.DEFAULT_ACM_CERTIFICATE_TIMEOUT_MILLIS, sourceConfig.getAcmCertificateTimeoutMillis()); - assertEquals((int)(HTTPSourceConfig.DEFAULT_REQUEST_TIMEOUT_MS * HTTPSourceConfig.BUFFER_TIMEOUT_FRACTION), - sourceConfig.getBufferTimeoutInMillis()); - assertEquals(CompressionOption.NONE, sourceConfig.getCompression()); - } - - @Nested - class SslValidationWithFile { - @Test - void isSslCertificateFileValidation_should_return_true_if_ssl_is_false() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", false); - - assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); - } - - @Test - void isSslCertificateFileValidation_should_return_false_if_ssl_is_true_and_sslCertificateFile_is_null() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - - assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(false)); - } - - @Test - void isSslCertificateFileValidation_should_return_true_if_ssl_is_true_and_sslCertificateFile_is_a_valid_file() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "sslCertificateFile", UUID.randomUUID().toString()); - - assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); - } - - @Test - void isSslKeyFileValidation_should_return_true_if_ssl_is_false() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", false); - - assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); - } - - @Test - void isSslKeyFileValidation_should_return_false_if_ssl_is_true_and_sslKeyFile_is_null() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - - assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(false)); - } - - @Test - void isSslKeyFileValidation_should_return_true_if_ssl_is_true_and_sslKeyFile_is_a_valid_file() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "sslKeyFile", UUID.randomUUID().toString()); - - assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); - } - - } - - @Nested - class SslValidationWithS3 { - @Test - void isSslCertAndKeyFileInS3_should_return_true_if_ssl_is_true_and_KeyFile_and_certFile_are_s3_paths() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "sslCertificateFile", getS3FilePath()); - reflectivelySetField(objectUnderTest, "sslKeyFile", getS3FilePath()); - - assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(true)); - } - - @Test - void isSslCertAndKeyFileInS3_should_return_false_if_ssl_is_true_and_KeyFile_and_certFile_are_not_s3_paths() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "sslCertificateFile", UUID.randomUUID().toString()); - reflectivelySetField(objectUnderTest, "sslKeyFile", UUID.randomUUID().toString()); - - assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(false)); - } - - @Test - void isAwsRegionValid_should_return_true_if_ssl_is_true_and_aws_region_is_null_without_s3_paths() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "sslCertificateFile", UUID.randomUUID().toString()); - reflectivelySetField(objectUnderTest, "sslKeyFile", UUID.randomUUID().toString()); - - assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(false)); - assertThat(objectUnderTest.isAwsRegionValid(), equalTo(true)); - } - - @Test - void isAwsRegionValid_should_return_false_if_ssl_is_true_and_aws_region_is_null_with_s3_paths() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "sslCertificateFile", getS3FilePath()); - reflectivelySetField(objectUnderTest, "sslKeyFile", getS3FilePath()); - - assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(true)); - assertThat(objectUnderTest.isAwsRegionValid(), equalTo(false)); - } + assertEquals(HTTPSourceConfig.DEFAULT_PORT, sourceConfig.getDefaultPort()); + assertEquals(HTTPSourceConfig.DEFAULT_LOG_INGEST_URI, sourceConfig.getDefaultPath()); - @Test - void isAwsRegionValid_should_return_true_if_ssl_is_true_and_aws_region_is_not_null_with_s3_paths() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "sslCertificateFile", getS3FilePath()); - reflectivelySetField(objectUnderTest, "sslKeyFile", getS3FilePath()); - reflectivelySetField(objectUnderTest, "awsRegion", UUID.randomUUID().toString()); - - assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(true)); - assertThat(objectUnderTest.isAwsRegionValid(), equalTo(true)); - } - - @Test - void isAwsRegionValid_should_return_false_if_ssl_is_true_and_aws_region_is_not_null_with_s3_paths() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "sslCertificateFile", getS3FilePath()); - reflectivelySetField(objectUnderTest, "sslKeyFile", getS3FilePath()); - reflectivelySetField(objectUnderTest, "awsRegion", UUID.randomUUID().toString()); - - assertThat(objectUnderTest.isSslKeyFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertificateFileValid(), equalTo(true)); - assertThat(objectUnderTest.isSslCertAndKeyFileInS3(), equalTo(true)); - assertThat(objectUnderTest.isAwsRegionValid(), equalTo(true)); - } - } - - @Nested - class SslValidationWithAcm { - @Test - void isAwsRegionValid_should_return_false_if_ssl_is_true_and_aws_region_is_null_with_acm() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "useAcmCertificateForSsl", true); - - assertThat(objectUnderTest.isAwsRegionValid(), equalTo(false)); - } - - @Test - void isAwsRegionValid_should_return_true_if_ssl_is_true_and_aws_region_is_not_null_with_acm() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "useAcmCertificateForSsl", true); - reflectivelySetField(objectUnderTest, "awsRegion", UUID.randomUUID().toString()); - - assertThat(objectUnderTest.isAwsRegionValid(), equalTo(true)); - } - - @Test - void isAcmCertificateArnValid_should_return_false_if_ssl_is_true_and_acm_is_true_and_arn_is_null() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "useAcmCertificateForSsl", true); - - assertThat(objectUnderTest.isAcmCertificateArnValid(), equalTo(false)); - } - - @Test - void isAcmCertificateArnValid_should_return_true_if_ssl_is_true_and_acm_is_true_and_arn_is_not_null() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "ssl", true); - reflectivelySetField(objectUnderTest, "useAcmCertificateForSsl", true); - reflectivelySetField(objectUnderTest, "acmCertificateArn", UUID.randomUUID().toString()); - - assertThat(objectUnderTest.isAcmCertificateArnValid(), equalTo(true)); - } } - - @ParameterizedTest - @MethodSource("provideCompressionOption") - void testValidCompression(final CompressionOption compressionOption) { - // Prepare - final Map settings = new HashMap<>(); - settings.put(HTTPSourceConfig.COMPRESSION, compressionOption.name()); - - final PluginSetting pluginSetting = new PluginSetting(PLUGIN_NAME, settings); - final HTTPSourceConfig httpSourceConfig = OBJECT_MAPPER.convertValue( - pluginSetting.getSettings(), HTTPSourceConfig.class); - - // When/Then - assertEquals(compressionOption, httpSourceConfig.getCompression()); - } - - @Test - void getPath_should_return_correct_path() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "path", "/my/custom/path"); - - assertThat(objectUnderTest.isPathValid(), equalTo(true)); - assertThat(objectUnderTest.getPath(), equalTo("/my/custom/path")); - } - - @Test - void isPathValid_should_return_false_for_invalid_path() throws NoSuchFieldException, IllegalAccessException { - final HTTPSourceConfig objectUnderTest = new HTTPSourceConfig(); - - reflectivelySetField(objectUnderTest, "path", "my/custom/path"); - - assertThat(objectUnderTest.isPathValid(), equalTo(false)); - } - - private void reflectivelySetField(final HTTPSourceConfig httpSourceConfig, final String fieldName, final Object value) throws NoSuchFieldException, IllegalAccessException { - final Field field = HTTPSourceConfig.class.getDeclaredField(fieldName); - try { - field.setAccessible(true); - field.set(httpSourceConfig, value); - } finally { - field.setAccessible(false); - } - } - - private String getS3FilePath() { - return S3_PREFIX.concat(UUID.randomUUID().toString()); - } } diff --git a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceTest.java b/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceTest.java index 29cac75de7..c6078c4095 100644 --- a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceTest.java +++ b/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/HTTPSourceTest.java @@ -35,6 +35,7 @@ import org.opensearch.dataprepper.HttpRequestExceptionHandler; import org.opensearch.dataprepper.armeria.authentication.ArmeriaHttpAuthenticationProvider; import org.opensearch.dataprepper.armeria.authentication.HttpBasicAuthenticationConfig; +import org.opensearch.dataprepper.http.LogThrottlingRejectHandler; import org.opensearch.dataprepper.metrics.MetricNames; import org.opensearch.dataprepper.metrics.MetricsTestUtil; import org.opensearch.dataprepper.metrics.PluginMetrics; diff --git a/settings.gradle b/settings.gradle index 3d07f3114b..a2495d9ffc 100644 --- a/settings.gradle +++ b/settings.gradle @@ -173,3 +173,4 @@ include 'data-prepper-plugins:http-common' include 'data-prepper-plugins:flatten-processor' include 'data-prepper-plugins:mongodb' include 'data-prepper-plugins:rds-source' +include 'data-prepper-plugins:http-source-common' \ No newline at end of file From 5cf0927ea8b192bfc714bdfa0b07a078ce1d583a Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Thu, 6 Jun 2024 12:25:36 -0700 Subject: [PATCH 007/159] Fix KeyValue Processor value grouping bug (#4606) Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../processor/keyvalue/KeyValueProcessor.java | 2 +- .../keyvalue/KeyValueProcessorTests.java | 18 ++++++++++++++++++ 2 files changed, 19 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java index 9f1cd56f20..ea3a7accdb 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java @@ -321,7 +321,7 @@ private List parseWithValueGrouping(String str) { if (groupIndex >= 0) { String[] s = keyValueDelimiterPattern.split(str.substring(start,i+1)); // Only handle Grouping patterns in the values, not keys - if (s.length > 1 || startGroupStrings[groupIndex].charAt(0) == stringLiteralCharacter) { + if (s.length > 1 || (stringLiteralCharacter != null && startGroupStrings[groupIndex].charAt(0) == stringLiteralCharacter)) { i = skipGroup(str, i+1, endGroupChars[groupIndex]); skippedGroup = true; } diff --git a/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java b/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java index 2adfc37884..505e236ea9 100644 --- a/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java +++ b/data-prepper-plugins/key-value-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorTests.java @@ -250,6 +250,24 @@ private static Stream getKeyValueGroupingTestdata() { ); } + @Test + void testValueGroupingWithOutStringLiterals() { + when(mockConfig.getDestination()).thenReturn(null); + String message = "text1 text2 [ key1=value1 value2"; + lenient().when(mockConfig.getStringLiteralCharacter()).thenReturn(null); + lenient().when(mockConfig.getFieldSplitCharacters()).thenReturn(" ,"); + lenient().when(mockConfig.getValueGrouping()).thenReturn(true); + final Record record = getMessage(message); + keyValueProcessor = createObjectUnderTest(); + final List> editedRecords = (List>) keyValueProcessor.doExecute(Collections.singletonList(record)); + + final Event event = editedRecords.get(0).getData(); + assertThat(event.containsKey("parsed_message"), is(false)); + + assertThat(event.containsKey("key1"), is(true)); + assertThat(event.get("key1", Object.class), is("value1")); + } + @ParameterizedTest @ValueSource(strings = {"\"", "'"}) void testStringLiteralCharacter(String literalString) { From 4cf86e97b8849366d246f4f74086a1f8860e6e93 Mon Sep 17 00:00:00 2001 From: Srikanth Govindarajan Date: Thu, 6 Jun 2024 13:17:27 -0700 Subject: [PATCH 008/159] Add support for lambda sink (#4292) * Add support for lambda sink Signed-off-by: srigovs * Address event handle comment Signed-off-by: Srikanth Govindarajan --------- Signed-off-by: srigovs Signed-off-by: Srikanth Govindarajan --- data-prepper-plugins/lambda-sink/README.md | 36 ++ data-prepper-plugins/lambda-sink/build.gradle | 63 ++++ .../sink/lambda/LambdaSinkServiceIT.java | 216 ++++++++++++ .../sink/lambda/LambdaClientFactory.java | 46 +++ .../plugins/sink/lambda/LambdaSink.java | 107 ++++++ .../plugins/sink/lambda/LambdaSinkConfig.java | 80 +++++ .../sink/lambda/LambdaSinkService.java | 231 +++++++++++++ .../plugins/sink/lambda/ThresholdCheck.java | 33 ++ .../sink/lambda/accumlator/Buffer.java | 31 ++ .../sink/lambda/accumlator/BufferFactory.java | 14 + .../lambda/accumlator/InMemoryBuffer.java | 110 ++++++ .../accumlator/InMemoryBufferFactory.java | 16 + .../sink/lambda/codec/LambdaJsonCodec.java | 79 +++++ .../config/AwsAuthenticationOptions.java | 46 +++ .../sink/lambda/config/BatchOptions.java | 27 ++ .../sink/lambda/config/ThresholdOptions.java | 58 ++++ .../sink/lambda/dlq/DlqPushHandler.java | 131 +++++++ .../lambda/dlq/LambdaSinkFailedDlqData.java | 60 ++++ .../sink/lambda/LambdaClientFactoryTest.java | 98 ++++++ .../sink/lambda/LambdaSinkConfigTest.java | 69 ++++ .../sink/lambda/LambdaSinkServiceTest.java | 326 ++++++++++++++++++ .../plugins/sink/lambda/LambdaSinkTest.java | 82 +++++ .../sink/lambda/ThresholdCheckTest.java | 126 +++++++ .../InMemoryBufferFactoryTest.java | 32 ++ .../accumulator/InMemoryBufferTest.java | 167 +++++++++ .../lambda/codec/LambdaJsonCodecTest.java | 110 ++++++ .../lambda/config/ThresholdOptionsTest.java | 33 ++ .../sink/lambda/dlq/DlqPushHandlerTest.java | 88 +++++ .../org.mockito.plugins.MockMaker | 3 + settings.gradle | 5 +- 30 files changed, 2521 insertions(+), 2 deletions(-) create mode 100644 data-prepper-plugins/lambda-sink/README.md create mode 100644 data-prepper-plugins/lambda-sink/build.gradle create mode 100644 data-prepper-plugins/lambda-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactory.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSink.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfig.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkService.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheck.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/Buffer.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/BufferFactory.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBuffer.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBufferFactory.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodec.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/AwsAuthenticationOptions.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/BatchOptions.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptions.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandler.java create mode 100644 data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/LambdaSinkFailedDlqData.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactoryTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfigTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheckTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferFactoryTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodecTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptionsTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandlerTest.java create mode 100644 data-prepper-plugins/lambda-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/data-prepper-plugins/lambda-sink/README.md b/data-prepper-plugins/lambda-sink/README.md new file mode 100644 index 0000000000..c0b2c29211 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/README.md @@ -0,0 +1,36 @@ +# Lambda Sink + +This plugin enables you to send data from your Data Prepper pipeline directly to AWS Lambda functions for further processing. + +## Usage +```aidl +lambda-pipeline: +... + sink: + - lambda: + aws: + region: "us-east-1" + sts_role_arn: "" + function_name: "uploadToS3Lambda" + max_retries: 3 + batch: + batch_key: "osi_key" + threshold: + event_count: 3 + maximum_size: 6mb + event_collect_timeout: 15s + dlq: + s3: + bucket: test-bucket + key_path_prefix: dlq/ +``` + +## Developer Guide + +The integration tests for this plugin do not run as part of the Data Prepper build. +The following command runs the integration tests: + +``` +./gradlew :data-prepper-plugins:lambda-sink:integrationTest -Dtests.sink.lambda.region="us-east-1" -Dtests.sink.lambda.functionName="lambda_test_function" -Dtests.sink.lambda.sts_role_arn="arn:aws:iam::123456789012:role/dataprepper-role + +``` diff --git a/data-prepper-plugins/lambda-sink/build.gradle b/data-prepper-plugins/lambda-sink/build.gradle new file mode 100644 index 0000000000..429e190a6a --- /dev/null +++ b/data-prepper-plugins/lambda-sink/build.gradle @@ -0,0 +1,63 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +dependencies { + implementation project(':data-prepper-api') + implementation project(path: ':data-prepper-plugins:common') + implementation project(':data-prepper-plugins:aws-plugin-api') + implementation project(':data-prepper-plugins:failures-common') + implementation 'io.micrometer:micrometer-core' + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'software.amazon.awssdk:lambda:2.17.99' + implementation 'software.amazon.awssdk:sdk-core:2.x.x' + implementation 'software.amazon.awssdk:sts' + implementation 'org.hibernate.validator:hibernate-validator:8.0.1.Final' + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + implementation'org.json:json' + implementation libs.commons.lang3 + implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' + testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' + testImplementation project(':data-prepper-test-common') + testImplementation project(':data-prepper-plugins:parse-json-processor') +} + +test { + useJUnitPlatform() +} + +sourceSets { + integrationTest { + java { + compileClasspath += main.output + test.output + runtimeClasspath += main.output + test.output + srcDir file('src/integrationTest/java') + } + resources.srcDir file('src/integrationTest/resources') + } +} + +configurations { + integrationTestImplementation.extendsFrom testImplementation + integrationTestRuntime.extendsFrom testRuntime +} + +task integrationTest(type: Test) { + group = 'verification' + testClassesDirs = sourceSets.integrationTest.output.classesDirs + + useJUnitPlatform() + + classpath = sourceSets.integrationTest.runtimeClasspath + + systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties' + systemProperty 'tests.sink.lambda.region', System.getProperty('tests.sink.lambda.region') + systemProperty 'tests.sink.lambda.functionName', System.getProperty('tests.sink.lambda.functionName') + systemProperty 'tests.sink.lambda.sts_role_arn', System.getProperty('tests.sink.lambda.sts_role_arn') + + filter { + includeTestsMatching '*IT' + } +} diff --git a/data-prepper-plugins/lambda-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java b/data-prepper-plugins/lambda-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java new file mode 100644 index 0000000000..89cf85ceac --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java @@ -0,0 +1,216 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.Mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import org.mockito.MockitoAnnotations; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.sink.lambda.config.BatchOptions; +import org.opensearch.dataprepper.plugins.sink.lambda.config.ThresholdOptions; +import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.lambda.dlq.DlqPushHandler; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.lambda.LambdaClient; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; + +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class LambdaSinkServiceIT { + + private LambdaClient lambdaClient; + private String functionName; + private String lambdaRegion; + private String role; + private BufferFactory bufferFactory; + @Mock + private LambdaSinkConfig lambdaSinkConfig; + @Mock + private BatchOptions batchOptions; + @Mock + private ThresholdOptions thresholdOptions; + @Mock + private AwsAuthenticationOptions awsAuthenticationOptions; + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + @Mock + private PluginMetrics pluginMetrics; + @Mock + private DlqPushHandler dlqPushHandler; + @Mock + private PluginFactory pluginFactory; + @Mock + private PluginSetting pluginSetting; + @Mock + private Counter numberOfRecordsSuccessCounter; + @Mock + private Counter numberOfRecordsFailedCounter; + private final ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + + + @BeforeEach + public void setUp() throws Exception { + MockitoAnnotations.openMocks(this); + lambdaRegion = System.getProperty("tests.sink.lambda.region"); + functionName = System.getProperty("tests.sink.lambda.functionName"); + role = System.getProperty("tests.sink.lambda.sts_role_arn"); + + final Region region = Region.of(lambdaRegion); + + lambdaClient = LambdaClient.builder() + .region(Region.of(lambdaRegion)) + .build(); + + bufferFactory = new InMemoryBufferFactory(); + + when(pluginMetrics.counter(LambdaSinkService.NUMBER_OF_RECORDS_FLUSHED_TO_LAMBDA_SUCCESS)). + thenReturn(numberOfRecordsSuccessCounter); + when(pluginMetrics.counter(LambdaSinkService.NUMBER_OF_RECORDS_FLUSHED_TO_LAMBDA_FAILED)). + thenReturn(numberOfRecordsFailedCounter); + } + + + private static Record createRecord() { + final JacksonEvent event = JacksonLog.builder().withData("[{\"name\":\"test\"}]").build(); + return new Record<>(event); + } + + public LambdaSinkService createObjectUnderTest(final String config) throws JsonProcessingException { + + final LambdaSinkConfig lambdaSinkConfig = objectMapper.readValue(config, LambdaSinkConfig.class); + OutputCodecContext codecContext = new OutputCodecContext("Tag", Collections.emptyList(), Collections.emptyList()); + pluginFactory = null; + return new LambdaSinkService(lambdaClient, + lambdaSinkConfig, + pluginMetrics, + pluginFactory, + pluginSetting, + codecContext, + awsCredentialsSupplier, + dlqPushHandler, + bufferFactory); + } + + public LambdaSinkService createObjectUnderTest(LambdaSinkConfig lambdaSinkConfig) throws JsonProcessingException { + + OutputCodecContext codecContext = new OutputCodecContext("Tag", Collections.emptyList(), Collections.emptyList()); + pluginFactory = null; + return new LambdaSinkService(lambdaClient, + lambdaSinkConfig, + pluginMetrics, + pluginFactory, + pluginSetting, + codecContext, + awsCredentialsSupplier, + dlqPushHandler, + bufferFactory); + } + + + private static Collection> generateRecords(int numberOfRecords) { + List> recordList = new ArrayList<>(); + + for (int rows = 0; rows < numberOfRecords; rows++) { + HashMap eventData = new HashMap<>(); + eventData.put("name", "Person" + rows); + eventData.put("age", Integer.toString(rows)); + + Record eventRecord = new Record<>(JacksonEvent.builder().withData(eventData).withEventType("event").build()); + recordList.add(eventRecord); + } + return recordList; + } + + @ParameterizedTest + @ValueSource(ints = {1,5}) + void verify_flushed_records_to_lambda_success(final int recordCount) throws Exception { + + final String LAMBDA_SINK_CONFIG_YAML = + " function_name: " + functionName +"\n" + + " aws:\n" + + " region: us-east-1\n" + + " sts_role_arn: " + role + "\n" + + " max_retries: 3\n"; + LambdaSinkService objectUnderTest = createObjectUnderTest(LAMBDA_SINK_CONFIG_YAML); + + Collection> recordsData = generateRecords(recordCount); + objectUnderTest.output(recordsData); + Thread.sleep(Duration.ofSeconds(10).toMillis()); + + verify(numberOfRecordsSuccessCounter, times(recordCount)).increment(1); + } + + @ParameterizedTest + @ValueSource(ints = {1,5,10}) + void verify_flushed_records_to_lambda_failed_and_dlq_works(final int recordCount) throws Exception { + final String LAMBDA_SINK_CONFIG_INVALID_FUNCTION_NAME = + " function_name: $$$\n" + + " aws:\n" + + " region: us-east-1\n" + + " sts_role_arn: arn:aws:iam::176893235612:role/osis-s3-opensearch-role\n" + + " max_retries: 3\n" + + " dlq: #any failed even\n"+ + " s3:\n"+ + " bucket: test-bucket\n"+ + " key_path_prefix: dlq/\n"; + LambdaSinkService objectUnderTest = createObjectUnderTest(LAMBDA_SINK_CONFIG_INVALID_FUNCTION_NAME); + + Collection> recordsData = generateRecords(recordCount); + objectUnderTest.output(recordsData); + Thread.sleep(Duration.ofSeconds(10).toMillis()); + + verify( numberOfRecordsFailedCounter, times(recordCount)).increment(1); + } + + @ParameterizedTest + @ValueSource(ints = {2,5}) + void verify_flushed_records_with_batching_to_lambda(final int recordCount) throws JsonProcessingException, InterruptedException { + + int event_count = 2; + when(lambdaSinkConfig.getFunctionName()).thenReturn(functionName); + when(lambdaSinkConfig.getMaxConnectionRetries()).thenReturn(3); + when(thresholdOptions.getEventCount()).thenReturn(event_count); + when(thresholdOptions.getMaximumSize()).thenReturn(ByteCount.parse("2mb")); + when(thresholdOptions.getEventCollectTimeOut()).thenReturn(Duration.parse("PT10s")); + when(batchOptions.getBatchKey()).thenReturn("lambda_batch_key"); + when(batchOptions.getThresholdOptions()).thenReturn(thresholdOptions); + when(lambdaSinkConfig.getBatchOptions()).thenReturn(batchOptions); + + LambdaSinkService objectUnderTest = createObjectUnderTest(lambdaSinkConfig); + Collection> recordsData = generateRecords(recordCount); + objectUnderTest.output(recordsData); + Thread.sleep(Duration.ofSeconds(10).toMillis()); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactory.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactory.java new file mode 100644 index 0000000000..3e33a4e835 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactory.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.services.lambda.LambdaClient; + +public final class LambdaClientFactory { + private LambdaClientFactory() { } + + static LambdaClient createLambdaClient(final LambdaSinkConfig lambdaSinkConfig, + final AwsCredentialsSupplier awsCredentialsSupplier) { + final AwsCredentialsOptions awsCredentialsOptions = convertToCredentialsOptions(lambdaSinkConfig.getAwsAuthenticationOptions()); + final AwsCredentialsProvider awsCredentialsProvider = awsCredentialsSupplier.getProvider(awsCredentialsOptions); + + return LambdaClient.builder() + .region(lambdaSinkConfig.getAwsAuthenticationOptions().getAwsRegion()) + .credentialsProvider(awsCredentialsProvider) + .overrideConfiguration(createOverrideConfiguration(lambdaSinkConfig)).build(); + + } + + private static ClientOverrideConfiguration createOverrideConfiguration(final LambdaSinkConfig lambdaSinkConfig) { + final RetryPolicy retryPolicy = RetryPolicy.builder().numRetries(lambdaSinkConfig.getMaxConnectionRetries()).build(); + return ClientOverrideConfiguration.builder() + .retryPolicy(retryPolicy) + .build(); + } + + private static AwsCredentialsOptions convertToCredentialsOptions(final AwsAuthenticationOptions awsAuthenticationOptions) { + return AwsCredentialsOptions.builder() + .withRegion(awsAuthenticationOptions.getAwsRegion()) + .withStsRoleArn(awsAuthenticationOptions.getAwsStsRoleArn()) + .withStsExternalId(awsAuthenticationOptions.getAwsStsExternalId()) + .withStsHeaderOverrides(awsAuthenticationOptions.getAwsStsHeaderOverrides()) + .build(); + } +} diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSink.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSink.java new file mode 100644 index 0000000000..b1ef905233 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSink.java @@ -0,0 +1,107 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.AbstractSink; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; +import org.opensearch.dataprepper.model.sink.Sink; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.sink.lambda.dlq.DlqPushHandler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.lambda.LambdaClient; + +import java.util.Collection; + +@DataPrepperPlugin(name = "lambda", pluginType = Sink.class, pluginConfigurationType = LambdaSinkConfig.class) +public class LambdaSink extends AbstractSink> { + + private static final Logger LOG = LoggerFactory.getLogger(LambdaSink.class); + private volatile boolean sinkInitialized; + private final LambdaSinkService lambdaSinkService; + private final BufferFactory bufferFactory; + private static final String BUCKET = "bucket"; + private static final String KEY_PATH = "key_path_prefix"; + private DlqPushHandler dlqPushHandler = null; + + @DataPrepperPluginConstructor + public LambdaSink(final PluginSetting pluginSetting, + final LambdaSinkConfig lambdaSinkConfig, + final PluginFactory pluginFactory, + final SinkContext sinkContext, + final AwsCredentialsSupplier awsCredentialsSupplier + ) { + super(pluginSetting); + sinkInitialized = Boolean.FALSE; + OutputCodecContext outputCodecContext = OutputCodecContext.fromSinkContext(sinkContext); + LambdaClient lambdaClient = LambdaClientFactory.createLambdaClient(lambdaSinkConfig, awsCredentialsSupplier); + if(lambdaSinkConfig.getDlqPluginSetting() != null) { + this.dlqPushHandler = new DlqPushHandler(pluginFactory, + String.valueOf(lambdaSinkConfig.getDlqPluginSetting().get(BUCKET)), + lambdaSinkConfig.getDlqStsRoleARN() + , lambdaSinkConfig.getDlqStsRegion(), + String.valueOf(lambdaSinkConfig.getDlqPluginSetting().get(KEY_PATH))); + } + this.bufferFactory = new InMemoryBufferFactory(); + + lambdaSinkService = new LambdaSinkService(lambdaClient, + lambdaSinkConfig, + pluginMetrics, + pluginFactory, + pluginSetting, + outputCodecContext, + awsCredentialsSupplier, + dlqPushHandler, + bufferFactory); + + } + + @Override + public boolean isReady() { + return sinkInitialized; + } + + @Override + public void doInitialize() { + try { + doInitializeInternal(); + } catch (InvalidPluginConfigurationException e) { + LOG.error("Invalid plugin configuration, Hence failed to initialize s3-sink plugin."); + this.shutdown(); + throw e; + } catch (Exception e) { + LOG.error("Failed to initialize lambda plugin."); + this.shutdown(); + throw e; + } + } + + private void doInitializeInternal() { + sinkInitialized = Boolean.TRUE; + } + + /** + * @param records Records to be output + */ + @Override + public void doOutput(final Collection> records) { + + if (records.isEmpty()) { + return; + } + lambdaSinkService.output(records); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfig.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfig.java new file mode 100644 index 0000000000..a20fa41181 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfig.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.lambda; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotEmpty; +import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.Size; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.lambda.config.BatchOptions; + +import java.util.Objects; +import java.util.Map; + +public class LambdaSinkConfig { + + private static final int DEFAULT_CONNECTION_RETRIES = 3; + + public static final String STS_REGION = "region"; + + public static final String STS_ROLE_ARN = "sts_role_arn"; + + @JsonProperty("aws") + @NotNull + @Valid + private AwsAuthenticationOptions awsAuthenticationOptions; + + @JsonProperty("function_name") + @NotEmpty + @NotNull + @Size(min = 3, max = 500, message = "function name length should be at least 3 characters") + private String functionName; + + @JsonProperty("max_retries") + private int maxConnectionRetries = DEFAULT_CONNECTION_RETRIES; + + @JsonProperty("dlq") + private PluginModel dlq; + + @JsonProperty("batch") + private BatchOptions batchOptions; + + public AwsAuthenticationOptions getAwsAuthenticationOptions() { + return awsAuthenticationOptions; + } + + public BatchOptions getBatchOptions(){return batchOptions;} + + public String getFunctionName() { + return functionName; + } + + public int getMaxConnectionRetries() { + return maxConnectionRetries; + } + + public PluginModel getDlq() { + return dlq; + } + + public String getDlqStsRoleARN(){ + return Objects.nonNull(getDlqPluginSetting().get(STS_ROLE_ARN)) ? + String.valueOf(getDlqPluginSetting().get(STS_ROLE_ARN)) : + awsAuthenticationOptions.getAwsStsRoleArn(); + } + + public String getDlqStsRegion(){ + return Objects.nonNull(getDlqPluginSetting().get(STS_REGION)) ? + String.valueOf(getDlqPluginSetting().get(STS_REGION)) : + awsAuthenticationOptions.getAwsRegion().toString(); + } + + public Map getDlqPluginSetting(){ + return dlq != null ? dlq.getPluginSettings() : Map.of(); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkService.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkService.java new file mode 100644 index 0000000000..f10607e7d1 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkService.java @@ -0,0 +1,231 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda; + +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.lambda.codec.LambdaJsonCodec; +import org.opensearch.dataprepper.plugins.sink.lambda.config.BatchOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.sink.lambda.dlq.DlqPushHandler; +import org.opensearch.dataprepper.plugins.sink.lambda.dlq.LambdaSinkFailedDlqData; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.lambda.LambdaClient; + +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.atomic.AtomicReference; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class LambdaSinkService { + + private static final Logger LOG = LoggerFactory.getLogger(LambdaSinkService.class); + public static final String NUMBER_OF_RECORDS_FLUSHED_TO_LAMBDA_SUCCESS = "lambdaSinkObjectsEventsSucceeded"; + public static final String NUMBER_OF_RECORDS_FLUSHED_TO_LAMBDA_FAILED = "lambdaSinkObjectsEventsFailed"; + private final PluginSetting pluginSetting; + private final Lock reentrantLock; + private final LambdaSinkConfig lambdaSinkConfig; + private LambdaClient lambdaClient; + private final String functionName; + private int maxEvents = 0; + private ByteCount maxBytes = null; + private Duration maxCollectionDuration = null; + private int maxRetries = 0; + private final Counter numberOfRecordsSuccessCounter; + private final Counter numberOfRecordsFailedCounter; + private final String ASYNC_INVOCATION_TYPE = "Event"; + private final String invocationType; + private Buffer currentBuffer; + private final BufferFactory bufferFactory; + private final DlqPushHandler dlqPushHandler; + private final Collection bufferedEventHandles; + private final List events; + private OutputCodec codec = null; + private final BatchOptions batchOptions; + private Boolean isBatchEnabled; + private OutputCodecContext codecContext = null; + private String batchKey; + + public LambdaSinkService(final LambdaClient lambdaClient, + final LambdaSinkConfig lambdaSinkConfig, + final PluginMetrics pluginMetrics, + final PluginFactory pluginFactory, + final PluginSetting pluginSetting, + final OutputCodecContext codecContext, + final AwsCredentialsSupplier awsCredentialsSupplier, + final DlqPushHandler dlqPushHandler, + final BufferFactory bufferFactory) { + this.lambdaSinkConfig = lambdaSinkConfig; + this.pluginSetting = pluginSetting; + this.dlqPushHandler = dlqPushHandler; + this.lambdaClient = lambdaClient; + reentrantLock = new ReentrantLock(); + numberOfRecordsSuccessCounter = pluginMetrics.counter(NUMBER_OF_RECORDS_FLUSHED_TO_LAMBDA_SUCCESS); + numberOfRecordsFailedCounter = pluginMetrics.counter(NUMBER_OF_RECORDS_FLUSHED_TO_LAMBDA_FAILED); + functionName = lambdaSinkConfig.getFunctionName(); + + maxRetries = lambdaSinkConfig.getMaxConnectionRetries(); + batchOptions = lambdaSinkConfig.getBatchOptions(); + + if (!Objects.isNull(batchOptions)){ + maxEvents = batchOptions.getThresholdOptions().getEventCount(); + maxBytes = batchOptions.getThresholdOptions().getMaximumSize(); + maxCollectionDuration = batchOptions.getThresholdOptions().getEventCollectTimeOut(); + batchKey = batchOptions.getBatchKey(); + isBatchEnabled = true; + }else{ + batchKey = null; + isBatchEnabled = false; + } + this.codecContext = codecContext; + + codec = new LambdaJsonCodec(batchKey); + bufferedEventHandles = new LinkedList<>(); + events = new ArrayList(); + + invocationType = ASYNC_INVOCATION_TYPE; + + this.bufferFactory = bufferFactory; + try { + currentBuffer = bufferFactory.getBuffer(lambdaClient,functionName,invocationType); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public void output(Collection> records){ + // Don't acquire the lock if there's no work to be done + if (records.isEmpty() && currentBuffer.getEventCount() == 0) { + return; + } + List failedEvents = new ArrayList<>(); + Exception sampleException = null; + reentrantLock.lock(); + try { + for (Record record : records) { + final Event event = record.getData(); + try { + if (currentBuffer.getEventCount() == 0) { + codec.start(currentBuffer.getOutputStream(), event, codecContext); + } + codec.writeEvent(event, currentBuffer.getOutputStream()); + int count = currentBuffer.getEventCount() + 1; + currentBuffer.setEventCount(count); + + bufferedEventHandles.add(event.getEventHandle()); + } catch (Exception ex) { + if(sampleException == null) { + sampleException = ex; + } + failedEvents.add(event); + } + + flushToLambdaIfNeeded(); + } + } finally { + reentrantLock.unlock(); + } + + if(!failedEvents.isEmpty()) { + failedEvents + .stream() + .map(Event::getEventHandle) + .forEach(eventHandle -> eventHandle.release(false)); + LOG.error("Unable to add {} events to buffer. Dropping these events. Sample exception provided.", failedEvents.size(), sampleException); + } + } + + private void releaseEventHandles(final boolean result) { + for (EventHandle eventHandle : bufferedEventHandles) { + eventHandle.release(result); + } + bufferedEventHandles.clear(); + } + + private void flushToLambdaIfNeeded() { + LOG.trace("Flush to Lambda check: currentBuffer.size={}, currentBuffer.events={}, currentBuffer.duration={}", + currentBuffer.getSize(), currentBuffer.getEventCount(), currentBuffer.getDuration()); + final AtomicReference errorMsgObj = new AtomicReference<>(); + + try { + if (ThresholdCheck.checkThresholdExceed(currentBuffer, maxEvents, maxBytes, maxCollectionDuration, isBatchEnabled)) { + try { + codec.complete(currentBuffer.getOutputStream()); + LOG.info("Writing {} to Lambda with {} events and size of {} bytes.", + functionName, currentBuffer.getEventCount(), currentBuffer.getSize()); + final boolean isFlushToLambda = retryFlushToLambda(currentBuffer, errorMsgObj); + + if (isFlushToLambda) { + LOG.info("Successfully flushed to Lambda {}.", functionName); + numberOfRecordsSuccessCounter.increment(currentBuffer.getEventCount()); + releaseEventHandles(true); + } else { + LOG.error("Failed to save to Lambda {}", functionName); + numberOfRecordsFailedCounter.increment(currentBuffer.getEventCount()); + SdkBytes payload = currentBuffer.getPayload(); + if(dlqPushHandler!=null) { + dlqPushHandler.perform(pluginSetting, new LambdaSinkFailedDlqData(payload, errorMsgObj.get(), 0)); + releaseEventHandles(true); + }else{ + releaseEventHandles(false); + } + } + + //reset buffer after flush + currentBuffer = bufferFactory.getBuffer(lambdaClient,functionName,invocationType); + } catch (final IOException e) { + releaseEventHandles(false); + LOG.error("Exception while completing codec", e); + } + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + protected boolean retryFlushToLambda(Buffer currentBuffer, + final AtomicReference errorMsgObj) throws InterruptedException { + boolean isUploadedToLambda = Boolean.FALSE; + int retryCount = maxRetries; + do { + + try { + currentBuffer.flushToLambda(); + isUploadedToLambda = Boolean.TRUE; + } catch (AwsServiceException | SdkClientException e) { + errorMsgObj.set(e.getMessage()); + LOG.error("Exception occurred while uploading records to lambda. Retry countdown : {} | exception:", + retryCount, e); + --retryCount; + if (retryCount == 0) { + return isUploadedToLambda; + } + Thread.sleep(5000); + } + } while (!isUploadedToLambda); + return isUploadedToLambda; + } +} diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheck.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheck.java new file mode 100644 index 0000000000..74aa98e7f9 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheck.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda; + +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.Buffer; + +import java.time.Duration; + +/** + * Check threshold limits. + */ +public class ThresholdCheck { + + private ThresholdCheck() { + } + + public static boolean checkThresholdExceed(final Buffer currentBuffer, final int maxEvents, final ByteCount maxBytes, final Duration maxCollectionDuration, final Boolean isBatchEnabled) { + if (!isBatchEnabled) return true; + + if (maxEvents > 0) { + return currentBuffer.getEventCount() + 1 > maxEvents || + currentBuffer.getDuration().compareTo(maxCollectionDuration) > 0 || + currentBuffer.getSize() > maxBytes.getBytes(); + } else { + return currentBuffer.getDuration().compareTo(maxCollectionDuration) > 0 || + currentBuffer.getSize() > maxBytes.getBytes(); + } + } +} diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/Buffer.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/Buffer.java new file mode 100644 index 0000000000..48afbe6a01 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/Buffer.java @@ -0,0 +1,31 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.accumlator; + +import software.amazon.awssdk.core.SdkBytes; + +import java.io.OutputStream; +import java.time.Duration; + +/** + * A buffer can hold data before flushing it. + */ +public interface Buffer { + + long getSize(); + + int getEventCount(); + + Duration getDuration(); + + void flushToLambda(); + + OutputStream getOutputStream(); + + SdkBytes getPayload(); + + void setEventCount(int eventCount); +} diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/BufferFactory.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/BufferFactory.java new file mode 100644 index 0000000000..80afd2f1ca --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/BufferFactory.java @@ -0,0 +1,14 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.accumlator; + +import software.amazon.awssdk.services.lambda.LambdaClient; + +import java.io.IOException; + +public interface BufferFactory { + Buffer getBuffer(LambdaClient lambdaClient, String functionName, String invocationType) throws IOException; +} diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBuffer.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBuffer.java new file mode 100644 index 0000000000..bba70c6e62 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBuffer.java @@ -0,0 +1,110 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.accumlator; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.commons.lang3.time.StopWatch; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.lambda.LambdaClient; +import software.amazon.awssdk.services.lambda.model.InvokeRequest; +import software.amazon.awssdk.services.lambda.model.InvokeResponse; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.time.Duration; +import java.util.concurrent.TimeUnit; + +/** + * A buffer can hold in memory data and flushing it. + */ +public class InMemoryBuffer implements Buffer { + + private static final ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + + private final LambdaClient lambdaClient; + private final String functionName; + private final String invocationType; + private int eventCount; + private final StopWatch watch; + private boolean isCodecStarted; + + + public InMemoryBuffer(LambdaClient lambdaClient, String functionName, String invocationType) { + this.lambdaClient = lambdaClient; + this.functionName = functionName; + this.invocationType = invocationType; + + byteArrayOutputStream.reset(); + eventCount = 0; + watch = new StopWatch(); + watch.start(); + isCodecStarted = false; + } + + @Override + public long getSize() { + return byteArrayOutputStream.size(); + } + + @Override + public int getEventCount() { + return eventCount; + } + + public Duration getDuration() { + return Duration.ofMillis(watch.getTime(TimeUnit.MILLISECONDS)); + } + + + @Override + public void flushToLambda() { + InvokeResponse resp; + SdkBytes payload = getPayload(); + + // Setup an InvokeRequest. + InvokeRequest request = InvokeRequest.builder() + .functionName(functionName) + .payload(payload) + .invocationType(invocationType) + .build(); + + resp = lambdaClient.invoke(request); + } + + private SdkBytes validatePayload(String payload_string) { + ObjectMapper mapper = new ObjectMapper(); + try { + JsonNode jsonNode = mapper.readTree(byteArrayOutputStream.toByteArray()); + + // Convert the JsonNode back to a String to normalize it (removes extra spaces, trailing commas, etc.) + String normalizedJson = mapper.writeValueAsString(jsonNode); + return SdkBytes.fromUtf8String(normalizedJson); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + + @Override + public void setEventCount(int eventCount) { + this.eventCount = eventCount; + } + + @Override + public OutputStream getOutputStream() { + return byteArrayOutputStream; + } + + @Override + public SdkBytes getPayload() { + byte[] bytes = byteArrayOutputStream.toByteArray(); + SdkBytes sdkBytes = SdkBytes.fromByteArray(bytes); + return sdkBytes; + } +} + diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBufferFactory.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBufferFactory.java new file mode 100644 index 0000000000..e58952c5cb --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBufferFactory.java @@ -0,0 +1,16 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.accumlator; + +import software.amazon.awssdk.services.lambda.LambdaClient; + + +public class InMemoryBufferFactory implements BufferFactory { + @Override + public Buffer getBuffer(LambdaClient lambdaClient, String functionName, String invocationType){ + return new InMemoryBuffer(lambdaClient, functionName, invocationType); + } +} diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodec.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodec.java new file mode 100644 index 0000000000..5bf21f5e18 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodec.java @@ -0,0 +1,79 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.lambda.codec; + +import com.fasterxml.jackson.core.JsonEncoding; +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.opensearch.dataprepper.model.codec.OutputCodec; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; +import java.util.Objects; + +public class LambdaJsonCodec implements OutputCodec { + private final ObjectMapper objectMapper = new ObjectMapper(); + private static final String JSON = "json"; + private static final JsonFactory factory = new JsonFactory(); + private JsonGenerator generator; + private OutputCodecContext codecContext; + private final String keyName; + + public LambdaJsonCodec(final String keyName) { + this.keyName = keyName; + } + + @Override + public String getExtension() { + return JSON; + } + + @Override + public void start(final OutputStream outputStream, Event event, final OutputCodecContext codecContext) throws IOException { + Objects.requireNonNull(outputStream); + Objects.requireNonNull(codecContext); + this.codecContext = codecContext; + generator = factory.createGenerator(outputStream, JsonEncoding.UTF8); + if(Objects.nonNull(keyName)){ + generator.writeStartObject(); + generator.writeFieldName(keyName); + generator.writeStartArray(); + } + } + + @Override + public void complete(final OutputStream outputStream) throws IOException { + if(!Objects.isNull(keyName)) { + generator.writeEndArray(); + generator.writeEndObject(); + } + + generator.close(); + outputStream.flush(); + outputStream.close(); + } + + @Override + public synchronized void writeEvent(final Event event, final OutputStream outputStream) throws IOException { + Objects.requireNonNull(event); + if(Objects.isNull(keyName)) { + Map eventMap = event.toMap(); + objectMapper.writeValue(outputStream, eventMap); + + }else{ + Map dataMap = event.toMap(); //(event); + objectMapper.writeValue(generator, dataMap); + } + generator.flush(); + } +} + + + + diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/AwsAuthenticationOptions.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/AwsAuthenticationOptions.java new file mode 100644 index 0000000000..8d6c64829d --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/AwsAuthenticationOptions.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.regions.Region; + +import java.util.Map; + +public class AwsAuthenticationOptions { + @JsonProperty("region") + @Size(min = 1, message = "Region cannot be empty string") + private String awsRegion; + + @JsonProperty("sts_role_arn") + @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") + private String awsStsRoleArn; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/BatchOptions.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/BatchOptions.java new file mode 100644 index 0000000000..3773d4e6ed --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/BatchOptions.java @@ -0,0 +1,27 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; + + +public class BatchOptions { + + private static final String DEFAULT_BATCH_KEY = "events"; + + @JsonProperty("batch_key") + private String batchKey = DEFAULT_BATCH_KEY; + + @JsonProperty("threshold") + @NotNull + ThresholdOptions thresholdOptions; + + public String getBatchKey(){return batchKey;} + + public ThresholdOptions getThresholdOptions(){return thresholdOptions;} + +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptions.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptions.java new file mode 100644 index 0000000000..031157c4be --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptions.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.config; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.hibernate.validator.constraints.time.DurationMax; +import org.hibernate.validator.constraints.time.DurationMin; +import org.opensearch.dataprepper.model.types.ByteCount; +import java.time.Duration; +import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.Size; + + +public class ThresholdOptions { + + private static final String DEFAULT_BYTE_CAPACITY = "6mb"; + + @JsonProperty("event_count") + @Size(min = 0, max = 10000000, message = "event_count size should be between 0 and 10000000") + @NotNull + private int eventCount; + + @JsonProperty("maximum_size") + private String maximumSize = DEFAULT_BYTE_CAPACITY; + + @JsonProperty("event_collect_timeout") + @DurationMin(seconds = 1) + @DurationMax(seconds = 3600) + @NotNull + private Duration eventCollectTimeOut; + + /** + * Read event collection duration configuration. + * @return event collect time out. + */ + public Duration getEventCollectTimeOut() { + return eventCollectTimeOut; + } + + /** + * Read byte capacity configuration. + * @return maximum byte count. + */ + public ByteCount getMaximumSize() { + return ByteCount.parse(maximumSize); + } + + /** + * Read the event count configuration. + * @return event count. + */ + public int getEventCount() { + return eventCount; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandler.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandler.java new file mode 100644 index 0000000000..1bdeb0a394 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandler.java @@ -0,0 +1,131 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.lambda.dlq; + +import com.fasterxml.jackson.databind.ObjectWriter; +import io.micrometer.core.instrument.util.StringUtils; +import org.opensearch.dataprepper.metrics.MetricNames; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.failures.DlqObject; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.plugins.dlq.DlqProvider; +import org.opensearch.dataprepper.plugins.dlq.DlqWriter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedWriter; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.StringJoiner; + +import static java.util.UUID.randomUUID; + + +/** + * * An Handler class which helps log failed data to AWS S3 bucket or file based on configuration. + */ + +public class DlqPushHandler { + + private static final Logger LOG = LoggerFactory.getLogger(DlqPushHandler.class); + + private static final String BUCKET = "bucket"; + + private static final String ROLE_ARN = "sts_role_arn"; + + private static final String REGION = "region"; + + private static final String S3_PLUGIN_NAME = "s3"; + + private static final String KEY_PATH_PREFIX = "key_path_prefix"; + + private String dlqFile; + + private String keyPathPrefix; + + private DlqProvider dlqProvider; + + private ObjectWriter objectWriter; + + public DlqPushHandler( + final PluginFactory pluginFactory, + final String bucket, + final String stsRoleArn, + final String awsRegion, + final String dlqPathPrefix) { + + this.dlqProvider = getDlqProvider(pluginFactory,bucket,stsRoleArn,awsRegion,dlqPathPrefix); + } + + public void perform(final PluginSetting pluginSetting, + final Object failedData) { + if(dlqFile != null) + writeToFile(failedData); + else + pushToS3(pluginSetting, failedData); + } + + private void writeToFile(Object failedData) { + try(BufferedWriter dlqFileWriter = Files.newBufferedWriter(Paths.get(dlqFile), + StandardOpenOption.CREATE, StandardOpenOption.APPEND)) { + dlqFileWriter.write(objectWriter.writeValueAsString(failedData)+"\n"); + } catch (IOException e) { + LOG.error("Exception while writing failed data to DLQ file Exception: ",e); + } + } + + private void pushToS3(PluginSetting pluginSetting, Object failedData) { + DlqWriter dlqWriter = getDlqWriter(pluginSetting.getPipelineName()); + try { + String pluginId = randomUUID().toString(); + DlqObject dlqObject = DlqObject.builder() + .withPluginId(pluginId) + .withPluginName(pluginSetting.getName()) + .withPipelineName(pluginSetting.getPipelineName()) + .withFailedData(failedData) + .build(); + final List dlqObjects = Arrays.asList(dlqObject); + dlqWriter.write(dlqObjects, pluginSetting.getPipelineName(), pluginId); + LOG.info("wrote {} events to DLQ",dlqObjects.size()); + } catch (final IOException e) { + LOG.error("Exception while writing failed data to DLQ, Exception : ", e); + } + } + + private DlqWriter getDlqWriter(final String writerPipelineName) { + Optional potentialDlq = dlqProvider.getDlqWriter(new StringJoiner(MetricNames.DELIMITER) + .add(writerPipelineName).toString()); + DlqWriter dlqWriter = potentialDlq.isPresent() ? potentialDlq.get() : null; + return dlqWriter; + } + + private DlqProvider getDlqProvider(final PluginFactory pluginFactory, + final String bucket, + final String stsRoleArn, + final String awsRegion, + final String dlqPathPrefix) { + final Map props = new HashMap<>(); + props.put(BUCKET, bucket); + props.put(ROLE_ARN, stsRoleArn); + props.put(REGION, awsRegion); + this.keyPathPrefix = StringUtils.isEmpty(dlqPathPrefix) ? dlqPathPrefix : enforceDefaultDelimiterOnKeyPathPrefix(dlqPathPrefix); + props.put(KEY_PATH_PREFIX, dlqPathPrefix); + final PluginSetting dlqPluginSetting = new PluginSetting(S3_PLUGIN_NAME, props); + DlqProvider dlqProvider = pluginFactory.loadPlugin(DlqProvider.class, dlqPluginSetting); + return dlqProvider; + } + + private String enforceDefaultDelimiterOnKeyPathPrefix(final String keyPathPrefix) { + return (keyPathPrefix.charAt(keyPathPrefix.length() - 1) == '/') ? keyPathPrefix : keyPathPrefix.concat("/"); + } +} + diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/LambdaSinkFailedDlqData.java b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/LambdaSinkFailedDlqData.java new file mode 100644 index 0000000000..0808010e37 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/LambdaSinkFailedDlqData.java @@ -0,0 +1,60 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.lambda.dlq; + +import com.fasterxml.jackson.core.JsonProcessingException; +import software.amazon.awssdk.core.SdkBytes; + + +public class LambdaSinkFailedDlqData { + + private SdkBytes payload; + + private String message; + + private int status; + + public LambdaSinkFailedDlqData(SdkBytes payload, String message, int status) throws JsonProcessingException { + this.payload = payload; + this.message = message; + this.status = status; + } + + public SdkBytes getPayload() { + return payload; + } + + public LambdaSinkFailedDlqData setPayload(SdkBytes payload) { + this.payload = payload; + return this; + } + + public String getMessage() { + return message; + } + + public LambdaSinkFailedDlqData setMessage(String message) { + this.message = message; + return this; + } + + public int getStatus() { + return status; + } + + public LambdaSinkFailedDlqData setStatus(int status) { + this.status = status; + return this; + } + + @Override + public String toString() { + + return "failedData\n" + + "payload \"" + payload.asUtf8String() + "\"\n" + + "message \"" + message + "\"\n" + + "status \"" + status + "\n"; + } +} diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactoryTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactoryTest.java new file mode 100644 index 0000000000..ab72ee44b8 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactoryTest.java @@ -0,0 +1,98 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.lambda; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.lambda.LambdaClient; +import software.amazon.awssdk.services.lambda.LambdaClientBuilder; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.util.Map; +import java.util.UUID; + +@ExtendWith(MockitoExtension.class) +class LambdaClientFactoryTest { + @Mock + private LambdaSinkConfig lambdaSinkConfig; + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + + @Mock + private AwsAuthenticationOptions awsAuthenticationOptions; + + @BeforeEach + void setUp() { + when(lambdaSinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + } + + @Test + void createLambdaClient_with_real_LambdaClient() { + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.US_EAST_1); + final LambdaClient lambdaClient = LambdaClientFactory.createLambdaClient(lambdaSinkConfig, awsCredentialsSupplier); + + assertThat(lambdaClient, notNullValue()); + } + + @ParameterizedTest + @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) + void createlambdaClient_provides_correct_inputs(final String regionString) { + final Region region = Region.of(regionString); + final String stsRoleArn = UUID.randomUUID().toString(); + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(region); + when(awsAuthenticationOptions.getAwsStsRoleArn()).thenReturn(stsRoleArn); + when(awsAuthenticationOptions.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); + + final AwsCredentialsProvider expectedCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(any())).thenReturn(expectedCredentialsProvider); + + final LambdaClientBuilder lambdaClientBuilder = mock(LambdaClientBuilder.class); + when(lambdaClientBuilder.region(region)).thenReturn(lambdaClientBuilder); + when(lambdaClientBuilder.credentialsProvider(any())).thenReturn(lambdaClientBuilder); + when(lambdaClientBuilder.overrideConfiguration(any(ClientOverrideConfiguration.class))).thenReturn(lambdaClientBuilder); + try(final MockedStatic lambdaClientMockedStatic = mockStatic(LambdaClient.class)) { + lambdaClientMockedStatic.when(LambdaClient::builder) + .thenReturn(lambdaClientBuilder); + LambdaClientFactory.createLambdaClient(lambdaSinkConfig, awsCredentialsSupplier); + } + + final ArgumentCaptor credentialsProviderArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsProvider.class); + verify(lambdaClientBuilder).credentialsProvider(credentialsProviderArgumentCaptor.capture()); + + final AwsCredentialsProvider actualCredentialsProvider = credentialsProviderArgumentCaptor.getValue(); + + assertThat(actualCredentialsProvider, equalTo(expectedCredentialsProvider)); + + final ArgumentCaptor optionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); + verify(awsCredentialsSupplier).getProvider(optionsArgumentCaptor.capture()); + + final AwsCredentialsOptions actualCredentialsOptions = optionsArgumentCaptor.getValue(); + assertThat(actualCredentialsOptions.getRegion(), equalTo(region)); + assertThat(actualCredentialsOptions.getStsRoleArn(), equalTo(stsRoleArn)); + assertThat(actualCredentialsOptions.getStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfigTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfigTest.java new file mode 100644 index 0000000000..eda9488a04 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfigTest.java @@ -0,0 +1,69 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.lambda; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.regions.Region; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + + +class LambdaSinkConfigTest { + public static final int DEFAULT_MAX_RETRIES = 3; + private ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + + @Test + void lambda_sink_default_max_connection_retries_test(){ + assertThat(new LambdaSinkConfig().getMaxConnectionRetries(),equalTo(DEFAULT_MAX_RETRIES)); + } + + @Test + void lambda_sink_pipeline_config_test() throws JsonProcessingException { + final String config = + " function_name: test_function\n" + + " aws:\n" + + " region: ap-south-1\n" + + " sts_role_arn: arn:aws:iam::524239988912:role/app-test\n" + + " sts_header_overrides: {\"test\":\"test\"}\n" + + " max_retries: 10\n" + + " dlq:\n" + + " s3:\n" + + " bucket: test\n" + + " key_path_prefix: test\n" + + " region: ap-south-1\n" + + " sts_role_arn: test-role-arn\n"; + final LambdaSinkConfig lambdaSinkConfig = objectMapper.readValue(config, LambdaSinkConfig.class); + assertThat(lambdaSinkConfig.getMaxConnectionRetries(),equalTo(10)); + assertThat(lambdaSinkConfig.getAwsAuthenticationOptions().getAwsRegion(),equalTo(Region.AP_SOUTH_1)); + assertThat(lambdaSinkConfig.getAwsAuthenticationOptions().getAwsStsRoleArn(),equalTo("arn:aws:iam::524239988912:role/app-test")); + assertThat(lambdaSinkConfig.getAwsAuthenticationOptions().getAwsStsHeaderOverrides().get("test"),equalTo("test")); + assertThat(lambdaSinkConfig.getDlqStsRegion(),equalTo("ap-south-1")); + assertThat(lambdaSinkConfig.getDlqStsRoleARN(),equalTo("test-role-arn")); + } + + @Test + void lambda_sink_pipeline_config_test_with_no_dlq() throws JsonProcessingException { + final String config = + " function_name: test_function\n" + + " aws:\n" + + " region: ap-south-1\n" + + " sts_role_arn: arn:aws:iam::524239988912:role/app-test\n" + + " sts_header_overrides: {\"test\":\"test\"}\n" + + " max_retries: 10\n"; + final LambdaSinkConfig lambdaSinkConfig = objectMapper.readValue(config, LambdaSinkConfig.class); + assertThat(lambdaSinkConfig.getMaxConnectionRetries(),equalTo(10)); + assertThat(lambdaSinkConfig.getAwsAuthenticationOptions().getAwsRegion(),equalTo(Region.AP_SOUTH_1)); + assertThat(lambdaSinkConfig.getAwsAuthenticationOptions().getAwsStsRoleArn(),equalTo("arn:aws:iam::524239988912:role/app-test")); + assertThat(lambdaSinkConfig.getAwsAuthenticationOptions().getAwsStsHeaderOverrides().get("test"),equalTo("test")); + assertThat(lambdaSinkConfig.getDlqStsRegion(),equalTo("ap-south-1")); + assertThat(lambdaSinkConfig.getDlqStsRoleARN(),equalTo("arn:aws:iam::524239988912:role/app-test")); + assertThat(lambdaSinkConfig.getDlqPluginSetting().get("key"),equalTo(null)); + } +} diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceTest.java new file mode 100644 index 0000000000..bbab8778c0 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceTest.java @@ -0,0 +1,326 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.lambda; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.BufferFactory; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBuffer; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.lambda.config.BatchOptions; +import org.opensearch.dataprepper.plugins.sink.lambda.config.ThresholdOptions; +import org.opensearch.dataprepper.plugins.sink.lambda.dlq.DlqPushHandler; +import org.opensearch.dataprepper.plugins.sink.lambda.dlq.LambdaSinkFailedDlqData; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.http.SdkHttpResponse; +import software.amazon.awssdk.services.lambda.LambdaClient; +import software.amazon.awssdk.services.lambda.model.InvokeRequest; +import software.amazon.awssdk.services.lambda.model.InvokeResponse; + + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.ArgumentMatchers.any; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; + +public class LambdaSinkServiceTest { + + public static final int maxEvents = 10; + public static final int maxRetries = 3; + public static final String region = "us-east-1"; + public static final String maxSize = "1kb"; + public static final String functionName = "testFunction"; + public static final String invocationType = "event"; + public static final String batchKey ="lambda_batch_key"; + public static final String config = + " function_name: testFunction\n" + + " aws:\n" + + " region: us-east-1\n" + + " sts_role_arn: arn:aws:iam::524239988912:role/app-test\n" + + " sts_header_overrides: {\"test\":\"test\"}\n" + + " max_retries: 10\n"; + + private final ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + private LambdaSinkConfig lambdaSinkConfig; + private LambdaClient lambdaClient; + private PluginMetrics pluginMetrics; + private Counter numberOfRecordsSuccessCounter; + private Counter numberOfRecordsFailedCounter; + private DlqPushHandler dlqPushHandler; + private Buffer buffer; + private BufferFactory bufferFactory; + + + private InvokeResponse invokeResponse; + + private SdkHttpResponse sdkHttpResponse; + + InvokeResponse mockResponse; + + @BeforeEach + public void setup() throws IOException { + this.lambdaClient = mock(LambdaClient.class); + this.pluginMetrics = mock(PluginMetrics.class); + this.buffer = mock(InMemoryBuffer.class); + this.lambdaSinkConfig = mock(LambdaSinkConfig.class); + this.numberOfRecordsSuccessCounter = mock(Counter.class); + this.numberOfRecordsFailedCounter = mock(Counter.class); + this.dlqPushHandler = mock(DlqPushHandler.class); + this.bufferFactory = mock(BufferFactory.class); + when(pluginMetrics.counter(LambdaSinkService.NUMBER_OF_RECORDS_FLUSHED_TO_LAMBDA_SUCCESS)).thenReturn(numberOfRecordsSuccessCounter); + when(pluginMetrics.counter(LambdaSinkService.NUMBER_OF_RECORDS_FLUSHED_TO_LAMBDA_FAILED)).thenReturn(numberOfRecordsFailedCounter); + mockResponse = InvokeResponse.builder() + .statusCode(200) // HTTP 200 for successful invocation + .payload(SdkBytes.fromString("{\"key\": \"value\"}", java.nio.charset.StandardCharsets.UTF_8)) + .build(); + } + + private LambdaSinkService createObjectUnderTest(LambdaSinkConfig lambdaSinkConfig) throws IOException { + bufferFactory = new InMemoryBufferFactory(); + buffer = bufferFactory.getBuffer(lambdaClient,functionName,invocationType); + return new LambdaSinkService(lambdaClient, + lambdaSinkConfig, + pluginMetrics, + mock(PluginFactory.class), + mock(PluginSetting.class), + mock(OutputCodecContext.class), + mock(AwsCredentialsSupplier.class), + dlqPushHandler, + bufferFactory); + } + + private LambdaSinkService createObjectUnderTest(String config) throws IOException { + this.lambdaSinkConfig = objectMapper.readValue(config, LambdaSinkConfig.class); + bufferFactory = new InMemoryBufferFactory(); + buffer = bufferFactory.getBuffer(lambdaClient,functionName,invocationType); + return new LambdaSinkService(lambdaClient, + lambdaSinkConfig, + pluginMetrics, + mock(PluginFactory.class), + mock(PluginSetting.class), + mock(OutputCodecContext.class), + mock(AwsCredentialsSupplier.class), + dlqPushHandler, + bufferFactory); + } + + @Test + public void lambda_sink_test_with_empty_payload_records() throws IOException { + numberOfRecordsSuccessCounter = mock(Counter.class); + LambdaSinkService lambdaSinkService = createObjectUnderTest(config); + lambdaSinkService.output(List.of()); + verifyNoInteractions(lambdaClient); + verifyNoInteractions(numberOfRecordsSuccessCounter); + verifyNoInteractions(numberOfRecordsFailedCounter); + } + + + @Test + public void lambda_sink_test_with_single_record_success_push_to_lambda() throws IOException { + LambdaSinkService lambdaSinkService = createObjectUnderTest(config); + + Map map = new HashMap<>(); + map.put("query1","test1"); + map.put("query2","test2"); + + final Record eventRecord = new Record<>(JacksonEvent.builder().withData(map).withEventType("event").build()); + Collection> records = List.of(eventRecord); + final ArgumentCaptor invokeRequestCaptor = ArgumentCaptor.forClass(InvokeRequest.class); + + when(lambdaClient.invoke(any(InvokeRequest.class))).thenReturn(mockResponse); + lambdaSinkService.output(records); + + verify(lambdaClient).invoke(invokeRequestCaptor.capture()); + final InvokeRequest actualRequest = invokeRequestCaptor.getValue(); + assertEquals(actualRequest.functionName(), "testFunction"); + assertEquals(actualRequest.invocationType().toString(), "Event"); + verify(numberOfRecordsSuccessCounter).increment(records.size()); + } + + @Test + public void lambda_sink_test_max_retires_works() throws IOException { + final String config = + " function_name: test_function\n" + + " aws:\n" + + " region: us-east-1\n" + + " sts_role_arn: arn:aws:iam::524239988912:role/app-test\n" + + " sts_header_overrides: {\"test\":\"test\"}\n" + + " max_retries: 3\n"; + this.buffer = mock(InMemoryBuffer.class); + when(lambdaClient.invoke(any(InvokeRequest.class))).thenThrow(AwsServiceException.class); + doNothing().when(dlqPushHandler).perform(any(PluginSetting.class), any(LambdaSinkFailedDlqData.class)); + + this.lambdaSinkConfig = objectMapper.readValue(config, LambdaSinkConfig.class); + bufferFactory = mock(BufferFactory.class); + buffer = mock(Buffer.class); + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + when(buffer.getOutputStream()).thenReturn(byteArrayOutputStream); + when(bufferFactory.getBuffer(any(LambdaClient.class),any(),any())).thenReturn(buffer); + doThrow(AwsServiceException.class).when(buffer).flushToLambda(); + + LambdaSinkService lambdaSinkService = new LambdaSinkService(lambdaClient, + lambdaSinkConfig, + pluginMetrics, + mock(PluginFactory.class), + mock(PluginSetting.class), + mock(OutputCodecContext.class), + mock(AwsCredentialsSupplier.class), + dlqPushHandler, + bufferFactory); + + final Record eventRecord = new Record<>(JacksonEvent.fromMessage("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}")); + Collection> records = List.of(eventRecord); + lambdaSinkService.output(records); + + verify(buffer, times(3)).flushToLambda(); + } + + @Test + public void lambda_sink_test_dlq_works() throws IOException { + final String config = + " function_name: test_function\n" + + " aws:\n" + + " region: us-east-1\n" + + " sts_role_arn: arn:aws:iam::524239988912:role/app-test\n" + + " sts_header_overrides: {\"test\":\"test\"}\n" + + " max_retries: 3\n"; + + when(lambdaClient.invoke(any(InvokeRequest.class))).thenThrow(AwsServiceException.class); + doNothing().when(dlqPushHandler).perform(any(PluginSetting.class), any(LambdaSinkFailedDlqData.class)); + + this.lambdaSinkConfig = objectMapper.readValue(config, LambdaSinkConfig.class); + bufferFactory = mock(BufferFactory.class); + buffer = mock(Buffer.class); + ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); + when(buffer.getOutputStream()).thenReturn(byteArrayOutputStream); + when(bufferFactory.getBuffer(any(LambdaClient.class),any(),any())).thenReturn(buffer); + + doThrow(AwsServiceException.class).when(buffer).flushToLambda(); + + LambdaSinkService lambdaSinkService = new LambdaSinkService(lambdaClient, + lambdaSinkConfig, + pluginMetrics, + mock(PluginFactory.class), + mock(PluginSetting.class), + mock(OutputCodecContext.class), + mock(AwsCredentialsSupplier.class), + dlqPushHandler, + bufferFactory); + + final Record eventRecord = new Record<>(JacksonEvent.fromMessage("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}")); + Collection> records = List.of(eventRecord); + + lambdaSinkService.output(records); + + verify(buffer, times(3)).flushToLambda(); + verify(dlqPushHandler,times(1)).perform(any(PluginSetting.class),any(Object.class)); + } + + @Test + public void lambda_sink_test_with_multiple_record_success_push_to_lambda() throws IOException { + LambdaSinkService lambdaSinkService = createObjectUnderTest(config); + final Record eventRecord = new Record<>(JacksonEvent.fromMessage("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}")); + Collection> records = new ArrayList<>(); + int totalRecords = 11; + for(int recordSize = 0; recordSize < totalRecords ; recordSize++) { + records.add(eventRecord); + } + when(lambdaClient.invoke(any(InvokeRequest.class))).thenReturn(mockResponse); + + lambdaSinkService.output(records); + + verify(lambdaClient,times(totalRecords)).invoke(any(InvokeRequest.class)); + + } + + @Test + void lambda_sink_service_test_output_with_single_record_ack_release() throws IOException { + final LambdaSinkService lambdaSinkService = createObjectUnderTest(config); + final Event event = mock(Event.class); + given(event.toJsonString()).willReturn("{\"message\":\"c3f847eb-333a-49c3-a4cd-54715ad1b58a\"}"); + given(event.getEventHandle()).willReturn(mock(EventHandle.class)); + + final ArgumentCaptor invokeRequestCaptor = ArgumentCaptor.forClass(InvokeRequest.class); + when(lambdaClient.invoke(any(InvokeRequest.class))).thenReturn(mockResponse); + + lambdaSinkService.output(List.of(new Record<>(event))); + + verify(lambdaClient,times(1)).invoke(invokeRequestCaptor.capture()); + final InvokeRequest actualRequest = invokeRequestCaptor.getValue(); + assertThat(actualRequest.functionName(), equalTo("testFunction")); + assertThat(actualRequest.invocationType().toString(), equalTo("Event")); + verify(numberOfRecordsSuccessCounter).increment(1); + } + + @Test + public void lambda_sink_test_batch_enabled() throws IOException { + when(lambdaSinkConfig.getFunctionName()).thenReturn(functionName); + when(lambdaSinkConfig.getMaxConnectionRetries()).thenReturn(maxRetries); + when(lambdaSinkConfig.getBatchOptions()).thenReturn(mock(BatchOptions.class)); + when(lambdaSinkConfig.getBatchOptions().getBatchKey()).thenReturn(batchKey); + when(lambdaSinkConfig.getBatchOptions().getThresholdOptions()).thenReturn(mock(ThresholdOptions.class)); + when(lambdaSinkConfig.getBatchOptions().getThresholdOptions().getEventCount()).thenReturn(maxEvents); + when(lambdaSinkConfig.getBatchOptions().getThresholdOptions().getMaximumSize()).thenReturn(ByteCount.parse(maxSize)); + when(lambdaSinkConfig.getBatchOptions().getThresholdOptions().getEventCollectTimeOut()).thenReturn(Duration.ofNanos(10L)); + when(lambdaSinkConfig.getAwsAuthenticationOptions()).thenReturn(mock(AwsAuthenticationOptions.class)); + + LambdaSinkService lambdaSinkService = createObjectUnderTest(lambdaSinkConfig); + + Map map = new HashMap<>(); + map.put("query1","test1"); + map.put("query2","test2"); + + String expected_payload = "{\"lambda_batch_key\":[{\"query1\":\"test1\",\"query2\":\"test2\"}]}"; + final Record eventRecord = new Record<>(JacksonEvent.builder().withData(map).withEventType("event").build()); + Collection> records = List.of(eventRecord); + final ArgumentCaptor invokeRequestCaptor = ArgumentCaptor.forClass(InvokeRequest.class); + + when(lambdaClient.invoke(any(InvokeRequest.class))).thenReturn(mockResponse); + lambdaSinkService.output(records); + + verify(lambdaClient).invoke(invokeRequestCaptor.capture()); + final InvokeRequest actualRequest = invokeRequestCaptor.getValue(); + assertEquals(actualRequest.functionName(), functionName); + assertEquals(actualRequest.invocationType().toString(), "Event"); + String actualRequestPayload = actualRequest.payload().asUtf8String(); + assertEquals(actualRequestPayload, expected_payload ); + verify(numberOfRecordsSuccessCounter).increment(records.size()); + } +} diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkTest.java new file mode 100644 index 0000000000..1687cbd285 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkTest.java @@ -0,0 +1,82 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.lambda.LambdaClient; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class LambdaSinkTest { + + public static final String S3_REGION = "us-east-1"; + public static final String CODEC_PLUGIN_NAME = "json"; + public static final String SINK_PLUGIN_NAME = "lambda"; + public static final String SINK_PIPELINE_NAME = "lambda-sink-pipeline"; + private LambdaSinkConfig lambdaSinkConfig; + private LambdaSink lambdaSink; + private PluginSetting pluginSetting; + private PluginFactory pluginFactory; + private AwsCredentialsSupplier awsCredentialsSupplier; + private SinkContext sinkContext; + + @BeforeEach + void setUp() { + lambdaSinkConfig = mock(LambdaSinkConfig.class); + sinkContext = mock(SinkContext.class); + AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + pluginSetting = mock(PluginSetting.class); + PluginModel pluginModel = mock(PluginModel.class); + pluginFactory = mock(PluginFactory.class); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + Map dlqMap = mock(HashMap.class); + LambdaClient lambdaClient = mock(LambdaClient.class); + + + when(lambdaSinkConfig.getDlq()).thenReturn(pluginModel); + when(pluginModel.getPluginSettings()).thenReturn(dlqMap); + when(lambdaSinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of(S3_REGION)); + when(pluginModel.getPluginName()).thenReturn(CODEC_PLUGIN_NAME); + when(pluginSetting.getName()).thenReturn(SINK_PLUGIN_NAME); + when(pluginSetting.getPipelineName()).thenReturn(SINK_PIPELINE_NAME); + } + + private LambdaSink createObjectUnderTest() { + return new LambdaSink(pluginSetting, lambdaSinkConfig, pluginFactory, sinkContext, awsCredentialsSupplier); + } + + @Test + void test_lambda_sink_plugin_isReady_positive() { + lambdaSink = createObjectUnderTest(); + Assertions.assertNotNull(lambdaSink); + Assertions.assertNotNull(lambdaSinkConfig); + lambdaSink.doInitialize(); + assertTrue(lambdaSink.isReady(), "lambda sink is not initialized and not ready to work"); + } + + @Test + void test_lambda_sink_plugin_isReady_negative() { + lambdaSink = createObjectUnderTest(); + Assertions.assertNotNull(lambdaSink); + assertFalse(lambdaSink.isReady(), "lambda sink is initialized and ready to work"); + } +} diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheckTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheckTest.java new file mode 100644 index 0000000000..b63553911a --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheckTest.java @@ -0,0 +1,126 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda; + +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.model.types.ByteCount; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.Buffer; + +import java.io.IOException; +import java.time.Duration; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ThresholdCheckTest { + + @Mock(lenient = true) + private Buffer buffer; + private int maxEvents; + private ByteCount maxBytes; + private Duration maxCollectionDuration; + private Boolean isBatchEnabled; + + @BeforeEach + void setUp() { + maxEvents = 10_000; + maxBytes = ByteCount.parse("48mb"); + maxCollectionDuration = Duration.ofMinutes(5); + isBatchEnabled = true; + } + + @Test + void test_exceedThreshold_true_dueTo_maxEvents_is_greater_than_buffered_event_count() throws IOException { + when(buffer.getSize()).thenReturn(maxBytes.getBytes() - 1000); + when(buffer.getEventCount()).thenReturn(maxEvents + 1); + when(buffer.getDuration()).thenReturn(maxCollectionDuration.minusSeconds(1)); + + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(buffer, maxEvents, + maxBytes, maxCollectionDuration, isBatchEnabled); + + assertTrue(isThresholdExceed, "Threshold exceeded"); + } + + @Test + void test_exceedThreshold_false_dueTo_maxEvents_is_less_than_buffered_event_count() { + when(buffer.getSize()).thenReturn(maxBytes.getBytes() - 1000); + when(buffer.getEventCount()).thenReturn(maxEvents - 1); + when(buffer.getDuration()).thenReturn(this.maxCollectionDuration.minusSeconds(1)); + + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(buffer, maxEvents, + maxBytes, maxCollectionDuration, isBatchEnabled); + + assertFalse(isThresholdExceed, "Threshold exceeded"); + } + + @Test + void test_exceedThreshold_true_dueTo_maxBytes_is_greater_than_buffered_byte_count() { + when(buffer.getSize()).thenReturn(maxBytes.getBytes() + 1000); + when(buffer.getEventCount()).thenReturn(maxEvents - 1); + when(buffer.getDuration()).thenReturn(maxCollectionDuration.minusSeconds(1)); + + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(buffer, maxEvents, + maxBytes, maxCollectionDuration, isBatchEnabled); + + assertTrue(isThresholdExceed, "Threshold not exceeded"); + } + + @Test + void test_exceedThreshold_false_dueTo_maxBytes_is_less_than_buffered_byte_count() { + when(buffer.getSize()).thenReturn(maxBytes.getBytes() - 1000); + when(buffer.getEventCount()).thenReturn(maxEvents - 1); + when(buffer.getDuration()).thenReturn(maxCollectionDuration.minusSeconds(1)); + + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(buffer, maxEvents, + maxBytes, maxCollectionDuration, isBatchEnabled); + + assertFalse(isThresholdExceed, "Threshold exceeded"); + } + + @Test + void test_exceedThreshold_true_dueTo_maxCollectionDuration_is_greater_than_buffered_event_collection_duration() { + when(buffer.getSize()).thenReturn(maxBytes.getBytes() - 1000); + when(buffer.getEventCount()).thenReturn(maxEvents - 1); + when(buffer.getDuration()).thenReturn(maxCollectionDuration.plusSeconds(1)); + + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(buffer, maxEvents, + maxBytes, maxCollectionDuration, isBatchEnabled); + + assertTrue(isThresholdExceed, "Threshold not exceeded"); + } + + @Test + void test_exceedThreshold_false_dueTo_maxCollectionDuration_is_less_than_buffered_event_collection_duration() { + when(buffer.getSize()).thenReturn(maxBytes.getBytes() - 1000); + when(buffer.getEventCount()).thenReturn(maxEvents - 1); + when(buffer.getDuration()).thenReturn(maxCollectionDuration.minusSeconds(1)); + + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(buffer, maxEvents, + maxBytes, maxCollectionDuration, isBatchEnabled); + + + assertFalse(isThresholdExceed, "Threshold exceeded"); + } + + @Test + void test_exceedThreshold_when_batch_is_enabled() throws IOException { + when(buffer.getSize()).thenReturn(maxBytes.getBytes() - 1000); + when(buffer.getEventCount()).thenReturn(maxEvents + 1); + when(buffer.getDuration()).thenReturn(maxCollectionDuration.minusSeconds(1)); + Boolean isBatchEnabled = false; + + boolean isThresholdExceed = ThresholdCheck.checkThresholdExceed(buffer, maxEvents, + maxBytes, maxCollectionDuration, isBatchEnabled); + + assertTrue(isThresholdExceed); + } +} diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferFactoryTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferFactoryTest.java new file mode 100644 index 0000000000..d161b28bb0 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferFactoryTest.java @@ -0,0 +1,32 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.accumulator; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBufferFactory; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; + +class InMemoryBufferFactoryTest { + + @Test + void test_inMemoryBufferFactory_notNull(){ + InMemoryBufferFactory inMemoryBufferFactory = new InMemoryBufferFactory(); + Assertions.assertNotNull(inMemoryBufferFactory); + } + + @Test + void test_buffer_notNull(){ + InMemoryBufferFactory inMemoryBufferFactory = new InMemoryBufferFactory(); + Assertions.assertNotNull(inMemoryBufferFactory); + Buffer buffer = inMemoryBufferFactory.getBuffer(null, null, null); + Assertions.assertNotNull(buffer); + assertThat(buffer, instanceOf(Buffer.class)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferTest.java new file mode 100644 index 0000000000..478650a300 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferTest.java @@ -0,0 +1,167 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.accumulator; + +import org.hamcrest.Matchers; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Disabled; +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.lambda.accumlator.InMemoryBuffer; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.lambda.LambdaClient; +import software.amazon.awssdk.services.lambda.model.InvokeRequest; +import software.amazon.awssdk.services.lambda.model.InvokeResponse; + +import java.io.IOException; +import java.io.OutputStream; +import java.time.Duration; +import java.time.Instant; +import java.time.temporal.ChronoUnit; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +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.when; + +@ExtendWith(MockitoExtension.class) +class InMemoryBufferTest { + + public static final int MAX_EVENTS = 55; + @Mock + private LambdaClient lambdaClient; + + private final String functionName = "testFunction"; + + private final String invocationType = "Event"; + + private InMemoryBuffer inMemoryBuffer; + + @Test + void test_with_write_event_into_buffer() throws IOException { + inMemoryBuffer = new InMemoryBuffer(lambdaClient, functionName, invocationType); + + while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); + } + assertThat(inMemoryBuffer.getSize(), greaterThanOrEqualTo(54110L)); + assertThat(inMemoryBuffer.getEventCount(), equalTo(MAX_EVENTS)); + assertThat(inMemoryBuffer.getDuration(), notNullValue()); + assertThat(inMemoryBuffer.getDuration(), greaterThanOrEqualTo(Duration.ZERO)); + + } + + @Test + @Disabled("unstable") + /** + * There are 5 checkpoints in the tests as below + * |-----------upperBoundDuration-------------| + * startTime --- stopWatchStart --- endTime --- checkpoint --- stopwatchGetDuration + * |-lowerBoundDuration-| + * |------------inMemoryBuffer.Duration-------------| + * This test assumes the startTime and stopWatchStart are same, checkpoint and stopwatchGetDuration are same. + * However, they are not true at some systems. + */ + void getDuration_provides_duration_within_expected_range() throws IOException, InterruptedException { + Instant startTime = Instant.now(); + inMemoryBuffer = new InMemoryBuffer(lambdaClient, functionName, invocationType); + Instant endTime = Instant.now(); + + + while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); + } + Thread.sleep(100); + + Instant durationCheckpointTime = Instant.now(); + Duration duration = inMemoryBuffer.getDuration(); + assertThat(duration, notNullValue()); + + Duration upperBoundDuration = Duration.between(startTime, durationCheckpointTime).truncatedTo(ChronoUnit.MILLIS); + Duration lowerBoundDuration = Duration.between(endTime, durationCheckpointTime).truncatedTo(ChronoUnit.MILLIS); + assertThat(duration, greaterThanOrEqualTo(lowerBoundDuration)); + assertThat(duration, lessThanOrEqualTo(upperBoundDuration)); + } + + @Test + void test_with_write_event_into_buffer_and_flush_toLambda() throws IOException { + + // Mock the response of the invoke method + InvokeResponse mockResponse = InvokeResponse.builder() + .statusCode(200) // HTTP 200 for successful invocation + .payload(SdkBytes.fromString("{\"key\": \"value\"}", java.nio.charset.StandardCharsets.UTF_8)) + .build(); + when(lambdaClient.invoke(any(InvokeRequest.class))).thenReturn(mockResponse); + + inMemoryBuffer = new InMemoryBuffer(lambdaClient, functionName, invocationType); + while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + int eventCount = inMemoryBuffer.getEventCount() +1; + inMemoryBuffer.setEventCount(eventCount); + } + assertDoesNotThrow(() -> { + inMemoryBuffer.flushToLambda(); + }); + } + + @Test + void test_uploadedToLambda_success() throws IOException { + // Mock the response of the invoke method + InvokeResponse mockResponse = InvokeResponse.builder() + .statusCode(200) // HTTP 200 for successful invocation + .payload(SdkBytes.fromString("{\"key\": \"value\"}", java.nio.charset.StandardCharsets.UTF_8)) + .build(); + when(lambdaClient.invoke(any(InvokeRequest.class))).thenReturn(mockResponse); + inMemoryBuffer = new InMemoryBuffer(lambdaClient, functionName, invocationType); + Assertions.assertNotNull(inMemoryBuffer); + OutputStream outputStream = inMemoryBuffer.getOutputStream(); + outputStream.write(generateByteArray()); + assertDoesNotThrow(() -> { + inMemoryBuffer.flushToLambda(); + }); + } + + @Test + void test_uploadedToLambda_fails() { + // Mock the response of the invoke method + InvokeResponse mockResponse = InvokeResponse.builder() + .statusCode(200) // HTTP 200 for successful invocation + .payload(SdkBytes.fromString("{\"key\": \"value\"}", java.nio.charset.StandardCharsets.UTF_8)) + .build(); + SdkClientException sdkClientException = mock(SdkClientException.class); + when(lambdaClient.invoke(any(InvokeRequest.class))) + .thenThrow(sdkClientException); + inMemoryBuffer = new InMemoryBuffer(lambdaClient, functionName, invocationType); + + Assertions.assertNotNull(inMemoryBuffer); + SdkClientException actualException = assertThrows(SdkClientException.class, () -> inMemoryBuffer.flushToLambda()); + assertThat(actualException, Matchers.equalTo(sdkClientException)); + } + + private byte[] generateByteArray() { + byte[] bytes = new byte[1000]; + for (int i = 0; i < 1000; i++) { + bytes[i] = (byte) i; + } + return bytes; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodecTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodecTest.java new file mode 100644 index 0000000000..6de6ce8a0e --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodecTest.java @@ -0,0 +1,110 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.codec; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.JsonNodeType; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.model.sink.OutputCodecContext; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; + +class LambdaJsonCodecTest { + private ByteArrayOutputStream outputStream; + + private LambdaJsonCodec createObjectUnderTest() { + String key = "event"; + return new LambdaJsonCodec(key); + } + + @Test + void test_happy_case_with_null_codec_key() throws IOException { + LambdaJsonCodec LambdaJsonCodec = new LambdaJsonCodec(null); + + outputStream = new ByteArrayOutputStream(); + OutputCodecContext codecContext = new OutputCodecContext(); + LambdaJsonCodec.start(outputStream, null, codecContext); + + final List> expectedData = generateRecords(1); + final Event event = convertToEvent(expectedData.get(0)); + LambdaJsonCodec.writeEvent(event, outputStream); + LambdaJsonCodec.complete(outputStream); + + ObjectMapper mapper = new ObjectMapper(); + JsonNode jsonNode = mapper.readTree(outputStream.toByteArray()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.OBJECT)); + assertEquals(jsonNode.toString(),"{\"name\":\"Person0\",\"age\":0}"); + } + + + @Test + void test_happy_case_with_codec_key() throws IOException { + String key = "events"; + final int numberOfRecords = 2; + LambdaJsonCodec LambdaJsonCodec = new LambdaJsonCodec(key); + + outputStream = new ByteArrayOutputStream(); + OutputCodecContext codecContext = new OutputCodecContext(); + LambdaJsonCodec.start(outputStream, null, codecContext); + + final List> expectedData = generateRecords(numberOfRecords); + for (int index = 0; index < numberOfRecords; index++) { + final Event event = convertToEvent(expectedData.get(index)); + LambdaJsonCodec.writeEvent(event, outputStream); + } + LambdaJsonCodec.complete(outputStream); + + String expectedString = "{\"events\":[{\"name\":\"Person0\",\"age\":0},{\"name\":\"Person1\",\"age\":1}]}"; + ObjectMapper mapper = new ObjectMapper(); + JsonNode jsonNode = mapper.readTree(outputStream.toByteArray()); + assertThat(jsonNode.getNodeType(), equalTo(JsonNodeType.OBJECT)); + assertEquals(jsonNode.toString(),expectedString); + } + + private static Event convertToEvent(Map data) { + return JacksonLog.builder().withData(data).build(); + } + + private static List> generateRecords(int numberOfRecords) { + + List> recordList = new ArrayList<>(); + + for (int rows = 0; rows < numberOfRecords; rows++) { + + Map eventData = new HashMap<>(); + + eventData.put("name", "Person" + rows); + eventData.put("age", rows); + recordList.add(eventData); + + } + + return recordList; + } + + + private Object getValue(JsonNode jsonNode) { + if(jsonNode.isTextual()) + return jsonNode.asText(); + + if(jsonNode.isInt()) + return jsonNode.asInt(); + + throw new RuntimeException("Test not setup correctly."); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptionsTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptionsTest.java new file mode 100644 index 0000000000..53bd0a4edf --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptionsTest.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.lambda.config; + +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.types.ByteCount; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +class ThresholdOptionsTest { + private static final String DEFAULT_BYTE_CAPACITY = "6mb"; + private static final int DEFAULT_EVENT_COUNT = 0; + + @Test + void test_default_byte_capacity_test() { + assertThat(new ThresholdOptions().getMaximumSize().getBytes(), + equalTo(ByteCount.parse(DEFAULT_BYTE_CAPACITY).getBytes())); + } + + @Test + void test_get_event_collection_duration_test() { + assertThat(new ThresholdOptions().getEventCollectTimeOut(), equalTo(null)); + } + + @Test + void test_get_event_count_test() { + assertThat(new ThresholdOptions().getEventCount(), equalTo(DEFAULT_EVENT_COUNT)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandlerTest.java b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandlerTest.java new file mode 100644 index 0000000000..17f39973b7 --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandlerTest.java @@ -0,0 +1,88 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.lambda.dlq; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.plugins.dlq.DlqProvider; +import org.opensearch.dataprepper.plugins.dlq.DlqWriter; +import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import software.amazon.awssdk.core.SdkBytes; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +class DlqPushHandlerTest { + + private static final String BUCKET = "bucket"; + private static final String BUCKET_VALUE = "test"; + private static final String ROLE = "arn:aws:iam::524239988122:role/app-test"; + + private static final String REGION = "ap-south-1"; + private static final String S3_PLUGIN_NAME = "s3"; + private static final String KEY_PATH_PREFIX = "key_path_prefix"; + + private static final String KEY_PATH_PREFIX_VALUE = "dlq/"; + + private static final String PIPELINE_NAME = "log-pipeline"; + + private static final String DLQ_FILE = "local_dlq_file"; + + private PluginModel pluginModel; + + private DlqPushHandler dlqPushHandler; + private PluginFactory pluginFactory; + + private AwsAuthenticationOptions awsAuthenticationOptions; + + private DlqProvider dlqProvider; + + private DlqWriter dlqWriter; + + + @BeforeEach + public void setUp(){ + this.pluginFactory = mock(PluginFactory.class); + this.pluginModel = mock(PluginModel.class); + this.awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + this.dlqProvider = mock(DlqProvider.class); + this.dlqWriter = mock(DlqWriter.class); + } + + @Test + void perform_for_dlq_s3_success() throws IOException { + Map props = new HashMap<>(); + props.put(BUCKET,BUCKET_VALUE); + props.put(KEY_PATH_PREFIX,KEY_PATH_PREFIX_VALUE); + + when(pluginFactory.loadPlugin(any(Class.class), any(PluginSetting.class))).thenReturn(dlqProvider); + + when(dlqProvider.getDlqWriter(anyString())).thenReturn(Optional.of(dlqWriter)); + doNothing().when(dlqWriter).write(anyList(), anyString(), anyString()); + SdkBytes payload = SdkBytes.fromUtf8String("{\"name\":\"dataprepper\"}"); + LambdaSinkFailedDlqData failedDlqData = new LambdaSinkFailedDlqData(payload,"message",0); + dlqPushHandler = new DlqPushHandler(pluginFactory, BUCKET_VALUE, ROLE, REGION,KEY_PATH_PREFIX_VALUE); + + PluginSetting pluginSetting = new PluginSetting(S3_PLUGIN_NAME, props); + pluginSetting.setPipelineName(PIPELINE_NAME); + dlqPushHandler.perform(pluginSetting, failedDlqData); + Assertions.assertNotNull(pluginFactory); + verify(dlqWriter).write(anyList(), anyString(), anyString()); + } +} diff --git a/data-prepper-plugins/lambda-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/lambda-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 0000000000..23c33feb6d --- /dev/null +++ b/data-prepper-plugins/lambda-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,3 @@ +# To enable mocking of final classes with vanilla Mockito +# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods +mock-maker-inline diff --git a/settings.gradle b/settings.gradle index a2495d9ffc..0a7718aa4d 100644 --- a/settings.gradle +++ b/settings.gradle @@ -169,8 +169,9 @@ include 'data-prepper-plugins:dissect-processor' include 'data-prepper-plugins:dynamodb-source' include 'data-prepper-plugins:decompress-processor' include 'data-prepper-plugins:split-event-processor' -include 'data-prepper-plugins:http-common' include 'data-prepper-plugins:flatten-processor' include 'data-prepper-plugins:mongodb' include 'data-prepper-plugins:rds-source' -include 'data-prepper-plugins:http-source-common' \ No newline at end of file +include 'data-prepper-plugins:http-source-common' +include 'data-prepper-plugins:http-common' +include 'data-prepper-plugins:lambda-sink' \ No newline at end of file From 0584573a7206630dcbc9f5b67256929bb73fe494 Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 7 Jun 2024 16:07:03 -0500 Subject: [PATCH 009/159] Updates Python requests to 2.32.3 in the smoke tests project to address CVE-2024-35195. (#4610) Signed-off-by: David Venable --- release/smoke-tests/otel-span-exporter/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/release/smoke-tests/otel-span-exporter/requirements.txt b/release/smoke-tests/otel-span-exporter/requirements.txt index 70fa887422..3cd451baf4 100644 --- a/release/smoke-tests/otel-span-exporter/requirements.txt +++ b/release/smoke-tests/otel-span-exporter/requirements.txt @@ -13,7 +13,7 @@ opentelemetry-proto==1.7.1 opentelemetry-sdk==1.7.1 opentelemetry-semantic-conventions==0.26b1 protobuf==3.19.5 -requests==2.31.0 +requests==2.32.3 six==1.16.0 urllib3==1.26.18 wrapt==1.13.3 From ad92aa25b80a4951176f8c45c475dc6d704c2e75 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Mon, 10 Jun 2024 14:55:09 -0700 Subject: [PATCH 010/159] Aggrerate processor : add option to allow raw events (#4598) * Aggregate Processor: Add support to allow raw events Signed-off-by: Krishna Kondaka * Modify test to check for aggregated tag Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../aggregate/AggregateProcessor.java | 13 ++++++ .../aggregate/AggregateProcessorConfig.java | 20 +++++++++ .../aggregate/AggregateProcessorIT.java | 45 +++++++++++++++++++ .../aggregate/AggregateProcessorTest.java | 29 ++++++++++++ 4 files changed, 107 insertions(+) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java index 823ae7f995..8a6f9d5a6a 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java @@ -49,6 +49,8 @@ public class AggregateProcessor extends AbstractProcessor, Record< private boolean localMode = false; private final String whenCondition; private final ExpressionEvaluator expressionEvaluator; + private final boolean outputUnaggregatedEvents; + private final String aggregatedEventsTag; @DataPrepperPluginConstructor public AggregateProcessor(final AggregateProcessorConfig aggregateProcessorConfig, final PluginMetrics pluginMetrics, final PluginFactory pluginFactory, final ExpressionEvaluator expressionEvaluator) { @@ -59,7 +61,9 @@ public AggregateProcessor(final AggregateProcessorConfig aggregateProcessorConfi final IdentificationKeysHasher identificationKeysHasher, final AggregateActionSynchronizer.AggregateActionSynchronizerProvider aggregateActionSynchronizerProvider, final ExpressionEvaluator expressionEvaluator) { super(pluginMetrics); this.aggregateProcessorConfig = aggregateProcessorConfig; + this.aggregatedEventsTag = aggregateProcessorConfig.getAggregatedEventsTag(); this.aggregateGroupManager = aggregateGroupManager; + this.outputUnaggregatedEvents = aggregateProcessorConfig.getOutputUnaggregatedEvents(); this.expressionEvaluator = expressionEvaluator; this.identificationKeysHasher = identificationKeysHasher; this.aggregateAction = loadAggregateAction(pluginFactory); @@ -92,6 +96,9 @@ public Collection> doExecute(Collection> records) { final List concludeGroupEvents = actionOutput != null ? actionOutput.getEvents() : null; if (!concludeGroupEvents.isEmpty()) { concludeGroupEvents.stream().forEach((event) -> { + if (aggregatedEventsTag != null) { + event.getMetadata().addTags(List.of(aggregatedEventsTag)); + } recordsOut.add(new Record(event)); actionConcludeGroupEventsOutCounter.increment(); }); @@ -116,11 +123,17 @@ public Collection> doExecute(Collection> records) { final Event aggregateActionResponseEvent = handleEventResponse.getEvent(); if (aggregateActionResponseEvent != null) { + if (aggregatedEventsTag != null) { + aggregateActionResponseEvent.getMetadata().addTags(List.of(aggregatedEventsTag)); + } recordsOut.add(new Record<>(aggregateActionResponseEvent, record.getMetadata())); handleEventsOut++; } else { handleEventsDropped++; } + if (outputUnaggregatedEvents) { + recordsOut.add(record); + } } actionHandleEventsOutCounter.increment(handleEventsOut); diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java index ed4f35de79..7f81a82194 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java @@ -7,6 +7,7 @@ import org.opensearch.dataprepper.model.configuration.PluginModel; import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -32,6 +33,12 @@ public class AggregateProcessorConfig { @NotNull private Boolean localMode = false; + @JsonProperty("output_unaggregated_events") + private Boolean outputUnaggregatedEvents = false; + + @JsonProperty("aggregated_events_tag") + private String aggregatedEventsTag; + @JsonProperty("aggregate_when") private String whenCondition; @@ -47,10 +54,23 @@ public String getWhenCondition() { return whenCondition; } + public String getAggregatedEventsTag() { + return aggregatedEventsTag; + } + + public Boolean getOutputUnaggregatedEvents() { + return outputUnaggregatedEvents; + } + public Boolean getLocalMode() { return localMode; } + @AssertTrue(message="Aggragated Events Tag must be set when output_unaggregated_events is set") + boolean isValidConfig() { + return (!outputUnaggregatedEvents || (outputUnaggregatedEvents && aggregatedEventsTag != null)); + } + public PluginModel getAggregateAction() { return aggregateAction; } } diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java index 9e68187255..ea7b6eb416 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java @@ -127,6 +127,7 @@ void setup() { pluginMetrics = PluginMetrics.fromNames(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + when(aggregateProcessorConfig.getOutputUnaggregatedEvents()).thenReturn(false); when(aggregateProcessorConfig.getIdentificationKeys()).thenReturn(identificationKeys); when(aggregateProcessorConfig.getAggregateAction()).thenReturn(actionConfiguration); when(actionConfiguration.getPluginName()).thenReturn(UUID.randomUUID().toString()); @@ -445,6 +446,50 @@ void aggregateWithCountAggregateActionWithCondition() throws InterruptedExceptio assertThat(record.getData().toMap(), hasKey(DEFAULT_START_TIME_KEY)); } + @RepeatedTest(value = 2) + void aggregateWithCountAggregateActionWithUnaggregatedEvents() throws InterruptedException, NoSuchFieldException, IllegalAccessException { + when(aggregateProcessorConfig.getOutputUnaggregatedEvents()).thenReturn(true); + String tag = UUID.randomUUID().toString(); + when(aggregateProcessorConfig.getAggregatedEventsTag()).thenReturn(tag); + CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig(); + setField(CountAggregateActionConfig.class, countAggregateActionConfig, "outputFormat", OutputFormat.RAW.toString()); + aggregateAction = new CountAggregateAction(countAggregateActionConfig); + when(pluginFactory.loadPlugin(eq(AggregateAction.class), any(PluginSetting.class))) + .thenReturn(aggregateAction); + when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); + eventBatch = getBatchOfEvents(true); + + final AggregateProcessor objectUnderTest = createObjectUnderTest(); + + final ExecutorService executorService = Executors.newFixedThreadPool(NUM_THREADS); + final CountDownLatch countDownLatch = new CountDownLatch(NUM_THREADS); + + for (int i = 0; i < NUM_THREADS; i++) { + executorService.execute(() -> { + final List> recordsOut = (List>) objectUnderTest.doExecute(eventBatch); + assertThat(recordsOut.size(), equalTo(NUM_EVENTS_PER_BATCH)); + countDownLatch.countDown(); + }); + } + // wait longer so that the raw events are processed. + Thread.sleep(2*GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + + boolean allThreadsFinished = countDownLatch.await(5L, TimeUnit.SECONDS); + assertThat(allThreadsFinished, equalTo(true)); + + Collection> results = objectUnderTest.doExecute(new ArrayList>()); + assertThat(results.size(), equalTo(1)); + + Map expectedEventMap = new HashMap<>(getEventMap(testValue)); + expectedEventMap.put(DEFAULT_COUNT_KEY, NUM_THREADS * NUM_EVENTS_PER_BATCH); + + final Record record = (Record)results.toArray()[0]; + assertTrue(record.getData().getMetadata().hasTags(List.of(tag))); + expectedEventMap.forEach((k, v) -> assertThat(record.getData().toMap(), hasEntry(k,v))); + assertThat(record.getData().toMap(), hasKey(DEFAULT_START_TIME_KEY)); + } + + @RepeatedTest(value = 2) void aggregateWithHistogramAggregateAction() throws InterruptedException, NoSuchFieldException, IllegalAccessException { 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 4c6203ad71..6e60e7d965 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 @@ -41,6 +41,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.verify; @@ -124,6 +125,7 @@ private AggregateProcessor createObjectUnderTest() { @BeforeEach void setUp() { when(aggregateProcessorConfig.getAggregateAction()).thenReturn(actionConfiguration); + when(aggregateProcessorConfig.getOutputUnaggregatedEvents()).thenReturn(false); when(aggregateProcessorConfig.getLocalMode()).thenReturn(false); when(actionConfiguration.getPluginName()).thenReturn(UUID.randomUUID().toString()); when(actionConfiguration.getPluginSettings()).thenReturn(Collections.emptyMap()); @@ -400,6 +402,33 @@ void handleEvent_returning_with_event_adds_event_to_records_out() { verify(aggregateGroupManager).getGroupsToConclude(eq(false)); } + @Test + void handleEvent_returning_with_event_adds_event_to_records_out_with_output_unaggregated_events() { + when(aggregateProcessorConfig.getOutputUnaggregatedEvents()).thenReturn(true); + String tag = UUID.randomUUID().toString(); + when(aggregateProcessorConfig.getAggregatedEventsTag()).thenReturn(tag); + final AggregateProcessor objectUnderTest = createObjectUnderTest(); + final Map.Entry groupEntry = new AbstractMap.SimpleEntry(identificationKeysMap, aggregateGroup); + when(aggregateGroupManager.getGroupsToConclude(eq(false))).thenReturn(Collections.singletonList(groupEntry)); + when(aggregateActionResponse.getEvent()).thenReturn(event); + when(aggregateActionSynchronizer.concludeGroup(identificationKeysMap, aggregateGroup, false)).thenReturn(new AggregateActionOutput(List.of())); + + final List> recordsOut = (List>) objectUnderTest.doExecute(Collections.singletonList(new Record<>(event))); + + assertThat(recordsOut.size(), equalTo(2)); + assertThat(recordsOut.get(0), notNullValue()); + assertThat(recordsOut.get(0).getData(), equalTo(event)); + assertThat(recordsOut.get(1), notNullValue()); + assertThat(recordsOut.get(1).getData(), equalTo(event)); + Event receivedEvent = recordsOut.get(1).getData(); + assertTrue(receivedEvent.getMetadata().hasTags(List.of(tag))); + + verify(actionHandleEventsOutCounter).increment(1); + verify(actionHandleEventsDroppedCounter).increment(0); + verifyNoInteractions(actionConcludeGroupEventsOutCounter); + + verify(aggregateGroupManager).getGroupsToConclude(eq(false)); + } @Test void concludeGroup_returning_with_no_event_does_not_add_event_to_records_out() { From 3ef20b0fe6b337a6dd30e538b762f90f1261d503 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jo=C3=ABl=20Marty?= <134835+joelmarty@users.noreply.github.com> Date: Wed, 12 Jun 2024 19:45:37 +0200 Subject: [PATCH 011/159] Fix missing closing parenthesis in CLOUDFRONT_ACCESS_LOG pattern (#4607) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Joël Marty <134835+joelmarty@users.noreply.github.com> --- .../grok-processor/src/main/resources/grok-patterns/patterns | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/data-prepper-plugins/grok-processor/src/main/resources/grok-patterns/patterns b/data-prepper-plugins/grok-processor/src/main/resources/grok-patterns/patterns index bb433620d7..b5d14ae632 100644 --- a/data-prepper-plugins/grok-processor/src/main/resources/grok-patterns/patterns +++ b/data-prepper-plugins/grok-processor/src/main/resources/grok-patterns/patterns @@ -14,6 +14,6 @@ ELB_ACCESS_LOG %{TIMESTAMP_ISO8601:timestamp}\s%{NOTSPACE:elb}\s%{IP:clientip}:% S3_HTTP_REQUEST ((?:%{WORD:verb}\s%{NOTSPACE:request}\s(?:HTTP/%{NUMBER:httpversion}))?|%{DATA:rawrequest}) S3_ACCESS_LOG %{WORD:owner}\s%{NOTSPACE:bucket}\s\[%{HTTPDATE:timestamp}\]\s%{IP:clientip}\s%{NOTSPACE:requester}\s%{NOTSPACE:request_id}\s%{NOTSPACE:operation}\s%{NOTSPACE:key}\s(?:-|"%{S3_HTTP_REQUEST}")\s(?:-|%{INT:response:int})\s(?:-|%{NOTSPACE:error_code})\s(?:-|%{INT:bytes_sent:int})\s(?:-|%{INT:object_size:int})\s(?:-|%{INT:request_time_ms:int})\s(?:-|%{INT:turnaround_time_ms:int})\s(?:%{QS:referrer})\s(?:-|"?%{QS:agent}"?)\s(?:-|%{NOTSPACE:version_id}) -CLOUDFRONT_ACCESS_LOG (?%{YEAR}-%{MONTHNUM}-%{MONTHDAY}\s%{TIME})\s%{NOTSPACE:x_edge_location}\s(?:-|%{NUMBER:sc_bytes:int})\s%{IPORHOST:clientip}\s%{WORD:cs_method}\s%{HOSTNAME:cs_host}\s%{NOTSPACE:cs_uri_stem}\s(?:-|%{NUMBER:sc_status:int})\s%{GREEDYDATA:referrer}\s%{GREEDYDATA:agent}\s%{GREEDYDATA:cs_uri_query}\s%{GREEDYDATA:cookies}\s%{WORD:x_edge_result_type}\s%{NOTSPACE:x_edge_request_id}\s%{HOSTNAME:x_host_header}\s%{URIPROTO:cs_protocol}\s(?:-|%{INT:cs_bytes:int})\s(?:-|%{GREEDYDATA:time_taken:float}\s%{GREEDYDATA:x_forwarded_for}\s%{GREEDYDATA:ssl_protocol}\s%{GREEDYDATA:ssl_cipher}\s%{GREEDYDATA:x_edge_response_result_type} +CLOUDFRONT_ACCESS_LOG (?%{YEAR}-%{MONTHNUM}-%{MONTHDAY}\s%{TIME})\s%{NOTSPACE:x_edge_location}\s(?:-|%{NUMBER:sc_bytes:int})\s%{IPORHOST:clientip}\s%{WORD:cs_method}\s%{HOSTNAME:cs_host}\s%{NOTSPACE:cs_uri_stem}\s(?:-|%{NUMBER:sc_status:int})\s%{GREEDYDATA:referrer}\s%{GREEDYDATA:agent}\s%{GREEDYDATA:cs_uri_query}\s%{GREEDYDATA:cookies}\s%{WORD:x_edge_result_type}\s%{NOTSPACE:x_edge_request_id}\s%{HOSTNAME:x_host_header}\s%{URIPROTO:cs_protocol}\s(?:-|%{INT:cs_bytes:int})\s(?:-|%{GREEDYDATA:time_taken:float})\s%{GREEDYDATA:x_forwarded_for}\s%{GREEDYDATA:ssl_protocol}\s%{GREEDYDATA:ssl_cipher}\s%{GREEDYDATA:x_edge_response_result_type} -VPC_FLOW_LOG %{NUMBER:version}\s%{NUMBER:account-id}\s%{NOTSPACE:interface-id}\s%{NOTSPACE:srcaddr}\s%{NOTSPACE:dstaddr}\s(?:-|%{NOTSPACE:srcport:int})\s(?:-|%{NOTSPACE:dstport:int})\s(?:-|%{NOTSPACE:protocol:int})\s(?:-|%{NOTSPACE:packets:int})\s(?:-|%{NOTSPACE:bytes:int})\s(?:-|%{NUMBER:start:int})\s(?:-|%{NUMBER:end:int})\s%{NOTSPACE:action}\s%{NOTSPACE:log-status} \ No newline at end of file +VPC_FLOW_LOG %{NUMBER:version}\s%{NUMBER:account-id}\s%{NOTSPACE:interface-id}\s%{NOTSPACE:srcaddr}\s%{NOTSPACE:dstaddr}\s(?:-|%{NOTSPACE:srcport:int})\s(?:-|%{NOTSPACE:dstport:int})\s(?:-|%{NOTSPACE:protocol:int})\s(?:-|%{NOTSPACE:packets:int})\s(?:-|%{NOTSPACE:bytes:int})\s(?:-|%{NUMBER:start:int})\s(?:-|%{NUMBER:end:int})\s%{NOTSPACE:action}\s%{NOTSPACE:log-status} From af7d1b54561a52ac13e7ba04d2b73a77cb0dc5cb Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Wed, 12 Jun 2024 14:46:08 -0500 Subject: [PATCH 012/159] FIX: decouple msk auth from glue auth in KafkaSource (#4613) * FIX: decouple msk from aws block Signed-off-by: George Chen --- .../kafka/util/KafkaSecurityConfigurer.java | 68 ++++++-- .../util/KafkaSecurityConfigurerTest.java | 145 ++++++++++++++++++ ...peline-bootstrap-servers-glue-default.yaml | 14 ++ ...ootstrap-servers-glue-sts-assume-role.yaml | 17 ++ ...ine-bootstrap-servers-override-by-msk.yaml | 20 +++ ...ne-bootstrap-servers-sasl-iam-default.yaml | 15 ++ ...eline-bootstrap-servers-sasl-iam-role.yaml | 18 +++ ...line-msk-default-glue-sts-assume-role.yaml | 20 +++ .../kafka-pipeline-msk-sasl-plain.yaml | 20 +++ 9 files changed, 320 insertions(+), 17 deletions(-) create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-glue-default.yaml create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-glue-sts-assume-role.yaml create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-override-by-msk.yaml create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-sasl-iam-default.yaml create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-sasl-iam-role.yaml create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-msk-default-glue-sts-assume-role.yaml create mode 100644 data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-msk-sasl-plain.yaml diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurer.java index a5e27e4d98..402f248ddf 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurer.java @@ -92,7 +92,8 @@ public class KafkaSecurityConfigurer { private static final String SSL_TRUSTSTORE_LOCATION = "ssl.truststore.location"; private static final String SSL_TRUSTSTORE_PASSWORD = "ssl.truststore.password"; - private static AwsCredentialsProvider credentialsProvider; + private static AwsCredentialsProvider mskCredentialsProvider; + private static AwsCredentialsProvider awsGlueCredentialsProvider; private static GlueSchemaRegistryKafkaDeserializer glueDeserializer; @@ -207,6 +208,9 @@ public static void setAwsIamAuthProperties(Properties properties, final AwsIamAu properties.put(SASL_MECHANISM, "AWS_MSK_IAM"); properties.put(SASL_CLIENT_CALLBACK_HANDLER_CLASS, "software.amazon.msk.auth.iam.IAMClientCallbackHandler"); if (awsIamAuthConfig == AwsIamAuthConfig.ROLE) { + if (Objects.isNull(awsConfig)) { + throw new RuntimeException("AWS Config needs to be specified when sasl/aws_msk_iam is set to \"role\""); + } String baseIamAuthConfig = "software.amazon.msk.auth.iam.IAMLoginModule required " + "awsRoleArn=\"%s\" " + "awsStsRegion=\"%s\""; @@ -225,14 +229,16 @@ public static void setAwsIamAuthProperties(Properties properties, final AwsIamAu } } - public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuthConfig, final AwsConfig awsConfig, final Logger LOG) { - if (awsIamAuthConfig == AwsIamAuthConfig.ROLE) { + private static void configureMSKCredentialsProvider(final AuthConfig authConfig, final AwsConfig awsConfig) { + mskCredentialsProvider = DefaultCredentialsProvider.create(); + if (Objects.nonNull(authConfig) && Objects.nonNull(authConfig.getSaslAuthConfig()) && + authConfig.getSaslAuthConfig().getAwsIamAuthConfig() == AwsIamAuthConfig.ROLE) { String sessionName = "data-prepper-kafka-session" + UUID.randomUUID(); StsClient stsClient = StsClient.builder() .region(Region.of(awsConfig.getRegion())) - .credentialsProvider(credentialsProvider) + .credentialsProvider(mskCredentialsProvider) .build(); - credentialsProvider = StsAssumeRoleCredentialsProvider + mskCredentialsProvider = StsAssumeRoleCredentialsProvider .builder() .stsClient(stsClient) .refreshRequest( @@ -242,12 +248,15 @@ public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuth .roleSessionName(sessionName) .build() ).build(); - } else if (awsIamAuthConfig != AwsIamAuthConfig.DEFAULT) { - throw new RuntimeException("Unknown AWS IAM auth mode"); } + } + + public static String getBootStrapServersForMsk(final AwsConfig awsConfig, + final AwsCredentialsProvider mskCredentialsProvider, + final Logger log) { final AwsConfig.AwsMskConfig awsMskConfig = awsConfig.getAwsMskConfig(); KafkaClient kafkaClient = KafkaClient.builder() - .credentialsProvider(credentialsProvider) + .credentialsProvider(mskCredentialsProvider) .region(Region.of(awsConfig.getRegion())) .build(); final GetBootstrapBrokersRequest request = @@ -264,7 +273,7 @@ public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuth try { result = kafkaClient.getBootstrapBrokers(request); } catch (KafkaException | StsException e) { - LOG.info("Failed to get bootstrap server information from MSK. Will try every 10 seconds for {} seconds", 10*MAX_KAFKA_CLIENT_RETRIES, 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) {} @@ -302,16 +311,19 @@ public static void setDynamicSaslClientCallbackHandler(final Properties properti } } } - public static void setAuthProperties(final Properties properties, final KafkaClusterAuthConfig kafkaClusterAuthConfig, final Logger LOG) { + public static void setAuthProperties(final Properties properties, final KafkaClusterAuthConfig kafkaClusterAuthConfig, final Logger log) { final AwsConfig awsConfig = kafkaClusterAuthConfig.getAwsConfig(); final AuthConfig authConfig = kafkaClusterAuthConfig.getAuthConfig(); final EncryptionConfig encryptionConfig = kafkaClusterAuthConfig.getEncryptionConfig(); - credentialsProvider = DefaultCredentialsProvider.create(); + configureMSKCredentialsProvider(authConfig, awsConfig); String bootstrapServers = ""; if (Objects.nonNull(kafkaClusterAuthConfig.getBootstrapServers())) { bootstrapServers = String.join(",", kafkaClusterAuthConfig.getBootstrapServers()); } + if (Objects.nonNull(awsConfig) && Objects.nonNull(awsConfig.getAwsMskConfig())) { + bootstrapServers = getBootStrapServersForMsk(awsConfig, mskCredentialsProvider, log); + } if (Objects.nonNull(authConfig)) { final AuthConfig.SaslAuthConfig saslAuthConfig = authConfig.getSaslAuthConfig(); @@ -323,11 +335,7 @@ public static void setAuthProperties(final Properties properties, final KafkaClu if (checkEncryptionType(encryptionConfig, EncryptionType.NONE)) { throw new RuntimeException("Encryption Config must be SSL to use IAM authentication mechanism"); } - if (Objects.isNull(awsConfig)) { - throw new RuntimeException("AWS Config is not specified"); - } setAwsIamAuthProperties(properties, awsIamAuthConfig, awsConfig); - bootstrapServers = getBootStrapServersForMsk(awsIamAuthConfig, awsConfig, LOG); } else if (Objects.nonNull(saslAuthConfig.getOAuthConfig())) { setOauthProperties(kafkaClusterAuthConfig, properties); } else if (Objects.nonNull(plainTextAuthConfig) && Objects.nonNull(kafkaClusterAuthConfig.getEncryptionConfig())) { @@ -358,19 +366,45 @@ private static boolean checkEncryptionType(final EncryptionConfig encryptionConf } public static GlueSchemaRegistryKafkaDeserializer getGlueSerializer(final KafkaConsumerConfig kafkaConsumerConfig) { + configureAwsGlueCredentialsProvider(kafkaConsumerConfig.getAwsConfig()); SchemaConfig schemaConfig = kafkaConsumerConfig.getSchemaConfig(); if (Objects.isNull(schemaConfig) || schemaConfig.getType() != SchemaRegistryType.AWS_GLUE) { return null; } Map configs = new HashMap<>(); - configs.put(AWSSchemaRegistryConstants.AWS_REGION, kafkaConsumerConfig.getAwsConfig().getRegion()); + final AwsConfig awsConfig = kafkaConsumerConfig.getAwsConfig(); + if (Objects.nonNull(awsConfig) && Objects.nonNull(awsConfig.getRegion())) { + configs.put(AWSSchemaRegistryConstants.AWS_REGION, kafkaConsumerConfig.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); + glueDeserializer = new GlueSchemaRegistryKafkaDeserializer(awsGlueCredentialsProvider, configs); return glueDeserializer; } + private static void configureAwsGlueCredentialsProvider(final AwsConfig awsConfig) { + awsGlueCredentialsProvider = DefaultCredentialsProvider.create(); + if (Objects.nonNull(awsConfig) && + Objects.nonNull(awsConfig.getRegion()) && Objects.nonNull(awsConfig.getStsRoleArn())) { + String sessionName = "data-prepper-kafka-session" + UUID.randomUUID(); + StsClient stsClient = StsClient.builder() + .region(Region.of(awsConfig.getRegion())) + .credentialsProvider(awsGlueCredentialsProvider) + .build(); + awsGlueCredentialsProvider = StsAssumeRoleCredentialsProvider + .builder() + .stsClient(stsClient) + .refreshRequest( + AssumeRoleRequest + .builder() + .roleArn(awsConfig.getStsRoleArn()) + .roleSessionName(sessionName) + .build() + ).build(); + } + } + } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurerTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurerTest.java index f1a9af8436..298457e21e 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurerTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSecurityConfigurerTest.java @@ -1,8 +1,11 @@ package org.opensearch.dataprepper.plugins.kafka.util; +import com.amazonaws.services.schemaregistry.deserializers.GlueSchemaRegistryKafkaDeserializer; import com.fasterxml.jackson.databind.ObjectMapper; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.ArgumentCaptor; import org.mockito.Captor; import org.mockito.Mock; @@ -19,6 +22,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.yaml.snakeyaml.Yaml; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.regions.providers.DefaultAwsRegionProviderChain; +import software.amazon.awssdk.services.kafka.KafkaClient; +import software.amazon.awssdk.services.kafka.KafkaClientBuilder; +import software.amazon.awssdk.services.kafka.model.GetBootstrapBrokersRequest; +import software.amazon.awssdk.services.kafka.model.GetBootstrapBrokersResponse; +import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; import java.io.FileReader; import java.io.IOException; @@ -27,12 +38,16 @@ import java.util.Map; import java.util.Objects; import java.util.Properties; +import java.util.UUID; import static org.apache.kafka.common.config.SaslConfigs.SASL_CLIENT_CALLBACK_HANDLER_CLASS; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.CoreMatchers.is; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.verify; @@ -128,6 +143,136 @@ public void testSetAuthPropertiesAuthSslWithNoCertContentNoTrustStore() throws E assertThat(props.get("ssl.engine.factory.class"), is(nullValue())); } + @Test + public void testSetAuthPropertiesBootstrapServersWithSaslIAMRole() throws IOException { + final Properties props = new Properties(); + final KafkaSourceConfig kafkaSourceConfig = createKafkaSinkConfig("kafka-pipeline-bootstrap-servers-sasl-iam-role.yaml"); + KafkaSecurityConfigurer.setAuthProperties(props, kafkaSourceConfig, LOG); + assertThat(props.getProperty("bootstrap.servers"), is("localhost:9092")); + assertThat(props.getProperty("sasl.mechanism"), is("AWS_MSK_IAM")); + assertThat(props.getProperty("sasl.jaas.config"), + is("software.amazon.msk.auth.iam.IAMLoginModule required " + + "awsRoleArn=\"test_sasl_iam_sts_role\" awsStsRegion=\"us-east-2\";")); + assertThat(props.getProperty("security.protocol"), is("SASL_SSL")); + assertThat(props.getProperty("certificateContent"), is(nullValue())); + assertThat(props.getProperty("ssl.truststore.location"), is(nullValue())); + assertThat(props.getProperty("ssl.truststore.password"), is(nullValue())); + assertThat(props.get("ssl.engine.factory.class"), is(nullValue())); + assertThat(props.get("sasl.client.callback.handler.class"), + is("software.amazon.msk.auth.iam.IAMClientCallbackHandler")); + } + + @Test + public void testSetAuthPropertiesBootstrapServersWithSaslIAMDefault() throws IOException { + final Properties props = new Properties(); + final KafkaSourceConfig kafkaSourceConfig = createKafkaSinkConfig("kafka-pipeline-bootstrap-servers-sasl-iam-default.yaml"); + KafkaSecurityConfigurer.setAuthProperties(props, kafkaSourceConfig, LOG); + assertThat(props.getProperty("bootstrap.servers"), is("localhost:9092")); + assertThat(props.getProperty("sasl.jaas.config"), is("software.amazon.msk.auth.iam.IAMLoginModule required;")); + assertThat(props.getProperty("sasl.mechanism"), is("AWS_MSK_IAM")); + assertThat(props.getProperty("security.protocol"), is("SASL_SSL")); + assertThat(props.getProperty("certificateContent"), is(nullValue())); + assertThat(props.getProperty("ssl.truststore.location"), is(nullValue())); + assertThat(props.getProperty("ssl.truststore.password"), is(nullValue())); + assertThat(props.get("ssl.engine.factory.class"), is(nullValue())); + assertThat(props.get("sasl.client.callback.handler.class"), + is("software.amazon.msk.auth.iam.IAMClientCallbackHandler")); + } + + @Test + public void testSetAuthPropertiesBootstrapServersOverrideByMSK() throws IOException { + final String testMSKEndpoint = UUID.randomUUID().toString(); + final Properties props = new Properties(); + final KafkaSourceConfig kafkaSourceConfig = createKafkaSinkConfig("kafka-pipeline-bootstrap-servers-override-by-msk.yaml"); + final KafkaClientBuilder kafkaClientBuilder = mock(KafkaClientBuilder.class); + final KafkaClient kafkaClient = mock(KafkaClient.class); + when(kafkaClientBuilder.credentialsProvider(any())).thenReturn(kafkaClientBuilder); + when(kafkaClientBuilder.region(any(Region.class))).thenReturn(kafkaClientBuilder); + when(kafkaClientBuilder.build()).thenReturn(kafkaClient); + final GetBootstrapBrokersResponse response = mock(GetBootstrapBrokersResponse.class); + when(response.bootstrapBrokerStringSaslIam()).thenReturn(testMSKEndpoint); + when(kafkaClient.getBootstrapBrokers(any(GetBootstrapBrokersRequest.class))).thenReturn(response); + try (MockedStatic mockedKafkaClient = mockStatic(KafkaClient.class)) { + mockedKafkaClient.when(KafkaClient::builder).thenReturn(kafkaClientBuilder); + KafkaSecurityConfigurer.setAuthProperties(props, kafkaSourceConfig, LOG); + } + assertThat(props.getProperty("bootstrap.servers"), is(testMSKEndpoint)); + assertThat(props.getProperty("sasl.mechanism"), is("AWS_MSK_IAM")); + assertThat(props.getProperty("sasl.jaas.config"), + is("software.amazon.msk.auth.iam.IAMLoginModule required awsRoleArn=\"sts_role_arn\" awsStsRegion=\"us-east-2\";")); + assertThat(props.getProperty("security.protocol"), is("SASL_SSL")); + assertThat(props.getProperty("certificateContent"), is(nullValue())); + assertThat(props.getProperty("ssl.truststore.location"), is(nullValue())); + assertThat(props.getProperty("ssl.truststore.password"), is(nullValue())); + assertThat(props.get("ssl.engine.factory.class"), is(nullValue())); + assertThat(props.get("sasl.client.callback.handler.class"), + is("software.amazon.msk.auth.iam.IAMClientCallbackHandler")); + } + + @Test + public void testSetAuthPropertiesMskWithSaslPlain() throws IOException { + final String testMSKEndpoint = UUID.randomUUID().toString(); + final Properties props = new Properties(); + final KafkaSourceConfig kafkaSourceConfig = createKafkaSinkConfig("kafka-pipeline-msk-sasl-plain.yaml"); + final KafkaClientBuilder kafkaClientBuilder = mock(KafkaClientBuilder.class); + final KafkaClient kafkaClient = mock(KafkaClient.class); + when(kafkaClientBuilder.credentialsProvider(any())).thenReturn(kafkaClientBuilder); + when(kafkaClientBuilder.region(any(Region.class))).thenReturn(kafkaClientBuilder); + when(kafkaClientBuilder.build()).thenReturn(kafkaClient); + final GetBootstrapBrokersResponse response = mock(GetBootstrapBrokersResponse.class); + when(response.bootstrapBrokerStringSaslIam()).thenReturn(testMSKEndpoint); + when(kafkaClient.getBootstrapBrokers(any(GetBootstrapBrokersRequest.class))).thenReturn(response); + try (MockedStatic mockedKafkaClient = mockStatic(KafkaClient.class)) { + mockedKafkaClient.when(KafkaClient::builder).thenReturn(kafkaClientBuilder); + KafkaSecurityConfigurer.setAuthProperties(props, kafkaSourceConfig, LOG); + } + assertThat(props.getProperty("bootstrap.servers"), is(testMSKEndpoint)); + assertThat(props.getProperty("sasl.mechanism"), is("PLAIN")); + assertThat(props.getProperty("sasl.jaas.config"), + is("org.apache.kafka.common.security.plain.PlainLoginModule required " + + "username=\"test_sasl_username\" password=\"test_sasl_password\";")); + assertThat(props.getProperty("security.protocol"), is("SASL_SSL")); + assertThat(props.getProperty("certificateContent"), is(nullValue())); + assertThat(props.getProperty("ssl.truststore.location"), is(nullValue())); + assertThat(props.getProperty("ssl.truststore.password"), is(nullValue())); + assertThat(props.get("ssl.engine.factory.class"), is(nullValue())); + } + + @ParameterizedTest + @ValueSource(strings = { + "kafka-pipeline-bootstrap-servers-glue-sts-assume-role.yaml", + "kafka-pipeline-msk-default-glue-sts-assume-role.yaml" + }) + void testGetGlueSerializerWithStsAssumeRoleCredentialsProvider(final String filename) throws IOException { + final KafkaSourceConfig kafkaSourceConfig = createKafkaSinkConfig(filename); + final GlueSchemaRegistryKafkaDeserializer glueSchemaRegistryKafkaDeserializer = KafkaSecurityConfigurer + .getGlueSerializer(kafkaSourceConfig); + assertThat(glueSchemaRegistryKafkaDeserializer, notNullValue()); + assertThat(glueSchemaRegistryKafkaDeserializer.getCredentialProvider(), + instanceOf(StsAssumeRoleCredentialsProvider.class)); + } + + @Test + void testGetGlueSerializerWithDefaultCredentialsProvider() throws IOException { + final KafkaSourceConfig kafkaSourceConfig = createKafkaSinkConfig( + "kafka-pipeline-bootstrap-servers-glue-default.yaml"); + final DefaultAwsRegionProviderChain.Builder defaultAwsRegionProviderChainBuilder = mock( + DefaultAwsRegionProviderChain.Builder.class); + final DefaultAwsRegionProviderChain defaultAwsRegionProviderChain = mock(DefaultAwsRegionProviderChain.class); + when(defaultAwsRegionProviderChainBuilder.build()).thenReturn(defaultAwsRegionProviderChain); + when(defaultAwsRegionProviderChain.getRegion()).thenReturn(Region.US_EAST_1); + try (MockedStatic defaultAwsRegionProviderChainMockedStatic = + mockStatic(DefaultAwsRegionProviderChain.class)) { + defaultAwsRegionProviderChainMockedStatic.when(DefaultAwsRegionProviderChain::builder) + .thenReturn(defaultAwsRegionProviderChainBuilder); + final GlueSchemaRegistryKafkaDeserializer glueSchemaRegistryKafkaDeserializer = KafkaSecurityConfigurer + .getGlueSerializer(kafkaSourceConfig); + assertThat(glueSchemaRegistryKafkaDeserializer, notNullValue()); + assertThat(glueSchemaRegistryKafkaDeserializer.getCredentialProvider(), + instanceOf(DefaultCredentialsProvider.class)); + } + } + @Test void testSetDynamicSaslClientCallbackHandlerWithNonNullPlainTextAuthConfig() { when(kafkaConnectionConfig.getAuthConfig()).thenReturn(authConfig); diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-glue-default.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-glue-default.yaml new file mode 100644 index 0000000000..5017333415 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-glue-default.yaml @@ -0,0 +1,14 @@ +log-pipeline : + source: + kafka: + bootstrap_servers: + - "localhost:9092" + encryption: + type: "SSL" + schema: + type: aws_glue + topics: + - name: "quickstart-events" + group_id: "groupdID1" + sink: + stdout: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-glue-sts-assume-role.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-glue-sts-assume-role.yaml new file mode 100644 index 0000000000..4fc036a9de --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-glue-sts-assume-role.yaml @@ -0,0 +1,17 @@ +log-pipeline : + source: + kafka: + bootstrap_servers: + - "localhost:9092" + encryption: + type: "SSL" + aws: + region: us-east-2 + sts_role_arn: sts_role_arn + schema: + type: aws_glue + topics: + - name: "quickstart-events" + group_id: "groupdID1" + sink: + stdout: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-override-by-msk.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-override-by-msk.yaml new file mode 100644 index 0000000000..889fd0c044 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-override-by-msk.yaml @@ -0,0 +1,20 @@ +log-pipeline : + source: + kafka: + bootstrap_servers: + - "localhost:9092" + encryption: + type: "SSL" + authentication: + sasl: + aws_msk_iam: role + aws: + region: us-east-2 + sts_role_arn: sts_role_arn + msk: + arn: service Arn + topics: + - name: "quickstart-events" + group_id: "groupdID1" + sink: + stdout: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-sasl-iam-default.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-sasl-iam-default.yaml new file mode 100644 index 0000000000..0edc808ce3 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-sasl-iam-default.yaml @@ -0,0 +1,15 @@ +log-pipeline : + source: + kafka: + bootstrap_servers: + - "localhost:9092" + encryption: + type: "SSL" + authentication: + sasl: + aws_msk_iam: default + topics: + - name: "quickstart-events" + group_id: "groupdID1" + sink: + stdout: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-sasl-iam-role.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-sasl-iam-role.yaml new file mode 100644 index 0000000000..a4ef7fd94b --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-bootstrap-servers-sasl-iam-role.yaml @@ -0,0 +1,18 @@ +log-pipeline : + source: + kafka: + bootstrap_servers: + - "localhost:9092" + encryption: + type: "SSL" + authentication: + sasl: + aws_msk_iam: role + aws: + region: us-east-2 + sts_role_arn: test_sasl_iam_sts_role + topics: + - name: "quickstart-events" + group_id: "groupdID1" + sink: + stdout: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-msk-default-glue-sts-assume-role.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-msk-default-glue-sts-assume-role.yaml new file mode 100644 index 0000000000..bf94287f26 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-msk-default-glue-sts-assume-role.yaml @@ -0,0 +1,20 @@ +log-pipeline : + source: + kafka: + encryption: + type: "SSL" + authentication: + sasl: + aws_msk_iam: default + aws: + region: us-east-2 + sts_role_arn: sts_role_arn + msk: + arn: service Arn + schema: + type: aws_glue + topics: + - name: "quickstart-events" + group_id: "groupdID1" + sink: + stdout: \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-msk-sasl-plain.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-msk-sasl-plain.yaml new file mode 100644 index 0000000000..f1a44ff414 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/kafka-pipeline-msk-sasl-plain.yaml @@ -0,0 +1,20 @@ +log-pipeline : + source: + kafka: + encryption: + type: "SSL" + authentication: + sasl: + plain: + username: test_sasl_username + password: test_sasl_password + aws: + region: us-east-2 + sts_role_arn: sts_role_arn + msk: + arn: service Arn + topics: + - name: "quickstart-events" + group_id: "groupdID1" + sink: + stdout: \ No newline at end of file From bc51e67400bbd56af956c549034a12db8576673c Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 13 Jun 2024 10:22:15 -0500 Subject: [PATCH 013/159] Caffeine-based caching parser for the user_agent processor (#4619) Adds and uses a Caffeine-based caching parser for the user_agent processor. Resolves #4618 Signed-off-by: David Venable --- data-prepper-plugins/opensearch/build.gradle | 2 +- .../otel-trace-raw-processor/build.gradle | 2 +- .../user-agent-processor/build.gradle | 3 +- .../useragent/CaffeineCachingParser.java | 75 +++++++++ .../useragent/UserAgentProcessor.java | 3 +- .../useragent/CaffeineCachingParserTest.java | 152 ++++++++++++++++++ settings.gradle | 1 + 7 files changed, 233 insertions(+), 5 deletions(-) create mode 100644 data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/CaffeineCachingParser.java create mode 100644 data-prepper-plugins/user-agent-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/useragent/CaffeineCachingParserTest.java diff --git a/data-prepper-plugins/opensearch/build.gradle b/data-prepper-plugins/opensearch/build.gradle index 1d5be32d00..1f96c1f6ea 100644 --- a/data-prepper-plugins/opensearch/build.gradle +++ b/data-prepper-plugins/opensearch/build.gradle @@ -32,7 +32,7 @@ dependencies { implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:opensearchserverless' implementation libs.commons.lang3 - implementation 'com.github.ben-manes.caffeine:caffeine:3.1.8' + implementation libs.caffeine implementation 'software.amazon.awssdk:apache-client' implementation 'software.amazon.awssdk:netty-nio-client' implementation 'co.elastic.clients:elasticsearch-java:7.17.0' diff --git a/data-prepper-plugins/otel-trace-raw-processor/build.gradle b/data-prepper-plugins/otel-trace-raw-processor/build.gradle index 6d9994abbb..ff2bfc4a60 100644 --- a/data-prepper-plugins/otel-trace-raw-processor/build.gradle +++ b/data-prepper-plugins/otel-trace-raw-processor/build.gradle @@ -18,7 +18,7 @@ dependencies { implementation libs.armeria.grpc implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' - implementation 'com.github.ben-manes.caffeine:caffeine:3.1.8' + implementation libs.caffeine testImplementation 'org.assertj:assertj-core:3.25.3' testImplementation testLibs.mockito.inline } diff --git a/data-prepper-plugins/user-agent-processor/build.gradle b/data-prepper-plugins/user-agent-processor/build.gradle index 6ad33c84ba..746ee40397 100644 --- a/data-prepper-plugins/user-agent-processor/build.gradle +++ b/data-prepper-plugins/user-agent-processor/build.gradle @@ -11,7 +11,8 @@ dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:common') implementation 'com.fasterxml.jackson.core:jackson-databind' - implementation "com.github.ua-parser:uap-java:1.6.1" + implementation 'com.github.ua-parser:uap-java:1.6.1' + implementation libs.caffeine } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/CaffeineCachingParser.java b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/CaffeineCachingParser.java new file mode 100644 index 0000000000..45a96bd909 --- /dev/null +++ b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/CaffeineCachingParser.java @@ -0,0 +1,75 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.useragent; + +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import ua_parser.Client; +import ua_parser.Device; +import ua_parser.OS; +import ua_parser.Parser; +import ua_parser.UserAgent; + +import java.util.function.Function; + +/** + * A superclass of {@link Parser} which uses Caffeine as a cache. + */ +class CaffeineCachingParser extends Parser { + private final Cache clientCache; + private final Cache userAgentCache; + private final Cache deviceCache; + private final Cache osCache; + + /** + * Constructs a new instance with a given cache size. Each parse method + * will have its own cache. + * + * @param cacheSize The size of the cache as a count of items. + */ + CaffeineCachingParser(final long cacheSize) { + userAgentCache = createCache(cacheSize); + clientCache = createCache(cacheSize); + deviceCache = createCache(cacheSize); + osCache = createCache(cacheSize); + } + + @Override + public Client parse(final String agentString) { + return parseCaching(agentString, clientCache, super::parse); + } + + @Override + public UserAgent parseUserAgent(final String agentString) { + return parseCaching(agentString, userAgentCache, super::parseUserAgent); + } + + @Override + public Device parseDevice(final String agentString) { + return parseCaching(agentString, deviceCache, super::parseDevice); + } + + @Override + public OS parseOS(final String agentString) { + return parseCaching(agentString, osCache, super::parseOS); + } + + private T parseCaching( + final String agentString, + final Cache cache, + final Function parseFunction) { + if (agentString == null) { + return null; + } + return cache.get(agentString, parseFunction); + } + + private static Cache createCache(final long maximumSize) { + return Caffeine.newBuilder() + .maximumSize(maximumSize) + .build(); + } +} diff --git a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessor.java b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessor.java index 220bb88287..32779655dc 100644 --- a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessor.java +++ b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessor.java @@ -14,7 +14,6 @@ import org.opensearch.dataprepper.model.record.Record; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import ua_parser.CachingParser; import ua_parser.Client; import ua_parser.Parser; @@ -36,7 +35,7 @@ public class UserAgentProcessor extends AbstractProcessor, Record< public UserAgentProcessor(final PluginMetrics pluginMetrics, final UserAgentProcessorConfig config) { super(pluginMetrics); this.config = config; - this.userAgentParser = new CachingParser(config.getCacheSize()); + this.userAgentParser = new CaffeineCachingParser(config.getCacheSize()); } @Override diff --git a/data-prepper-plugins/user-agent-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/useragent/CaffeineCachingParserTest.java b/data-prepper-plugins/user-agent-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/useragent/CaffeineCachingParserTest.java new file mode 100644 index 0000000000..14c72fa354 --- /dev/null +++ b/data-prepper-plugins/user-agent-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/useragent/CaffeineCachingParserTest.java @@ -0,0 +1,152 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.useragent; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import ua_parser.Client; +import ua_parser.Device; +import ua_parser.OS; +import ua_parser.UserAgent; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; + +@SuppressWarnings("StringOperationCanBeSimplified") +class CaffeineCachingParserTest { + private static final String KNOWN_USER_AGENT_STRING = "Mozilla/5.0 (iPhone; CPU iPhone OS 13_5_1 like Mac OS X) AppleWebKit/605.1.15 (KHTML, like Gecko) Version/13.1.1 Mobile/15E148 Safari/604.1"; + long cacheSize; + + @BeforeEach + void setUp() { + cacheSize = 1000; + } + + private CaffeineCachingParser createObjectUnderTest() { + return new CaffeineCachingParser(cacheSize); + } + + @Test + void parse_returns_expected_results() { + final Client client = createObjectUnderTest().parse(KNOWN_USER_AGENT_STRING); + + assertThat(client, notNullValue()); + assertThat(client.userAgent, notNullValue()); + assertThat(client.userAgent.family, equalTo("Mobile Safari")); + assertThat(client.userAgent.major, equalTo("13")); + assertThat(client.device.family, equalTo("iPhone")); + assertThat(client.os.family, equalTo("iOS")); + } + + @Test + void parse_with_null_returns_null() { + assertThat(createObjectUnderTest().parse(null), + nullValue()); + } + + @Test + void parse_called_multiple_times_returns_same_instance() { + final CaffeineCachingParser objectUnderTest = createObjectUnderTest(); + + final String userAgentString = KNOWN_USER_AGENT_STRING; + final Client client = objectUnderTest.parse(userAgentString); + + assertThat(client, notNullValue()); + + assertThat(objectUnderTest.parse(new String(userAgentString)), sameInstance(client)); + assertThat(objectUnderTest.parse(new String(userAgentString)), sameInstance(client)); + assertThat(objectUnderTest.parse(new String(userAgentString)), sameInstance(client)); + } + + @Test + void parseUserAgent_returns_expected_results() { + final UserAgent userAgent = createObjectUnderTest().parseUserAgent(KNOWN_USER_AGENT_STRING); + + assertThat(userAgent, notNullValue()); + assertThat(userAgent.family, equalTo("Mobile Safari")); + assertThat(userAgent.major, equalTo("13")); + } + + @Test + void parseUserAgent_with_null_returns_null() { + assertThat(createObjectUnderTest().parseUserAgent(null), + nullValue()); + } + + @Test + void parseUserAgent_called_multiple_times_returns_same_instance() { + final CaffeineCachingParser objectUnderTest = createObjectUnderTest(); + + final String userAgentString = KNOWN_USER_AGENT_STRING; + final UserAgent userAgent = objectUnderTest.parseUserAgent(userAgentString); + + assertThat(userAgent, notNullValue()); + + assertThat(objectUnderTest.parseUserAgent(new String(userAgentString)), sameInstance(userAgent)); + assertThat(objectUnderTest.parseUserAgent(new String(userAgentString)), sameInstance(userAgent)); + assertThat(objectUnderTest.parseUserAgent(new String(userAgentString)), sameInstance(userAgent)); + } + + @Test + void parseDevice_returns_expected_results() { + final Device device = createObjectUnderTest().parseDevice(KNOWN_USER_AGENT_STRING); + + assertThat(device, notNullValue()); + assertThat(device.family, equalTo("iPhone")); + } + + @Test + void parseDevice_with_null_returns_null() { + assertThat(createObjectUnderTest().parseDevice(null), + nullValue()); + } + + @Test + void parseDevice_called_multiple_times_returns_same_instance() { + final CaffeineCachingParser objectUnderTest = createObjectUnderTest(); + + final String userAgentString = KNOWN_USER_AGENT_STRING; + final Device device = objectUnderTest.parseDevice(userAgentString); + + assertThat(device, notNullValue()); + + assertThat(objectUnderTest.parseDevice(new String(userAgentString)), sameInstance(device)); + assertThat(objectUnderTest.parseDevice(new String(userAgentString)), sameInstance(device)); + assertThat(objectUnderTest.parseDevice(new String(userAgentString)), sameInstance(device)); + } + + @Test + void parseOS_returns_expected_results() { + final OS os = createObjectUnderTest().parseOS(KNOWN_USER_AGENT_STRING); + + assertThat(os, notNullValue()); + assertThat(os.family, equalTo("iOS")); + assertThat(os.major, equalTo("13")); + } + + @Test + void parseOS_with_null_returns_null() { + assertThat(createObjectUnderTest().parseOS(null), + nullValue()); + } + + @Test + void parseOS_called_multiple_times_returns_same_instance() { + final CaffeineCachingParser objectUnderTest = createObjectUnderTest(); + + final String userAgentString = KNOWN_USER_AGENT_STRING; + final OS os = objectUnderTest.parseOS(userAgentString); + + assertThat(os, notNullValue()); + + assertThat(objectUnderTest.parseOS(new String(userAgentString)), sameInstance(os)); + assertThat(objectUnderTest.parseOS(new String(userAgentString)), sameInstance(os)); + assertThat(objectUnderTest.parseOS(new String(userAgentString)), sameInstance(os)); + } +} \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index 0a7718aa4d..c72986922e 100644 --- a/settings.gradle +++ b/settings.gradle @@ -65,6 +65,7 @@ dependencyResolutionManagement { library('hadoop-mapreduce', 'org.apache.hadoop', 'hadoop-mapreduce-client-core').versionRef('hadoop') version('avro', '1.11.3') library('avro-core', 'org.apache.avro', 'avro').versionRef('avro') + library('caffeine', 'com.github.ben-manes.caffeine', 'caffeine').version('3.1.8') } testLibs { version('junit', '5.8.2') From f9c5399189c74444e4c03f21b6e61534fc45a120 Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Fri, 14 Jun 2024 02:36:00 -0400 Subject: [PATCH 014/159] OpenSearch Sink add support for sending pipeline parameter in BulkRequest (#4609) * Add support in OpenSearch sink to pass pipeline parameter in the index and create action operation for bulk api requests Signed-off-by: Souvik Bose * Add more unit tests to increase code coverage Signed-off-by: Souvik Bose * Update README Signed-off-by: Souvik Bose * Fix the OpenSearch Integration tests Signed-off-by: Souvik Bose --------- Signed-off-by: Souvik Bose --- data-prepper-plugins/opensearch/README.md | 2 ++ .../sink/opensearch/OpenSearchSinkIT.java | 6 ++-- .../sink/opensearch/OpenSearchSink.java | 20 ++++++++++- .../sink/opensearch/bulk/SerializedJson.java | 5 +-- .../opensearch/bulk/SerializedJsonImpl.java | 7 +++- .../opensearch/bulk/SerializedJsonNode.java | 5 +++ .../opensearch/index/IndexConfiguration.java | 18 ++++++++++ .../opensearch/BulkRetryStrategyTests.java | 2 +- .../sink/opensearch/OpenSearchSinkTest.java | 17 +++++++++ .../bulk/Es6BulkApiWrapperTest.java | 21 +++++++++++ .../bulk/SerializedJsonImplTest.java | 9 +++-- .../bulk/SerializedJsonNodeTest.java | 5 ++- .../opensearch/bulk/SerializedJsonTest.java | 12 ++++--- .../dlq/FailedBulkOperationConverterTest.java | 2 +- .../index/IndexConfigurationTests.java | 36 +++++++++++++++++++ 15 files changed, 150 insertions(+), 17 deletions(-) diff --git a/data-prepper-plugins/opensearch/README.md b/data-prepper-plugins/opensearch/README.md index f6f7b1a3ea..628a75cc80 100644 --- a/data-prepper-plugins/opensearch/README.md +++ b/data-prepper-plugins/opensearch/README.md @@ -238,6 +238,8 @@ the flush timeout and instead flush whatever is present at the end of each batch bulk_size: 4 ``` +- `pipeline` (optional): A string which is used to represent the pipeline Id for preprocessing documents. Each incoming record is searched for this field and if it is present, it is used as the pipeline field for the document. Standard Data Prepper Json pointer syntax is used for retrieving the value. If the field has "/" in it then the incoming record is searched in the json sub-objects instead of just in the root of the json object. For example, if the field is specified as `info/id`, then the root of the event is searched for `info` and if it is found, then `id` is searched inside it. The value specified for `id` is used as the pipeline id. This field can also be a Data Prepper expression that is evaluated to determine the `pipeline_id`. For example, setting to `getMetadata(\"some_metadata_key\")` will use the value of the metadata key as the pipeline_id. + - `ism_policy_file` (optional): A String of absolute file path or AWS S3 URI for an ISM (Index State Management) policy JSON file. This policy file is effective only when there is no built-in policy file for the index type. For example, `custom` index type is currently the only one without a built-in policy file, thus it would use the policy file here if it's provided through this parameter. OpenSearch documentation has more about [ISM policies.](https://opensearch.org/docs/latest/im-plugin/ism/policies/) - `s3_aws_region` (optional): A String represents the region of S3 bucket to read `template_file` or `ism_policy_file`, e.g. us-west-2. Only applies to Amazon OpenSearch Service. Defaults to `us-east-1`. diff --git a/data-prepper-plugins/opensearch/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSinkIT.java b/data-prepper-plugins/opensearch/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSinkIT.java index 0e0bc87cb4..b17c0ea47c 100644 --- a/data-prepper-plugins/opensearch/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSinkIT.java +++ b/data-prepper-plugins/opensearch/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSinkIT.java @@ -303,7 +303,7 @@ public void testOutputRawSpanDefault(final boolean estimateBulkSizeUsingCompress .add(OpenSearchSink.BULKREQUEST_SIZE_BYTES).toString()); assertThat(bulkRequestSizeBytesMetrics.size(), equalTo(3)); assertThat(bulkRequestSizeBytesMetrics.get(0).getValue(), closeTo(1.0, 0)); - final double expectedBulkRequestSizeBytes = isRequestCompressionEnabled && estimateBulkSizeUsingCompression ? 773.0 : 2058.0; + final double expectedBulkRequestSizeBytes = isRequestCompressionEnabled && estimateBulkSizeUsingCompression ? 792.0 : 2058.0; assertThat(bulkRequestSizeBytesMetrics.get(1).getValue(), closeTo(expectedBulkRequestSizeBytes, 0)); assertThat(bulkRequestSizeBytesMetrics.get(2).getValue(), closeTo(expectedBulkRequestSizeBytes, 0)); } @@ -364,7 +364,7 @@ public void testOutputRawSpanWithDLQ(final boolean estimateBulkSizeUsingCompress .add(OpenSearchSink.BULKREQUEST_SIZE_BYTES).toString()); assertThat(bulkRequestSizeBytesMetrics.size(), equalTo(3)); assertThat(bulkRequestSizeBytesMetrics.get(0).getValue(), closeTo(1.0, 0)); - final double expectedBulkRequestSizeBytes = isRequestCompressionEnabled && estimateBulkSizeUsingCompression ? 1066.0 : 2072.0; + final double expectedBulkRequestSizeBytes = isRequestCompressionEnabled && estimateBulkSizeUsingCompression ? 1078.0 : 2072.0; assertThat(bulkRequestSizeBytesMetrics.get(1).getValue(), closeTo(expectedBulkRequestSizeBytes, 0)); assertThat(bulkRequestSizeBytesMetrics.get(2).getValue(), closeTo(expectedBulkRequestSizeBytes, 0)); @@ -426,7 +426,7 @@ public void testOutputServiceMapDefault(final boolean estimateBulkSizeUsingCompr .add(OpenSearchSink.BULKREQUEST_SIZE_BYTES).toString()); assertThat(bulkRequestSizeBytesMetrics.size(), equalTo(3)); assertThat(bulkRequestSizeBytesMetrics.get(0).getValue(), closeTo(1.0, 0)); - final double expectedBulkRequestSizeBytes = isRequestCompressionEnabled && estimateBulkSizeUsingCompression ? 366.0 : 265.0; + final double expectedBulkRequestSizeBytes = isRequestCompressionEnabled && estimateBulkSizeUsingCompression ? 376.0 : 265.0; assertThat(bulkRequestSizeBytesMetrics.get(1).getValue(), closeTo(expectedBulkRequestSizeBytes, 0)); assertThat(bulkRequestSizeBytesMetrics.get(2).getValue(), closeTo(expectedBulkRequestSizeBytes, 0)); 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 e1547a925c..199b4e1e0e 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 @@ -115,6 +115,7 @@ public class OpenSearchSink extends AbstractSink> { private final String documentId; private final String routingField; private final String routing; + private final String pipeline; private final String action; private final List> actions; private final String documentRootKey; @@ -170,6 +171,7 @@ public OpenSearchSink(final PluginSetting pluginSetting, this.documentId = openSearchSinkConfig.getIndexConfiguration().getDocumentId(); this.routingField = openSearchSinkConfig.getIndexConfiguration().getRoutingField(); this.routing = openSearchSinkConfig.getIndexConfiguration().getRouting(); + this.pipeline = openSearchSinkConfig.getIndexConfiguration().getPipeline(); this.action = openSearchSinkConfig.getIndexConfiguration().getAction(); this.actions = openSearchSinkConfig.getIndexConfiguration().getActions(); this.documentRootKey = openSearchSinkConfig.getIndexConfiguration().getDocumentRootKey(); @@ -299,6 +301,7 @@ private BulkOperation getBulkOperationForAction(final String action, BulkOperation bulkOperation; final Optional docId = document.getDocumentId(); final Optional routing = document.getRoutingField(); + final Optional pipeline = document.getPipelineField(); if (StringUtils.equals(action, OpenSearchBulkActions.CREATE.toString())) { final CreateOperation.Builder createOperationBuilder = @@ -307,6 +310,8 @@ private BulkOperation getBulkOperationForAction(final String action, .document(document); docId.ifPresent(createOperationBuilder::id); routing.ifPresent(createOperationBuilder::routing); + pipeline.ifPresent(createOperationBuilder::pipeline); + bulkOperation = new BulkOperation.Builder() .create(createOperationBuilder.build()) .build(); @@ -367,6 +372,7 @@ private BulkOperation getBulkOperationForAction(final String action, .versionType(versionType); docId.ifPresent(indexOperationBuilder::id); routing.ifPresent(indexOperationBuilder::routing); + pipeline.ifPresent(indexOperationBuilder::pipeline); bulkOperation = new BulkOperation.Builder() .index(indexOperationBuilder.build()) .build(); @@ -502,9 +508,21 @@ SerializedJson getDocument(final Event event) { } } + String pipelineValue = null; + if (pipeline != null) { + try { + pipelineValue = event.formatString(pipeline, expressionEvaluator); + } catch (final ExpressionEvaluationException | EventKeyNotFoundException e) { + LOG.error("Unable to construct pipeline with format {}", pipeline, e); + } + if (StringUtils.isEmpty(pipelineValue) || StringUtils.isBlank(pipelineValue)) { + pipelineValue = null; + } + } + final String document = DocumentBuilder.build(event, documentRootKey, sinkContext.getTagsTargetKey(), sinkContext.getIncludeKeys(), sinkContext.getExcludeKeys()); - return SerializedJson.fromStringAndOptionals(document, docId, routingValue); + return SerializedJson.fromStringAndOptionals(document, docId, routingValue, pipelineValue); } private void flushBatch(AccumulatingBulkRequest accumulatingBulkRequest) { diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJson.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJson.java index 671a4d0423..d85a5992c5 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJson.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJson.java @@ -17,6 +17,7 @@ public interface SerializedJson extends SizedDocument { byte[] getSerializedJson(); Optional getDocumentId(); Optional getRoutingField(); + Optional getPipelineField(); /** * Creates a new {@link SerializedJson} from a JSON string and optional documentId and routingField. @@ -26,9 +27,9 @@ public interface SerializedJson extends SizedDocument { * @param routingField Optional routing field string * @return A new {@link SerializedJson}. */ - static SerializedJson fromStringAndOptionals(String jsonString, String docId, String routingField) { + static SerializedJson fromStringAndOptionals(String jsonString, String docId, String routingField, String pipelineField) { Objects.requireNonNull(jsonString); - return new SerializedJsonImpl(jsonString.getBytes(StandardCharsets.UTF_8), docId, routingField); + return new SerializedJsonImpl(jsonString.getBytes(StandardCharsets.UTF_8), docId, routingField, pipelineField); } static SerializedJson fromJsonNode(final JsonNode jsonNode, SerializedJson document) { diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonImpl.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonImpl.java index 06a26cba65..2f1fbd2b96 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonImpl.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonImpl.java @@ -12,11 +12,13 @@ class SerializedJsonImpl implements SerializedJson, Serializable { private byte[] document; private String documentId = null; private String routingField = null; + private String pipelineField = null; - public SerializedJsonImpl(final byte[] document, String docId, String routingField) { + public SerializedJsonImpl(final byte[] document, String docId, String routingField, String pipelineField) { this.document = document; this.documentId = docId; this.routingField = routingField; + this.pipelineField = pipelineField; } public SerializedJsonImpl(final byte[] document) { @@ -42,4 +44,7 @@ public Optional getDocumentId() { public Optional getRoutingField() { return Optional.ofNullable(routingField); } + + @Override + public Optional getPipelineField() { return Optional.ofNullable(pipelineField); } } diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonNode.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonNode.java index 7e56839d8c..41d9459347 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonNode.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonNode.java @@ -14,12 +14,14 @@ class SerializedJsonNode implements SerializedJson, Serializable { private JsonNode jsonNode; private String documentId = null; private String routingField = null; + private String pipelineField = null; public SerializedJsonNode(final JsonNode jsonNode, SerializedJson doc) { this.jsonNode = jsonNode; this.documentId = doc.getDocumentId().orElse(null); this.routingField = doc.getRoutingField().orElse(null); this.document = jsonNode.toString().getBytes(); + this.pipelineField = doc.getPipelineField().orElse(null);; } public SerializedJsonNode(final JsonNode jsonNode) { @@ -46,4 +48,7 @@ public Optional getDocumentId() { public Optional getRoutingField() { return Optional.ofNullable(routingField); } + + @Override + public Optional getPipelineField() { return Optional.ofNullable(pipelineField); } } 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 b3bbb213b2..21b178ea9c 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 @@ -57,6 +57,7 @@ public class IndexConfiguration { public static final String DOCUMENT_ID = "document_id"; public static final String ROUTING_FIELD = "routing_field"; public static final String ROUTING = "routing"; + public static final String PIPELINE = "pipeline"; public static final String ISM_POLICY_FILE = "ism_policy_file"; public static final long DEFAULT_BULK_SIZE = 5L; public static final boolean DEFAULT_ESTIMATE_BULK_SIZE_USING_COMPRESSION = false; @@ -81,6 +82,7 @@ public class IndexConfiguration { private final Map indexTemplate; private final String documentIdField; private final String documentId; + private final String pipeline; private final String routingField; private final String routing; private final long bulkSize; @@ -147,6 +149,7 @@ private IndexConfiguration(final Builder builder) { this.flushTimeout = builder.flushTimeout; this.routingField = builder.routingField; this.routing = builder.routing; + this.pipeline = builder.pipeline; String documentIdField = builder.documentIdField; String documentId = builder.documentId; @@ -266,6 +269,11 @@ public static IndexConfiguration readIndexConfig(final PluginSetting pluginSetti builder = builder.withRouting(routing); } + final String pipeline = pluginSetting.getStringOrDefault(PIPELINE, null); + if (pipeline != null) { + builder = builder.withPipeline(pipeline); + } + final String ismPolicyFile = pluginSetting.getStringOrDefault(ISM_POLICY_FILE, null); builder = builder.withIsmPolicyFile(ismPolicyFile); @@ -336,6 +344,10 @@ public String getRouting() { return routing; } + public String getPipeline() { + return pipeline; + } + public long getBulkSize() { return bulkSize; } @@ -459,6 +471,7 @@ public static class Builder { private int numReplicas; private String routingField; private String routing; + private String pipeline; private String documentIdField; private String documentId; private long bulkSize = DEFAULT_BULK_SIZE; @@ -534,6 +547,11 @@ public Builder withRouting(final String routing) { return this; } + public Builder withPipeline(final String pipeline) { + this.pipeline = pipeline; + return this; + } + public Builder withBulkSize(final long bulkSize) { this.bulkSize = bulkSize; return this; diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/BulkRetryStrategyTests.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/BulkRetryStrategyTests.java index 09b78a1de8..cc05514502 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/BulkRetryStrategyTests.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/BulkRetryStrategyTests.java @@ -717,7 +717,7 @@ private static BulkResponseItem customBulkFailureResponse(final RestStatus restS } private SerializedJson arbitraryDocument() { - return SerializedJson.fromStringAndOptionals("{}", null, null); + return SerializedJson.fromStringAndOptionals("{}", null, null, null); } private static class FakeClient { diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSinkTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSinkTest.java index 9b51954b62..31b77e0bf3 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSinkTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchSinkTest.java @@ -146,6 +146,7 @@ void setup() { when(indexConfiguration.getDocumentIdField()).thenReturn(null); when(indexConfiguration.getRoutingField()).thenReturn(null); when(indexConfiguration.getRouting()).thenReturn(null); + when(indexConfiguration.getPipeline()).thenReturn(null); when(indexConfiguration.getActions()).thenReturn(null); when(indexConfiguration.getDocumentRootKey()).thenReturn(null); lenient().when(indexConfiguration.getVersionType()).thenReturn(null); @@ -289,6 +290,22 @@ void test_routing_in_document() throws IOException { assertThat(objectUnderTest2.getDocument(event).getRoutingField(), equalTo(Optional.of(routingValue))); } + @Test + void test_pipeline_in_document() throws IOException { + String pipelineValue = UUID.randomUUID().toString(); + String pipelineKey = UUID.randomUUID().toString(); + final OpenSearchSink objectUnderTest = createObjectUnderTest(); + final Event event = JacksonEvent.builder() + .withEventType("event") + .withData(Collections.singletonMap(pipelineKey, pipelineValue)) + .build(); + assertThat(objectUnderTest.getDocument(event).getPipelineField(), equalTo(Optional.empty())); + + when(indexConfiguration.getPipeline()).thenReturn("${"+pipelineKey+"}"); + final OpenSearchSink objectUnderTest2 = createObjectUnderTest(); + assertThat(objectUnderTest2.getDocument(event).getPipelineField(), equalTo(Optional.of(pipelineValue))); + } + @Test void doOutput_with_invalid_version_expression_result_catches_RuntimeException_and_creates_DLQObject() throws IOException { diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/Es6BulkApiWrapperTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/Es6BulkApiWrapperTest.java index 558671f091..2ec15e7348 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/Es6BulkApiWrapperTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/Es6BulkApiWrapperTest.java @@ -26,6 +26,7 @@ import java.io.IOException; import java.util.List; +import java.util.UUID; import java.util.stream.Stream; import static org.hamcrest.CoreMatchers.equalTo; @@ -122,6 +123,26 @@ void testBulk_when_request_index_missing(final boolean isIndex, final boolean is assertThat(endpoint.requestUrl(bulkRequest), equalTo(expectedURI)); } + @Test + void testBulkWithAdditionParameters() throws IOException { + final String requestIndex = "test-index"; + final String expectedURI = String.format(ES6_URI_PATTERN, "test-index"); + when(openSearchClient._transport()).thenReturn(openSearchTransport); + when(openSearchClient._transportOptions()).thenReturn(transportOptions); + when(bulkRequest.index()).thenReturn(requestIndex); + final String pipeline = UUID.randomUUID().toString(); + when(bulkRequest.pipeline()).thenReturn(pipeline); + objectUnderTest.bulk(bulkRequest); + + ArgumentCaptor bulkRequestArgumentCaptor = ArgumentCaptor.forClass(BulkRequest.class); + + verify(openSearchTransport).performRequest( + bulkRequestArgumentCaptor.capture(), jsonEndpointArgumentCaptor.capture(), eq(transportOptions)); + final JsonEndpoint endpoint = jsonEndpointArgumentCaptor.getValue(); + assertThat(endpoint.requestUrl(bulkRequest), equalTo(expectedURI)); + assertThat(bulkRequestArgumentCaptor.getValue().pipeline(), equalTo(pipeline)); + } + private static Stream getTypeFlags() { return Stream.of( Arguments.of(true, false, false, false), diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonImplTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonImplTest.java index ff52a332c1..04fea232c8 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonImplTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonImplTest.java @@ -20,6 +20,7 @@ class SerializedJsonImplTest { private byte[] documentBytes; private String documentId; private String routingField; + private String pipelineField; @BeforeEach void setUp() { @@ -27,12 +28,13 @@ void setUp() { documentSize = random.nextInt(1_000) + 100; documentBytes = new byte[documentSize]; - documentId = RandomStringUtils.randomAlphabetic(10); - routingField = RandomStringUtils.randomAlphabetic(10); + documentId = RandomStringUtils.randomAlphabetic(10); + routingField = RandomStringUtils.randomAlphabetic(10); + pipelineField = RandomStringUtils.randomAlphabetic(10); } private SerializedJsonImpl createObjectUnderTest() { - return new SerializedJsonImpl(documentBytes, documentId, routingField); + return new SerializedJsonImpl(documentBytes, documentId, routingField, pipelineField); } @Test @@ -45,5 +47,6 @@ void getSerializedJson_returns_the_document_byte_array_and_fields() { assertThat(createObjectUnderTest().getSerializedJson(), sameInstance(documentBytes)); assertThat(createObjectUnderTest().getDocumentId().get(), equalTo(documentId)); assertThat(createObjectUnderTest().getRoutingField().get(), equalTo(routingField)); + assertThat(createObjectUnderTest().getPipelineField().get(), equalTo(pipelineField)); } } diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonNodeTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonNodeTest.java index 1131a13b5d..0716b83248 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonNodeTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonNodeTest.java @@ -21,6 +21,7 @@ class SerializedJsonNodeTest { private byte[] documentBytes; private String documentId; private String routingField; + private String pipelineField; private JsonNode jsonNode; private SerializedJson document; private String jsonString; @@ -39,7 +40,8 @@ void setUp() { } documentId = RandomStringUtils.randomAlphabetic(10); routingField = RandomStringUtils.randomAlphabetic(10); - document = SerializedJson.fromStringAndOptionals(jsonString, documentId, routingField); + pipelineField = RandomStringUtils.randomAlphabetic(10); + document = SerializedJson.fromStringAndOptionals(jsonString, documentId, routingField, pipelineField); } private SerializedJsonNode createObjectUnderTest() { @@ -56,6 +58,7 @@ void getSerializedJson_returns_the_document_byte_array_and_fields() { assertThat(createObjectUnderTest().getSerializedJson(), equalTo(jsonString.getBytes())); assertThat(createObjectUnderTest().getDocumentId().get(), equalTo(documentId)); assertThat(createObjectUnderTest().getRoutingField().get(), equalTo(routingField)); + assertThat(createObjectUnderTest().getPipelineField().get(), equalTo(pipelineField)); } } diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonTest.java index 836a1b1c4d..7735d117c0 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJsonTest.java @@ -18,22 +18,24 @@ class SerializedJsonTest { @Test void fromString_returns_SerializedJsonImpl() { - assertThat(SerializedJson.fromStringAndOptionals("{}", null, null), instanceOf(SerializedJsonImpl.class)); + assertThat(SerializedJson.fromStringAndOptionals("{}", null, null, null), instanceOf(SerializedJsonImpl.class)); } @Test void fromString_throws_if_the_jsonString_is_null() { - assertThrows(NullPointerException.class, () -> SerializedJson.fromStringAndOptionals(null, null, null)); + assertThrows(NullPointerException.class, () -> SerializedJson.fromStringAndOptionals(null, null, null, null)); } @Test void fromString_returns_SerializedJsonImpl_with_correctValues() { String documentId = RandomStringUtils.randomAlphabetic(10); String routingField = RandomStringUtils.randomAlphabetic(10); - SerializedJson serializedJson = SerializedJson.fromStringAndOptionals("{}", documentId, routingField); + String pipelineField = RandomStringUtils.randomAlphabetic(10); + SerializedJson serializedJson = SerializedJson.fromStringAndOptionals("{}", documentId, routingField, pipelineField); assertThat(serializedJson, instanceOf(SerializedJsonImpl.class)); assertThat(serializedJson.getDocumentId().get(), equalTo(documentId)); assertThat(serializedJson.getRoutingField().get(), equalTo(routingField)); + assertThat(serializedJson.getPipelineField().get(), equalTo(pipelineField)); assertThat(serializedJson.getSerializedJson(), equalTo("{}".getBytes())); } @@ -41,6 +43,7 @@ void fromString_returns_SerializedJsonImpl_with_correctValues() { void fromString_returns_SerializedJsonNode_with_correctValues() { String documentId = RandomStringUtils.randomAlphabetic(10); String routingField = RandomStringUtils.randomAlphabetic(10); + String pipelineField = RandomStringUtils.randomAlphabetic(10); final String jsonString = "{\"key\":\"value\"}"; JsonNode jsonNode; ObjectMapper objectMapper = new ObjectMapper(); @@ -49,11 +52,12 @@ void fromString_returns_SerializedJsonNode_with_correctValues() { } catch (Exception e) { jsonNode = null; } - SerializedJson document = SerializedJson.fromStringAndOptionals(jsonString, documentId, routingField); + SerializedJson document = SerializedJson.fromStringAndOptionals(jsonString, documentId, routingField, pipelineField); SerializedJson serializedJson = SerializedJson.fromJsonNode(jsonNode, document); assertThat(serializedJson, instanceOf(SerializedJsonNode.class)); assertThat(serializedJson.getDocumentId().get(), equalTo(documentId)); assertThat(serializedJson.getRoutingField().get(), equalTo(routingField)); + assertThat(serializedJson.getPipelineField().get(), equalTo(pipelineField)); assertThat(serializedJson.getSerializedJson(), equalTo(jsonString.getBytes())); } } diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/dlq/FailedBulkOperationConverterTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/dlq/FailedBulkOperationConverterTest.java index aedd2d304a..17f0e52079 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/dlq/FailedBulkOperationConverterTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/dlq/FailedBulkOperationConverterTest.java @@ -160,7 +160,7 @@ private void generateRandomDocument() { final String jsonString = String.format("{\"%s\": \"%s\", \"%s\": \"%s\"}", key1, value1, key2, value2); - document = SerializedJson.fromStringAndOptionals(jsonString, null, null); + document = SerializedJson.fromStringAndOptionals(jsonString, null, null, null); } 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 99251fb956..e14689e25d 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 @@ -52,6 +52,9 @@ import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.DISTRIBUTION_VERSION; import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.DOCUMENT_ROOT_KEY; import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.DOCUMENT_VERSION_EXPRESSION; +import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.PIPELINE; +import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.ROUTING; +import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.ROUTING_FIELD; import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.SERVERLESS; import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConfiguration.TEMPLATE_TYPE; import static org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexConstants.RAW_DEFAULT_TEMPLATE_FILE; @@ -476,6 +479,39 @@ public void testReadIndexConfig_emptyDocumentRootKey() { assertThrows(IllegalArgumentException.class, () -> IndexConfiguration.readIndexConfig(pluginSetting)); } + @Test + public void testReadIndexConfig_pipeline() { + final Map metadata = initializeConfigMetaData( + IndexType.CUSTOM.getValue(), "foo", null, null, null, null, null); + final String expectedPipelineValue = UUID.randomUUID().toString(); + metadata.put(PIPELINE, expectedPipelineValue); + final PluginSetting pluginSetting = getPluginSetting(metadata); + final IndexConfiguration indexConfiguration = IndexConfiguration.readIndexConfig(pluginSetting); + assertEquals(expectedPipelineValue, indexConfiguration.getPipeline()); + } + + @Test + public void testReadIndexConfig_routing() { + final Map metadata = initializeConfigMetaData( + IndexType.CUSTOM.getValue(), "foo", null, null, null, null, null); + final String expectedRoutingValue = UUID.randomUUID().toString(); + metadata.put(ROUTING, expectedRoutingValue); + final PluginSetting pluginSetting = getPluginSetting(metadata); + final IndexConfiguration indexConfiguration = IndexConfiguration.readIndexConfig(pluginSetting); + assertEquals(expectedRoutingValue, indexConfiguration.getRouting()); + } + + @Test + public void testReadIndexConfig_routingField() { + final Map metadata = initializeConfigMetaData( + IndexType.CUSTOM.getValue(), "foo", null, null, null, null, null); + final String expectedRoutingFieldValue = UUID.randomUUID().toString(); + metadata.put(ROUTING_FIELD, expectedRoutingFieldValue); + final PluginSetting pluginSetting = getPluginSetting(metadata); + final IndexConfiguration indexConfiguration = IndexConfiguration.readIndexConfig(pluginSetting); + assertEquals(expectedRoutingFieldValue, indexConfiguration.getRoutingField()); + } + @ParameterizedTest @ValueSource(strings = {"${key}", "${getMetadata(\"key\")}"}) public void testReadIndexConfig_withValidDocumentVersionExpression(final String versionExpression) { From 01ae1dc8c63bba2e19948f4a5af21f6bd1170bea Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Fri, 14 Jun 2024 08:37:55 -0700 Subject: [PATCH 015/159] Rebased to latest (#4614) Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../aggregate/AggregateProcessor.java | 18 +++ .../actions/CountAggregateAction.java | 40 ++++-- .../actions/CountAggregateActionConfig.java | 10 +- .../actions/HistogramAggregateAction.java | 34 +++-- ...AggregateProcessorStaticFunctionsTest.java | 40 ++++++ .../actions/CountAggregateActionTest.java | 88 ++++++++++++ .../HistogramAggregateActionTests.java | 135 +++++++++++++++++- 7 files changed, 344 insertions(+), 21 deletions(-) create mode 100644 data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorStaticFunctionsTest.java diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java index 8a6f9d5a6a..68cb6f6e65 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java @@ -20,6 +20,7 @@ import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher; +import java.math.BigDecimal; import java.util.Collection; import java.util.LinkedList; import java.util.List; @@ -147,6 +148,23 @@ public static long getTimeNanos(final Instant time) { return currentTimeNanos; } + public static Instant convertObjectToInstant(Object timeObject) { + if (timeObject instanceof Instant) { + return (Instant)timeObject; + } else if (timeObject instanceof String) { + return Instant.parse((String)timeObject); + } else if (timeObject instanceof Integer || timeObject instanceof Long) { + long value = ((Number)timeObject).longValue(); + return (value > 1E10) ? Instant.ofEpochMilli(value) : Instant.ofEpochSecond(value); + } else if (timeObject instanceof Double || timeObject instanceof Float || timeObject instanceof BigDecimal) { + double value = ((Number)timeObject).doubleValue(); + long seconds = (long) value; + long nanos = (long) ((value - seconds) * 1_000_000_000); + return Instant.ofEpochSecond(seconds, nanos); + } else { + throw new RuntimeException("Invalid format for time "+timeObject); + } + } @Override public void prepareForShutdown() { diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java index fae6a19289..30ccccb2d5 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java @@ -18,6 +18,8 @@ import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionInput; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionOutput; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionResponse; +import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateProcessor; +import static org.opensearch.dataprepper.plugins.processor.aggregate.AggregateProcessor.getTimeNanos; import org.opensearch.dataprepper.plugins.processor.aggregate.GroupState; import io.opentelemetry.proto.metrics.v1.AggregationTemporality; @@ -44,6 +46,7 @@ public class CountAggregateAction implements AggregateAction { static final boolean SUM_METRIC_IS_MONOTONIC = true; public final String countKey; public final String startTimeKey; + public final String endTimeKey; public final String outputFormat; private long startTimeNanos; @@ -51,15 +54,10 @@ public class CountAggregateAction implements AggregateAction { public CountAggregateAction(final CountAggregateActionConfig countAggregateActionConfig) { this.countKey = countAggregateActionConfig.getCountKey(); this.startTimeKey = countAggregateActionConfig.getStartTimeKey(); + this.endTimeKey = countAggregateActionConfig.getEndTimeKey(); this.outputFormat = countAggregateActionConfig.getOutputFormat(); } - private long getTimeNanos(Instant time) { - final long NANO_MULTIPLIER = 1_000 * 1_000 * 1_000; - long currentTimeNanos = time.getEpochSecond() * NANO_MULTIPLIER + time.getNano(); - return currentTimeNanos; - } - public Exemplar createExemplar(final Event event) { long curTimeNanos = getTimeNanos(Instant.now()); Map attributes = event.toMap(); @@ -81,15 +79,33 @@ public Exemplar createExemplar(final Event event) { @Override public AggregateActionResponse handleEvent(final Event event, final AggregateActionInput aggregateActionInput) { final GroupState groupState = aggregateActionInput.getGroupState(); + Instant eventStartTime = Instant.now(); + Instant eventEndTime = eventStartTime; + Object startTime = event.get(startTimeKey, Object.class); + Object endTime = event.get(endTimeKey, Object.class); + + if (startTime != null) { + eventStartTime = AggregateProcessor.convertObjectToInstant(startTime); + } + if (endTime != null) { + eventEndTime = AggregateProcessor.convertObjectToInstant(endTime); + } if (groupState.get(countKey) == null) { - groupState.put(startTimeKey, Instant.now()); groupState.putAll(aggregateActionInput.getIdentificationKeys()); groupState.put(countKey, 1); groupState.put(exemplarKey, createExemplar(event)); + groupState.put(startTimeKey, eventStartTime); + groupState.put(endTimeKey, eventEndTime); } else { Integer v = (Integer)groupState.get(countKey) + 1; groupState.put(countKey, v); - } + Instant groupStartTime = (Instant)groupState.get(startTimeKey); + Instant groupEndTime = (Instant)groupState.get(endTimeKey); + if (eventStartTime.isBefore(groupStartTime)) + groupState.put(startTimeKey, eventStartTime); + if (eventEndTime.isAfter(groupEndTime)) + groupState.put(endTimeKey, eventEndTime); + } return AggregateActionResponse.nullEventResponse(); } @@ -98,6 +114,8 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA GroupState groupState = aggregateActionInput.getGroupState(); Event event; Instant startTime = (Instant)groupState.get(startTimeKey); + Instant endTime = (Instant)groupState.get(endTimeKey); + groupState.remove(endTimeKey); if (outputFormat.equals(OutputFormat.RAW.toString())) { groupState.put(startTimeKey, startTime.atZone(ZoneId.of(ZoneId.systemDefault().toString())).format(DateTimeFormatter.ofPattern(DATE_FORMAT))); event = JacksonEvent.builder() @@ -110,14 +128,14 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA groupState.remove(exemplarKey); groupState.remove(countKey); groupState.remove(startTimeKey); - long currentTimeNanos = getTimeNanos(Instant.now()); + long endTimeNanos = getTimeNanos(endTime); long startTimeNanos = getTimeNanos(startTime); Map attr = new HashMap(); groupState.forEach((k, v) -> attr.put((String)k, v)); JacksonSum sum = JacksonSum.builder() .withName(SUM_METRIC_NAME) .withDescription(SUM_METRIC_DESCRIPTION) - .withTime(OTelProtoCodec.convertUnixNanosToISO8601(currentTimeNanos)) + .withTime(OTelProtoCodec.convertUnixNanosToISO8601(endTimeNanos)) .withStartTime(OTelProtoCodec.convertUnixNanosToISO8601(startTimeNanos)) .withIsMonotonic(SUM_METRIC_IS_MONOTONIC) .withUnit(SUM_METRIC_UNIT) @@ -128,7 +146,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA .build(false); event = (Event)sum; } - + return new AggregateActionOutput(List.of(event)); } } diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java index cbe5ebb20b..3577e035c4 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java @@ -12,6 +12,7 @@ public class CountAggregateActionConfig { public static final String DEFAULT_COUNT_KEY = "aggr._count"; public static final String DEFAULT_START_TIME_KEY = "aggr._start_time"; + public static final String DEFAULT_END_TIME_KEY = "aggr._end_time"; public static final Set validOutputFormats = new HashSet<>(Set.of(OutputFormat.OTEL_METRICS.toString(), OutputFormat.RAW.toString())); @JsonProperty("count_key") @@ -20,6 +21,9 @@ public class CountAggregateActionConfig { @JsonProperty("start_time_key") String startTimeKey = DEFAULT_START_TIME_KEY; + @JsonProperty("end_time_key") + String endTimeKey = DEFAULT_END_TIME_KEY; + @JsonProperty("output_format") String outputFormat = OutputFormat.OTEL_METRICS.toString(); @@ -27,6 +31,10 @@ public String getCountKey() { return countKey; } + public String getEndTimeKey() { + return endTimeKey; + } + public String getStartTimeKey() { return startTimeKey; } @@ -37,4 +45,4 @@ public String getOutputFormat() { } return outputFormat; } -} +} diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java index 5e93f305bc..fd4d7542f5 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java @@ -17,6 +17,7 @@ import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateAction; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionInput; +import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateProcessor; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionOutput; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionResponse; import org.opensearch.dataprepper.plugins.processor.aggregate.GroupState; @@ -35,8 +36,8 @@ import java.util.ArrayList; /** - * An AggregateAction that combines multiple Events into a single Event. This action will create a combined event with histogram buckets of the values - * of specified list of keys from the groupState on concludeGroup. + * An AggregateAction that combines multiple Events into a single Event. This action will create a combined event with histogram buckets of the values + * of specified list of keys from the groupState on concludeGroup. * @since 2.1 */ @DataPrepperPlugin(name = "histogram", pluginType = AggregateAction.class, pluginConfigurationType = HistogramAggregateActionConfig.class) @@ -137,16 +138,29 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct return AggregateActionResponse.nullEventResponse(); } double doubleValue = convertToDouble(value); - + int idx = Arrays.binarySearch(this.buckets, doubleValue); if (idx < 0) { idx = -idx-2; } + Instant eventTime = Instant.now(); + Instant eventStartTime = eventTime; + Instant eventEndTime = eventTime; + Object startTime = event.get(startTimeKey, Object.class); + Object endTime = event.get(endTimeKey, Object.class); + if (startTime != null) { + eventStartTime = AggregateProcessor.convertObjectToInstant(startTime); + } + if (endTime != null) { + eventEndTime = AggregateProcessor.convertObjectToInstant(endTime); + } if (groupState.get(bucketCountsKey) == null) { + groupState.put(startTimeKey, eventStartTime); + groupState.put(endTimeKey, eventEndTime); Long[] bucketCountsList = new Long[buckets.length-1]; Arrays.fill(bucketCountsList, (long)0); bucketCountsList[idx]++; - groupState.put(startTimeKey, Instant.now()); + groupState.put(startTimeKey, eventTime); groupState.putAll(aggregateActionInput.getIdentificationKeys()); groupState.put(sumKey, doubleValue); groupState.put(countKey, 1); @@ -180,9 +194,13 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct maxValue = doubleValue; } } - } - // Keep over-writing endTime to get the last time a record of this group received - groupState.put(endTimeKey, Instant.now()); + Instant groupStartTime = (Instant)groupState.get(startTimeKey); + Instant groupEndTime = (Instant)groupState.get(endTimeKey); + if (eventStartTime.isBefore(groupStartTime)) + groupState.put(startTimeKey, eventStartTime); + if (eventEndTime.isAfter(groupEndTime)) + groupState.put(endTimeKey, eventEndTime); + } return AggregateActionResponse.nullEventResponse(); } @@ -247,7 +265,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA .build(false); event = (Event)histogram; } - + return new AggregateActionOutput(List.of(event)); } } diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorStaticFunctionsTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorStaticFunctionsTest.java new file mode 100644 index 0000000000..8c9892ab29 --- /dev/null +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorStaticFunctionsTest.java @@ -0,0 +1,40 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.aggregate; + +import org.junit.jupiter.api.Test; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.time.Instant; +import java.time.Duration; + +public class AggregateProcessorStaticFunctionsTest { + @Test + public void testConvertObjectToInstant() { + Instant now = Instant.now(); + assertThat(AggregateProcessor.convertObjectToInstant(now), equalTo(now)); + String nowStr = now.toString(); + long nowSeconds = now.getEpochSecond(); + long nowMillis = now.toEpochMilli(); + int nowNanos = now.getNano(); + double nowDouble = nowSeconds+(double)nowNanos/1000_000_000; + assertThat(AggregateProcessor.convertObjectToInstant(nowStr), equalTo(now)); + assertThat(AggregateProcessor.convertObjectToInstant(nowSeconds), equalTo(Instant.ofEpochSecond(nowSeconds))); + assertThat(AggregateProcessor.convertObjectToInstant(nowMillis), equalTo(Instant.ofEpochMilli(nowMillis))); + Duration tolerance = Duration.ofNanos(1000); + assertTrue((Duration.between(AggregateProcessor.convertObjectToInstant(nowDouble), Instant.ofEpochSecond(nowSeconds, nowNanos))).abs().compareTo(tolerance) <= 0); + } + + @Test + public void testGetTimeNanos() { + Instant now = Instant.now(); + assertThat(AggregateProcessor.getTimeNanos(now) / 1000_000_000, equalTo(now.getEpochSecond())); + assertThat(AggregateProcessor.getTimeNanos(now) % 1000_000_000, equalTo((long)now.getNano())); + } +} + diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java index 66936fa7f8..f98fe0028a 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java @@ -25,8 +25,12 @@ import java.util.HashMap; import java.util.Map; import java.util.List; +import java.util.Random; import java.util.UUID; +import java.time.Instant; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.mock; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; @@ -139,4 +143,88 @@ void testCountAggregateOTelFormat(int testCount) { assertThat(attributes.get(key2), equalTo(value2)); assertTrue(attributes.containsKey(dataKey2)); } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 10, 100}) + void testCountAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCount) { + CountAggregateActionConfig mockConfig = mock(CountAggregateActionConfig.class); + when(mockConfig.getCountKey()).thenReturn(CountAggregateActionConfig.DEFAULT_COUNT_KEY); + String startTimeKey = UUID.randomUUID().toString(); + String endTimeKey = UUID.randomUUID().toString(); + when(mockConfig.getStartTimeKey()).thenReturn(startTimeKey); + when(mockConfig.getEndTimeKey()).thenReturn(endTimeKey); + when(mockConfig.getOutputFormat()).thenReturn(OutputFormat.OTEL_METRICS.toString()); + countAggregateAction = createObjectUnderTest(mockConfig); + final String key1 = "key-"+UUID.randomUUID().toString(); + final String value1 = UUID.randomUUID().toString(); + final String dataKey1 = "datakey-"+UUID.randomUUID().toString(); + final String key2 = "key-"+UUID.randomUUID().toString(); + final String value2 = UUID.randomUUID().toString(); + final String dataKey2 = "datakey-"+UUID.randomUUID().toString(); + final Instant testTime = Instant.ofEpochSecond(Instant.now().getEpochSecond()); + Map eventMap = Collections.singletonMap(key1, value1); + Event testEvent = JacksonEvent.builder() + .withEventType("event") + .withData(eventMap) + .build(); + Map eventMap2 = Collections.singletonMap(key2, value2); + JacksonEvent testEvent2 = JacksonEvent.builder() + .withEventType("event") + .withData(eventMap2) + .build(); + AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(eventMap); + AggregateActionInput aggregateActionInput2 = new AggregateActionTestUtils.TestAggregateActionInput(eventMap2); + Random random = new Random(); + for (int i = 0; i < testCount; i++) { + testEvent.put(dataKey1, UUID.randomUUID().toString()); + Instant sTime = (i == 0) ? testTime : testTime.plusSeconds(random.nextInt(5)); + Instant eTime = (i == testCount-1) ? testTime.plusSeconds(100) : testTime.plusSeconds (50+random.nextInt(45)); + testEvent.put(startTimeKey, sTime); + testEvent.put(endTimeKey, eTime); + testEvent2.put(dataKey2, UUID.randomUUID().toString()); + testEvent2.put(startTimeKey, sTime.toString()); + testEvent2.put(endTimeKey, eTime.toString()); + AggregateActionResponse aggregateActionResponse = countAggregateAction.handleEvent(testEvent, aggregateActionInput); + assertThat(aggregateActionResponse.getEvent(), equalTo(null)); + aggregateActionResponse = countAggregateAction.handleEvent(testEvent2, aggregateActionInput2); + assertThat(aggregateActionResponse.getEvent(), equalTo(null)); + } + + AggregateActionOutput actionOutput = countAggregateAction.concludeGroup(aggregateActionInput); + final List result = actionOutput.getEvents(); + assertThat(result.size(), equalTo(1)); + Map expectedEventMap = new HashMap<>(); + expectedEventMap.put("value", (double)testCount); + expectedEventMap.put("name", "count"); + expectedEventMap.put("description", "Number of events"); + expectedEventMap.put("isMonotonic", true); + expectedEventMap.put("aggregationTemporality", "AGGREGATION_TEMPORALITY_DELTA"); + expectedEventMap.put("unit", "1"); + expectedEventMap.forEach((k, v) -> assertThat(result.get(0).toMap(), hasEntry(k,v))); + assertThat(result.get(0).toMap().get("attributes"), equalTo(eventMap)); + JacksonMetric metric = (JacksonMetric) result.get(0); + assertThat(metric.toJsonString().indexOf("attributes"), not(-1)); + assertThat(result.get(0).get("startTime", String.class), equalTo(testTime.toString())); + assertThat(result.get(0).get("time", String.class), equalTo(testTime.plusSeconds(100).toString())); + + assertThat(result.get(0).toMap(), hasKey("startTime")); + assertThat(result.get(0).toMap(), hasKey("time")); + List> exemplars = (List >)result.get(0).toMap().get("exemplars"); + assertThat(exemplars.size(), equalTo(1)); + Map exemplar = exemplars.get(0); + Map attributes = (Map)exemplar.get("attributes"); + assertThat(attributes.get(key1), equalTo(value1)); + assertTrue(attributes.containsKey(dataKey1)); + + actionOutput = countAggregateAction.concludeGroup(aggregateActionInput2); + final List result2 = actionOutput.getEvents(); + assertThat(result2.size(), equalTo(1)); + + exemplars = (List >)result2.get(0).toMap().get("exemplars"); + assertThat(exemplars.size(), equalTo(1)); + exemplar = exemplars.get(0); + attributes = (Map)exemplar.get("attributes"); + assertThat(attributes.get(key2), equalTo(value2)); + assertTrue(attributes.containsKey(dataKey2)); + } } diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java index b2b498306b..1f5ca5e52c 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java @@ -23,13 +23,18 @@ import org.apache.commons.lang3.RandomStringUtils; import java.util.Arrays; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.List; -import java.util.ArrayList; +import java.util.Random; +import java.util.UUID; +import java.time.Instant; import java.util.concurrent.ThreadLocalRandom; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.mock; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; @@ -235,4 +240,132 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException } } } + + @ParameterizedTest + @ValueSource(ints = {10, 20, 50, 100}) + void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCount) throws NoSuchFieldException, IllegalAccessException { + HistogramAggregateActionConfig mockConfig = mock(HistogramAggregateActionConfig.class); + String startTimeKey = UUID.randomUUID().toString(); + String endTimeKey = UUID.randomUUID().toString(); + final String testKeyPrefix = RandomStringUtils.randomAlphabetic(5)+"_"; + when(mockConfig.getStartTimeKey()).thenReturn(startTimeKey); + when(mockConfig.getEndTimeKey()).thenReturn(endTimeKey); + when(mockConfig.getOutputFormat()).thenReturn(OutputFormat.OTEL_METRICS.toString()); + String keyPrefix = UUID.randomUUID().toString(); + final String testUnits = "ms"; + when(mockConfig.getUnits()).thenReturn(testUnits); + when(mockConfig.getRecordMinMax()).thenReturn(true); + final double TEST_VALUE_RANGE_MIN = 0.0; + final double TEST_VALUE_RANGE_MAX = 6.0; + final double TEST_VALUE_RANGE_STEP = 2.0; + final double bucket1 = TEST_VALUE_RANGE_MIN; + final double bucket2 = bucket1 + TEST_VALUE_RANGE_STEP; + final double bucket3 = bucket2 + TEST_VALUE_RANGE_STEP; + List buckets = new ArrayList(); + buckets.add(bucket1); + buckets.add(bucket2); + buckets.add(bucket3); + when(mockConfig.getBuckets()).thenReturn(buckets); + final String testKey = RandomStringUtils.randomAlphabetic(10); + when(mockConfig.getKey()).thenReturn(testKey); + final String testPrefix = RandomStringUtils.randomAlphabetic(7); + when(mockConfig.getSumKey()).thenReturn(testPrefix+"sum"); + when(mockConfig.getMinKey()).thenReturn(testPrefix+"min"); + when(mockConfig.getMaxKey()).thenReturn(testPrefix+"max"); + when(mockConfig.getCountKey()).thenReturn(testPrefix+"count"); + when(mockConfig.getBucketsKey()).thenReturn(testPrefix+"buckets"); + when(mockConfig.getBucketCountsKey()).thenReturn(testPrefix+"bucketcounts"); + when(mockConfig.getDurationKey()).thenReturn(testPrefix+"duration"); + histogramAggregateAction = createObjectUnderTest(mockConfig); + final String dataKey = RandomStringUtils.randomAlphabetic(10); + final String dataValue = RandomStringUtils.randomAlphabetic(15); + final AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(Map.of(dataKey, dataValue)); + Long[] expectedBucketCounts = new Long[buckets.size()+1]; + double expectedSum = 0.0; + double expectedMin = TEST_VALUE_RANGE_MAX+TEST_VALUE_RANGE_STEP+1.0; + double expectedMax = TEST_VALUE_RANGE_MIN-TEST_VALUE_RANGE_STEP-1.0; + Arrays.fill(expectedBucketCounts, (long)0); + Random random = new Random(); + final Instant testTime = Instant.ofEpochSecond(Instant.now().getEpochSecond()); + for (int i = 0; i < testCount; i++) { + final double value = ThreadLocalRandom.current().nextDouble(TEST_VALUE_RANGE_MIN-TEST_VALUE_RANGE_STEP, TEST_VALUE_RANGE_MAX+TEST_VALUE_RANGE_STEP); + if (value < bucket1) { + expectedBucketCounts[0]++; + } else if (value < bucket2) { + expectedBucketCounts[1]++; + } else if (value < bucket3) { + expectedBucketCounts[2]++; + } else { + expectedBucketCounts[3]++; + } + expectedSum += value; + if (value < expectedMin) { + expectedMin = value; + } + if (value > expectedMax) { + expectedMax = value; + } + Instant sTime = (i == 0) ? testTime : testTime.plusSeconds(random.nextInt(5)); + Instant eTime = (i == testCount-1) ? testTime.plusSeconds(100) : testTime.plusSeconds (50+random.nextInt(45)); + Map eventMap = Collections.synchronizedMap(Map.of(testKey, value, startTimeKey, sTime, endTimeKey, eTime)); + Event testEvent = JacksonEvent.builder() + .withEventType("event") + .withData(eventMap) + .build(); + final AggregateActionResponse aggregateActionResponse = histogramAggregateAction.handleEvent(testEvent, aggregateActionInput); + assertThat(aggregateActionResponse.getEvent(), equalTo(null)); + } + + final AggregateActionOutput actionOutput = histogramAggregateAction.concludeGroup(aggregateActionInput); + final List result = actionOutput.getEvents(); + assertThat(result.size(), equalTo(1)); + final String expectedCountKey = mockConfig.getCountKey(); + final String expectedStartTimeKey = mockConfig.getStartTimeKey(); + Map expectedEventMap = new HashMap<>(Collections.singletonMap("count", (long)testCount)); + expectedEventMap.put("unit", testUnits); + expectedEventMap.put("name", HistogramAggregateAction.HISTOGRAM_METRIC_NAME); + expectedEventMap.put("sum", expectedSum); + expectedEventMap.put("min", expectedMin); + expectedEventMap.put("max", expectedMax); + expectedEventMap.put("bucketCounts", expectedBucketCounts.length); + expectedEventMap.put("explicitBoundsCount", expectedBucketCounts.length-1); + + expectedEventMap.forEach((k, v) -> assertThat(result.get(0).toMap(), hasEntry(k, v))); + assertThat(result.get(0).toMap(), hasKey("startTime")); + assertThat(result.get(0).toMap(), hasKey("time")); + final List bucketCountsFromResult = (ArrayList)result.get(0).toMap().get("bucketCountsList"); + for (int i = 0; i < expectedBucketCounts.length; i++) { + assertThat(expectedBucketCounts[i], equalTo(bucketCountsFromResult.get(i))); + } + assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(HistogramAggregateAction.HISTOGRAM_METRIC_NAME+"_key", testKey)); + List exemplars = (List )result.get(0).toMap().get("exemplars"); + assertThat(exemplars.size(), equalTo(2)); + assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(dataKey, dataValue)); + final String expectedDurationKey = mockConfig.getDurationKey(); + assertThat(((Map)result.get(0).toMap().get("attributes")), hasKey(expectedDurationKey)); + JacksonMetric metric = (JacksonMetric) result.get(0); + assertThat(metric.toJsonString().indexOf("attributes"), not(-1)); + final List explicitBoundsFromResult = (ArrayList)result.get(0).toMap().get("explicitBounds"); + double bucketVal = TEST_VALUE_RANGE_MIN; + for (int i = 0; i < explicitBoundsFromResult.size(); i++) { + assertThat(explicitBoundsFromResult.get(i), equalTo(bucketVal)); + bucketVal += TEST_VALUE_RANGE_STEP; + } + final List> bucketsFromResult = (ArrayList>)result.get(0).toMap().get("buckets"); + double expectedBucketMin = -Float.MAX_VALUE; + double expectedBucketMax = TEST_VALUE_RANGE_MIN; + for (int i = 0; i < bucketsFromResult.size(); i++) { + assertThat(bucketsFromResult.get(i), hasEntry("min", expectedBucketMin)); + assertThat(bucketsFromResult.get(i), hasEntry("max", expectedBucketMax)); + assertThat(bucketsFromResult.get(i), hasEntry("count", expectedBucketCounts[i])); + expectedBucketMin = expectedBucketMax; + expectedBucketMax += TEST_VALUE_RANGE_STEP; + if (i == bucketsFromResult.size()-2) { + expectedBucketMax = Float.MAX_VALUE; + } + } + + assertThat(result.get(0).get("startTime", String.class), equalTo(testTime.toString())); + assertThat(result.get(0).get("time", String.class), equalTo(testTime.plusSeconds(100).toString())); + } } From 7f64a6d6ccba8840f0e8c22eb8f52d426d590f0c Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 14 Jun 2024 11:00:52 -0500 Subject: [PATCH 016/159] Tools to generate User Agent strings in the performance-test project (#4620) Changes to the performance-test project to generate User Agent strings. Used to help reproduce and test #4618. Signed-off-by: David Venable --- .../test/performance/tools/Templates.java | 25 +++++-- .../test/data/generation/UserAgent.java | 66 +++++++++++++++++++ .../test/data/generation/UserAgentTest.java | 40 +++++++++++ 3 files changed, 124 insertions(+), 7 deletions(-) create mode 100644 performance-test/src/main/java/org/opensearch/dataprepper/test/data/generation/UserAgent.java create mode 100644 performance-test/src/test/java/org/opensearch/dataprepper/test/data/generation/UserAgentTest.java diff --git a/performance-test/src/gatling/java/org/opensearch/dataprepper/test/performance/tools/Templates.java b/performance-test/src/gatling/java/org/opensearch/dataprepper/test/performance/tools/Templates.java index 27df8a1848..67347ff393 100644 --- a/performance-test/src/gatling/java/org/opensearch/dataprepper/test/performance/tools/Templates.java +++ b/performance-test/src/gatling/java/org/opensearch/dataprepper/test/performance/tools/Templates.java @@ -7,12 +7,14 @@ import io.gatling.javaapi.core.Session; import org.opensearch.dataprepper.test.data.generation.IpAddress; +import org.opensearch.dataprepper.test.data.generation.UserAgent; import java.time.LocalDateTime; import java.time.format.DateTimeFormatter; import java.util.List; import java.util.Random; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -30,19 +32,28 @@ public static String now() { } public static Function apacheCommonLogTemplate(final int batchSize) { - return session -> { - final List logs = IntStream.range(0, batchSize) - .mapToObj(i -> "{\"log\": \"" + ipAddress() + " - frank [" + now() + "] \\\"" + httpMethod() + " /apache_pb.gif HTTP/1.0\\\" "+ statusCode() + " " + responseSize() + "\"}") - .collect(Collectors.toList()); - final String logArray = String.join(",", logs); - return "[" + logArray + "]"; - }; + return generateLogArray(batchSize, + () -> ipAddress() + " - frank [" + now() + "] \\\"" + httpMethod() + " /apache_pb.gif HTTP/1.0\\\" "+ statusCode() + " " + responseSize()); + } + + public static Function userAgent(final int batchSize) { + return generateLogArray(batchSize, () -> userAgent()); + } + + private static Function generateLogArray(final int batchSize, final Supplier stringSupplier) { + return session -> IntStream.range(0, batchSize) + .mapToObj(i -> "{\"log\": \"" + stringSupplier.get() + "\"}") + .collect(Collectors.joining(",", "[", "]")); } private static String ipAddress() { return IpAddress.getInstance().ipAddress(); } + private static String userAgent() { + return UserAgent.getInstance().userAgent(); + } + private static String httpMethod() { return randomFromList(HTTP_METHODS); } diff --git a/performance-test/src/main/java/org/opensearch/dataprepper/test/data/generation/UserAgent.java b/performance-test/src/main/java/org/opensearch/dataprepper/test/data/generation/UserAgent.java new file mode 100644 index 0000000000..ced3b090c4 --- /dev/null +++ b/performance-test/src/main/java/org/opensearch/dataprepper/test/data/generation/UserAgent.java @@ -0,0 +1,66 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.test.data.generation; + +import java.util.Random; +import java.util.UUID; + +public class UserAgent { + private static final UserAgent USER_AGENT = new UserAgent(); + private final Random random; + + private UserAgent() { + random = new Random(); + } + + public static UserAgent getInstance() { + return USER_AGENT; + } + + public String userAgent() { + final StringBuilder userAgentBuilder = new StringBuilder(); + + buildBrowserPart(userAgentBuilder); + userAgentBuilder.append(" ("); + + buildDevicePart(userAgentBuilder); + userAgentBuilder.append(" "); + + buildOsPart(userAgentBuilder); + userAgentBuilder.append(")"); + + return userAgentBuilder.toString(); + } + + private void buildOsPart(final StringBuilder userAgentBuilder) { + userAgentBuilder.append(randomString()); + userAgentBuilder.append(" "); + buildVersionString(userAgentBuilder); + } + + private void buildDevicePart(final StringBuilder userAgentBuilder) { + userAgentBuilder.append(randomString()); + userAgentBuilder.append(";"); + } + + private void buildBrowserPart(final StringBuilder userAgentBuilder) { + userAgentBuilder.append(randomString()); + userAgentBuilder.append("/"); + buildVersionString(userAgentBuilder); + } + + private void buildVersionString(final StringBuilder userAgentBuilder) { + userAgentBuilder.append(random.nextInt(9) + 1); + userAgentBuilder.append("."); + userAgentBuilder.append(random.nextInt(30)); + userAgentBuilder.append("."); + userAgentBuilder.append(random.nextInt(30)); + } + + private static String randomString() { + return UUID.randomUUID().toString().replaceAll("-", ""); + } +} diff --git a/performance-test/src/test/java/org/opensearch/dataprepper/test/data/generation/UserAgentTest.java b/performance-test/src/test/java/org/opensearch/dataprepper/test/data/generation/UserAgentTest.java new file mode 100644 index 0000000000..0967c3c3e4 --- /dev/null +++ b/performance-test/src/test/java/org/opensearch/dataprepper/test/data/generation/UserAgentTest.java @@ -0,0 +1,40 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.test.data.generation; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.matchesPattern; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.notNullValue; + +class UserAgentTest { + @Test + void userAgent_returns_string() { + final String userAgent = UserAgent.getInstance().userAgent(); + + assertThat(userAgent, notNullValue()); + assertThat(userAgent.length(), greaterThanOrEqualTo(10)); + + String expectedRegex = "^[A-Za-z0-9]+/[0-9]+.[0-9]+.[0-9]+ \\([A-Za-z0-9]+; [A-Za-z0-9]+ [0-9]+.[0-9]+.[0-9]+\\)$"; + + assertThat(userAgent, matchesPattern(expectedRegex)); + } + + @Test + void userAgent_returns_unique_value_on_multiple_calls() { + final UserAgent objectUnderTest = UserAgent.getInstance(); + final String userAgent = objectUnderTest.userAgent(); + + assertThat(userAgent, notNullValue()); + assertThat(objectUnderTest.userAgent(), not(equalTo(userAgent))); + assertThat(objectUnderTest.userAgent(), not(equalTo(userAgent))); + assertThat(objectUnderTest.userAgent(), not(equalTo(userAgent))); + } +} \ No newline at end of file From 515230042f763ffc92b93fb136abdcb91412f1e2 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Fri, 14 Jun 2024 09:27:17 -0700 Subject: [PATCH 017/159] Support multiple aggregate processors in local mode (#4574) * Rebased to latest Signed-off-by: Krishna Kondaka * Fixed tests Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../DefaultPeerForwarderProvider.java | 102 ++++++++++++++ .../LocalModePeerForwarderProvider.java | 51 +++++++ .../peerforwarder/PeerForwarderAppConfig.java | 16 ++- .../peerforwarder/PeerForwarderProvider.java | 126 ++++++------------ .../PeerForwardingProcessorDecorator.java | 9 +- ... => DefaultPeerForwarderProviderTest.java} | 32 +++-- .../PeerForwarder_ClientServerIT.java | 12 +- ...PeerForwardingProcessingDecoratorTest.java | 65 +++++++-- 8 files changed, 294 insertions(+), 119 deletions(-) create mode 100644 data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProvider.java create mode 100644 data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/LocalModePeerForwarderProvider.java rename data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/{PeerForwarderProviderTest.java => DefaultPeerForwarderProviderTest.java} (82%) diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProvider.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProvider.java new file mode 100644 index 0000000000..ff638ee26f --- /dev/null +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProvider.java @@ -0,0 +1,102 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.peerforwarder; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.processor.Processor; +import org.opensearch.dataprepper.peerforwarder.client.PeerForwarderClient; +import org.opensearch.dataprepper.peerforwarder.discovery.DiscoveryMode; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; + +public class DefaultPeerForwarderProvider implements PeerForwarderProvider { + + private final PeerForwarderClientFactory peerForwarderClientFactory; + private final PeerForwarderClient peerForwarderClient; + private final PeerForwarderConfiguration peerForwarderConfiguration; + private final PluginMetrics pluginMetrics; + private final Map>>> pipelinePeerForwarderReceiveBufferMap = new HashMap<>(); + private HashRing hashRing; + + DefaultPeerForwarderProvider(final PeerForwarderClientFactory peerForwarderClientFactory, + final PeerForwarderClient peerForwarderClient, + final PeerForwarderConfiguration peerForwarderConfiguration, + final PluginMetrics pluginMetrics) { + this.peerForwarderClientFactory = peerForwarderClientFactory; + this.peerForwarderClient = peerForwarderClient; + this.peerForwarderConfiguration = peerForwarderConfiguration; + this.pluginMetrics = pluginMetrics; + } + + public PeerForwarder register(final String pipelineName, final Processor processor, final String pluginId, final Set identificationKeys, + final Integer pipelineWorkerThreads) { + if (pipelinePeerForwarderReceiveBufferMap.containsKey(pipelineName) && + pipelinePeerForwarderReceiveBufferMap.get(pipelineName).containsKey(pluginId)) { + throw new RuntimeException("Data Prepper 2.0 will only support a single peer-forwarder per pipeline/plugin type"); + } + + final PeerForwarderReceiveBuffer> peerForwarderReceiveBuffer = createBufferPerPipelineProcessor(pipelineName, pluginId); + + if (isPeerForwardingRequired()) { + if (hashRing == null) { + hashRing = peerForwarderClientFactory.createHashRing(); + } + return new RemotePeerForwarder( + peerForwarderClient, + hashRing, + peerForwarderReceiveBuffer, + pipelineName, + pluginId, + identificationKeys, + pluginMetrics, + peerForwarderConfiguration.getBatchDelay(), + peerForwarderConfiguration.getFailedForwardingRequestLocalWriteTimeout(), + peerForwarderConfiguration.getForwardingBatchSize(), + peerForwarderConfiguration.getForwardingBatchQueueDepth(), + peerForwarderConfiguration.getForwardingBatchTimeout(), + pipelineWorkerThreads + ); + } + else { + return new LocalPeerForwarder(); + } + } + + private PeerForwarderReceiveBuffer> createBufferPerPipelineProcessor(final String pipelineName, final String pluginId) { + final PeerForwarderReceiveBuffer> peerForwarderReceiveBuffer = new + PeerForwarderReceiveBuffer<>(peerForwarderConfiguration.getBufferSize(), peerForwarderConfiguration.getBatchSize(), pipelineName, pluginId); + + final Map>> pluginsBufferMap = + pipelinePeerForwarderReceiveBufferMap.computeIfAbsent(pipelineName, k -> new HashMap<>()); + + pluginsBufferMap.put(pluginId, peerForwarderReceiveBuffer); + + return peerForwarderReceiveBuffer; + } + + public boolean isPeerForwardingRequired() { + return arePeersConfigured() && pipelinePeerForwarderReceiveBufferMap.size() > 0; + } + + public boolean arePeersConfigured() { + final DiscoveryMode discoveryMode = peerForwarderConfiguration.getDiscoveryMode(); + if (discoveryMode.equals(DiscoveryMode.LOCAL_NODE)) { + return false; + } + else if (discoveryMode.equals(DiscoveryMode.STATIC) && peerForwarderConfiguration.getStaticEndpoints().size() <= 1) { + return false; + } + return true; + } + + public Map>>> getPipelinePeerForwarderReceiveBufferMap() { + return pipelinePeerForwarderReceiveBufferMap; + } +} diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/LocalModePeerForwarderProvider.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/LocalModePeerForwarderProvider.java new file mode 100644 index 0000000000..6c2c4fe688 --- /dev/null +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/LocalModePeerForwarderProvider.java @@ -0,0 +1,51 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.peerforwarder; + +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.processor.Processor; +import org.opensearch.dataprepper.model.peerforwarder.RequiresPeerForwarding; + +import java.util.Map; +import java.util.Set; + +public class LocalModePeerForwarderProvider implements PeerForwarderProvider { + + private final PeerForwarderProvider peerForwarderProvider; + private boolean isRemotePeerForwarderRegistered; + + public LocalModePeerForwarderProvider(final PeerForwarderProvider peerForwarderProvider) { + this.peerForwarderProvider = peerForwarderProvider; + this.isRemotePeerForwarderRegistered = false; + } + + @Override + public PeerForwarder register(final String pipelineName, final Processor processor, final String pluginId, final Set identificationKeys, final Integer pipelineWorkerThreads) { + if (((RequiresPeerForwarding)processor).isForLocalProcessingOnly(null)) { + return new LocalPeerForwarder(); + } + isRemotePeerForwarderRegistered = true; + return peerForwarderProvider.register(pipelineName, processor, pluginId, identificationKeys, pipelineWorkerThreads); + } + + @Override + public boolean isPeerForwardingRequired() { + return isRemotePeerForwarderRegistered; + } + + @Override + public Map>>> getPipelinePeerForwarderReceiveBufferMap() { + return (isRemotePeerForwarderRegistered) ? + peerForwarderProvider.getPipelinePeerForwarderReceiveBufferMap() : + Map.of(); + } + + @Override + public boolean arePeersConfigured() { + return isRemotePeerForwarderRegistered ? peerForwarderProvider.arePeersConfigured() : false; + } +} diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderAppConfig.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderAppConfig.java index e3123b67f1..4cca81819d 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderAppConfig.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderAppConfig.java @@ -20,6 +20,10 @@ import org.springframework.beans.factory.annotation.Qualifier; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; +import org.springframework.context.annotation.Primary; + +import javax.inject.Named; + @Configuration class PeerForwarderAppConfig { @@ -71,12 +75,18 @@ public PeerForwarderClient peerForwarderClient(final PeerForwarderConfiguration peerForwarderConfiguration, peerForwarderClientFactory, peerForwarderCodec, pluginMetrics); } - @Bean - public PeerForwarderProvider peerForwarderProvider(final PeerForwarderClientFactory peerForwarderClientFactory, + @Bean(name = "defaultPeerForwarder") + public DefaultPeerForwarderProvider peerForwarderProvider(final PeerForwarderClientFactory peerForwarderClientFactory, final PeerForwarderClient peerForwarderClient, final PeerForwarderConfiguration peerForwarderConfiguration, @Qualifier("peerForwarderMetrics") final PluginMetrics pluginMetrics) { - return new PeerForwarderProvider(peerForwarderClientFactory, peerForwarderClient, peerForwarderConfiguration, pluginMetrics); + return new DefaultPeerForwarderProvider(peerForwarderClientFactory, peerForwarderClient, peerForwarderConfiguration, pluginMetrics); + } + + @Bean + @Primary + public PeerForwarderProvider peerForwarderProvider(@Named("defaultPeerForwarder") final PeerForwarderProvider peerForwarderProvider) { + return new LocalModePeerForwarderProvider(peerForwarderProvider); } @Bean diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProvider.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProvider.java index ea89c1cbdd..40b3a03deb 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProvider.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProvider.java @@ -5,97 +5,49 @@ package org.opensearch.dataprepper.peerforwarder; -import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.peerforwarder.client.PeerForwarderClient; -import org.opensearch.dataprepper.peerforwarder.discovery.DiscoveryMode; +import org.opensearch.dataprepper.model.processor.Processor; -import java.util.HashMap; import java.util.Map; import java.util.Set; -public class PeerForwarderProvider { - - private final PeerForwarderClientFactory peerForwarderClientFactory; - private final PeerForwarderClient peerForwarderClient; - private final PeerForwarderConfiguration peerForwarderConfiguration; - private final PluginMetrics pluginMetrics; - private final Map>>> pipelinePeerForwarderReceiveBufferMap = new HashMap<>(); - private HashRing hashRing; - - PeerForwarderProvider(final PeerForwarderClientFactory peerForwarderClientFactory, - final PeerForwarderClient peerForwarderClient, - final PeerForwarderConfiguration peerForwarderConfiguration, - final PluginMetrics pluginMetrics) { - this.peerForwarderClientFactory = peerForwarderClientFactory; - this.peerForwarderClient = peerForwarderClient; - this.peerForwarderConfiguration = peerForwarderConfiguration; - this.pluginMetrics = pluginMetrics; - } - - public PeerForwarder register(final String pipelineName, final String pluginId, final Set identificationKeys, - final Integer pipelineWorkerThreads) { - if (pipelinePeerForwarderReceiveBufferMap.containsKey(pipelineName) && - pipelinePeerForwarderReceiveBufferMap.get(pipelineName).containsKey(pluginId)) { - throw new RuntimeException("Data Prepper 2.0 will only support a single peer-forwarder per pipeline/plugin type"); - } - - final PeerForwarderReceiveBuffer> peerForwarderReceiveBuffer = createBufferPerPipelineProcessor(pipelineName, pluginId); - - if (isPeerForwardingRequired()) { - if (hashRing == null) { - hashRing = peerForwarderClientFactory.createHashRing(); - } - return new RemotePeerForwarder( - peerForwarderClient, - hashRing, - peerForwarderReceiveBuffer, - pipelineName, - pluginId, - identificationKeys, - pluginMetrics, - peerForwarderConfiguration.getBatchDelay(), - peerForwarderConfiguration.getFailedForwardingRequestLocalWriteTimeout(), - peerForwarderConfiguration.getForwardingBatchSize(), - peerForwarderConfiguration.getForwardingBatchQueueDepth(), - peerForwarderConfiguration.getForwardingBatchTimeout(), - pipelineWorkerThreads - ); - } - else { - return new LocalPeerForwarder(); - } - } - - private PeerForwarderReceiveBuffer> createBufferPerPipelineProcessor(final String pipelineName, final String pluginId) { - final PeerForwarderReceiveBuffer> peerForwarderReceiveBuffer = new - PeerForwarderReceiveBuffer<>(peerForwarderConfiguration.getBufferSize(), peerForwarderConfiguration.getBatchSize(), pipelineName, pluginId); - - final Map>> pluginsBufferMap = - pipelinePeerForwarderReceiveBufferMap.computeIfAbsent(pipelineName, k -> new HashMap<>()); - - pluginsBufferMap.put(pluginId, peerForwarderReceiveBuffer); - - return peerForwarderReceiveBuffer; - } - - public boolean isPeerForwardingRequired() { - return arePeersConfigured() && pipelinePeerForwarderReceiveBufferMap.size() > 0; - } - - private boolean arePeersConfigured() { - final DiscoveryMode discoveryMode = peerForwarderConfiguration.getDiscoveryMode(); - if (discoveryMode.equals(DiscoveryMode.LOCAL_NODE)) { - return false; - } - else if (discoveryMode.equals(DiscoveryMode.STATIC) && peerForwarderConfiguration.getStaticEndpoints().size() <= 1) { - return false; - } - return true; - } - - public Map>>> getPipelinePeerForwarderReceiveBufferMap() { - return pipelinePeerForwarderReceiveBufferMap; - } +public interface PeerForwarderProvider { + /** + * Registers a pipeline and identification keys + * + * @param pipelineName pipeline name + * @param processor processor + * @param pluginId plugin id + * @param identificationKeys identification keys + * @param pipelineWorkerThreads number of pipeline worker threads + * @return peer forwarder + * @since 2.9 + */ + PeerForwarder register(final String pipelineName, final Processor processor, final String pluginId, final Set identificationKeys, final Integer pipelineWorkerThreads); + + /** + * Returns if peer forwarding required + * + * @return returns if peer forwarding required or nto + * @since 2.9 + */ + boolean isPeerForwardingRequired(); + + /** + * Returns if peers configured + * + * @return returns if peers configured + * @since 2.9 + */ + boolean arePeersConfigured(); + + /** + * Returns pipeline peer forwarder receive buffer map + * + * @return Map of buffer per pipeline per pluginId + * @since 2.9 + */ + Map>>> getPipelinePeerForwarderReceiveBufferMap(); } + diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java index 58a99aadae..097b2b6552 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java @@ -67,10 +67,13 @@ public static List decorateProcessors( "Peer Forwarder Plugin: %s cannot have empty identification keys." + pluginId); } - final PeerForwarder peerForwarder = peerForwarderProvider.register(pipelineName, pluginId, identificationKeys, pipelineWorkerThreads); - return processors.stream().map(processor -> new PeerForwardingProcessorDecorator(peerForwarder, processor)) - .collect(Collectors.toList()); + return processors.stream() + .map(processor -> { + PeerForwarder peerForwarder = peerForwarderProvider.register(pipelineName, processor, pluginId, identificationKeys, pipelineWorkerThreads); + return new PeerForwardingProcessorDecorator(peerForwarder, processor); + }) + .collect(Collectors.toList()); } private PeerForwardingProcessorDecorator(final PeerForwarder peerForwarder, final Processor innerProcessor) { diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProviderTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProviderTest.java similarity index 82% rename from data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProviderTest.java rename to data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProviderTest.java index 08964d3a80..4c1c36482c 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarderProviderTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/DefaultPeerForwarderProviderTest.java @@ -13,6 +13,7 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.peerforwarder.client.PeerForwarderClient; import org.opensearch.dataprepper.peerforwarder.discovery.DiscoveryMode; @@ -35,7 +36,7 @@ import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -class PeerForwarderProviderTest { +class DefaultPeerForwarderProviderTest { private static final int PIPELINE_WORKER_THREADS = new Random().nextInt(10) + 1; @Mock @@ -50,6 +51,9 @@ class PeerForwarderProviderTest { @Mock private HashRing hashRing; + @Mock + private Processor processor; + @Mock private PluginMetrics pluginMetrics; @@ -71,13 +75,13 @@ void setUp() { } private PeerForwarderProvider createObjectUnderTest() { - return new PeerForwarderProvider(peerForwarderClientFactory, peerForwarderClient, peerForwarderConfiguration, pluginMetrics); + return new DefaultPeerForwarderProvider(peerForwarderClientFactory, peerForwarderClient, peerForwarderConfiguration, pluginMetrics); } @Test void register_creates_a_new_RemotePeerForwarder_with_cloud_map_discovery_mode() { when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.AWS_CLOUD_MAP); - final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); assertThat(peerForwarder, instanceOf(RemotePeerForwarder.class)); } @@ -86,7 +90,7 @@ void register_creates_a_new_RemotePeerForwarder_with_cloud_map_discovery_mode() void register_creates_a_new_RemotePeerForwarder_with_static_discovery_mode_of_size_grater_than_one() { when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.STATIC); when(peerForwarderConfiguration.getStaticEndpoints()).thenReturn(List.of("endpoint1", "endpoint2")); - final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); assertThat(peerForwarder, instanceOf(RemotePeerForwarder.class)); } @@ -95,14 +99,14 @@ void register_creates_a_new_RemotePeerForwarder_with_static_discovery_mode_of_si void register_creates_a_new_RemotePeerForwarder_with_static_discovery_mode_of_size_one() { when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.STATIC); when(peerForwarderConfiguration.getStaticEndpoints()).thenReturn(List.of("endpoint1")); - final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); assertThat(peerForwarder, instanceOf(LocalPeerForwarder.class)); } @Test void register_creates_a_new_LocalPeerForwarder_with_local_discovery_mode() { - final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + final PeerForwarder peerForwarder = createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); assertThat(peerForwarder, instanceOf(LocalPeerForwarder.class)); } @@ -110,7 +114,7 @@ void register_creates_a_new_LocalPeerForwarder_with_local_discovery_mode() { @Test void register_creates_HashRing_if_peer_forwarding_is_required() { when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.AWS_CLOUD_MAP); - createObjectUnderTest().register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + createObjectUnderTest().register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); verify(peerForwarderClientFactory).createHashRing(); } @@ -121,7 +125,7 @@ void register_called_multiple_times_creates_only_one_HashRing_if_peer_forwarding final PeerForwarderProvider objectUnderTest = createObjectUnderTest(); for (int i = 0; i < 10; i++) - objectUnderTest.register(pipelineName, UUID.randomUUID().toString(), identificationKeys, PIPELINE_WORKER_THREADS); + objectUnderTest.register(pipelineName, processor, UUID.randomUUID().toString(), identificationKeys, PIPELINE_WORKER_THREADS); verify(peerForwarderClientFactory, times(1)).createHashRing(); } @@ -137,17 +141,17 @@ void isAtLeastOnePeerForwarderRegistered_should_return_false_if_register_is_not_ void isAtLeastOnePeerForwarderRegistered_should_throw_when_register_is_called_with_same_pipeline_and_plugin() { final PeerForwarderProvider objectUnderTest = createObjectUnderTest(); - objectUnderTest.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + objectUnderTest.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); assertThrows(RuntimeException.class, () -> - objectUnderTest.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)); + objectUnderTest.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)); } @Test void isAtLeastOnePeerForwarderRegistered_should_return_false_if_register_is_called_with_local_discovery_mode() { final PeerForwarderProvider objectUnderTest = createObjectUnderTest(); - objectUnderTest.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + objectUnderTest.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); assertThat(objectUnderTest.isPeerForwardingRequired(), equalTo(false)); } @@ -157,7 +161,7 @@ void isAtLeastOnePeerForwarderRegistered_should_return_true_if_register_is_calle when(peerForwarderConfiguration.getDiscoveryMode()).thenReturn(DiscoveryMode.AWS_CLOUD_MAP); final PeerForwarderProvider objectUnderTest = createObjectUnderTest(); - objectUnderTest.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + objectUnderTest.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); assertThat(objectUnderTest.isPeerForwardingRequired(), equalTo(true)); } @@ -179,7 +183,7 @@ void getPipelinePeerForwarderReceiveBufferMap_should_return_empty_map_when_regis void getPipelinePeerForwarderReceiveBufferMap_should_return_non_empty_map_when_register_is_called() { final PeerForwarderProvider objectUnderTest = createObjectUnderTest(); - objectUnderTest.register(pipelineName, UUID.randomUUID().toString(), identificationKeys, PIPELINE_WORKER_THREADS); + objectUnderTest.register(pipelineName, processor, UUID.randomUUID().toString(), identificationKeys, PIPELINE_WORKER_THREADS); final Map>>> pipelinePeerForwarderReceiveBufferMap = objectUnderTest .getPipelinePeerForwarderReceiveBufferMap(); @@ -189,4 +193,4 @@ void getPipelinePeerForwarderReceiveBufferMap_should_return_non_empty_map_when_r assertThat(pipelinePeerForwarderReceiveBufferMap.size(), equalTo(1)); assertThat(pipelinePeerForwarderReceiveBufferMap.containsKey(pipelineName), equalTo(true)); } -} \ No newline at end of file +} diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarder_ClientServerIT.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarder_ClientServerIT.java index 2b4d875e45..f706cb97d7 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarder_ClientServerIT.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwarder_ClientServerIT.java @@ -22,6 +22,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.peerforwarder.certificate.CertificateProviderFactory; import org.opensearch.dataprepper.peerforwarder.client.PeerForwarderClient; import org.opensearch.dataprepper.peerforwarder.codec.PeerForwarderCodecAppConfig; @@ -133,7 +134,7 @@ private PeerForwarderProvider createPeerForwarderProvider( final PeerForwarderClient clientForProvider = createClient(peerForwarderConfiguration); final PeerClientPool peerClientPool = new PeerClientPool(); final PeerForwarderClientFactory clientFactoryForProvider = new PeerForwarderClientFactory(peerForwarderConfiguration, peerClientPool, certificateProviderFactory, pluginMetrics); - return new PeerForwarderProvider(clientFactoryForProvider, clientForProvider, peerForwarderConfiguration, pluginMetrics); + return new DefaultPeerForwarderProvider(clientFactoryForProvider, clientForProvider, peerForwarderConfiguration, pluginMetrics); } private PeerForwarderClient createClient( @@ -160,6 +161,7 @@ private Collection> getServerSideRecords(final PeerForwarderProvid class WithSSL { private PeerForwarderServer server; private PeerForwarderProvider peerForwarderProvider; + private Processor processor; void setUpServer(final boolean binaryCodec) { peerForwarderConfiguration = createConfiguration(true, ForwardingAuthentication.UNAUTHENTICATED, binaryCodec); @@ -168,7 +170,7 @@ void setUpServer(final boolean binaryCodec) { final CertificateProviderFactory certificateProviderFactory = new CertificateProviderFactory(peerForwarderConfiguration); peerForwarderProvider = createPeerForwarderProvider(peerForwarderConfiguration, certificateProviderFactory); - peerForwarderProvider.register(pipelineName, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS); + peerForwarderProvider.register(pipelineName, processor, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS); server = createServer(peerForwarderConfiguration, certificateProviderFactory, peerForwarderProvider); server.start(); } @@ -280,6 +282,7 @@ void send_Events_with_fingerprint_verification_to_unknown_server_should_throw(fi class WithoutSSL { private PeerForwarderServer server; private PeerForwarderProvider peerForwarderProvider; + private Processor processor; void setUpServer(final boolean binaryCodec) { peerForwarderConfiguration = createConfiguration(false, ForwardingAuthentication.UNAUTHENTICATED, binaryCodec); @@ -288,7 +291,7 @@ void setUpServer(final boolean binaryCodec) { final CertificateProviderFactory certificateProviderFactory = new CertificateProviderFactory(peerForwarderConfiguration); peerForwarderProvider = createPeerForwarderProvider(peerForwarderConfiguration, certificateProviderFactory); - peerForwarderProvider.register(pipelineName, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS); + peerForwarderProvider.register(pipelineName, processor, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS); server = createServer(peerForwarderConfiguration, certificateProviderFactory, peerForwarderProvider); server.start(); } @@ -339,6 +342,7 @@ void send_Events_to_server_when_expecting_SSL_should_throw(final boolean binaryC class WithMutualTls { private PeerForwarderServer server; private PeerForwarderProvider peerForwarderProvider; + private Processor processor; void setUpServer(final boolean binaryCodec) { peerForwarderConfiguration = createConfiguration(true, ForwardingAuthentication.MUTUAL_TLS, binaryCodec); @@ -347,7 +351,7 @@ void setUpServer(final boolean binaryCodec) { final CertificateProviderFactory certificateProviderFactory = new CertificateProviderFactory(peerForwarderConfiguration); peerForwarderProvider = createPeerForwarderProvider(peerForwarderConfiguration, certificateProviderFactory); - peerForwarderProvider.register(pipelineName, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS); + peerForwarderProvider.register(pipelineName, processor, pluginId, Collections.singleton(UUID.randomUUID().toString()), PIPELINE_WORKER_THREADS); server = createServer(peerForwarderConfiguration, certificateProviderFactory, peerForwarderProvider); server.start(); } diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java index d0c71a52d0..ceb424b0cb 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java @@ -17,6 +17,8 @@ import org.apache.commons.collections.CollectionUtils; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; import org.opensearch.dataprepper.peerforwarder.exception.EmptyPeerForwarderPluginIdentificationKeysException; import org.opensearch.dataprepper.peerforwarder.exception.UnsupportedPeerForwarderPluginException; @@ -49,6 +51,18 @@ class PeerForwardingProcessingDecoratorTest { @Mock private Processor processor; + @Mock + private Processor processor1; + + @Mock + private Processor processor2; + + @Mock(extraInterfaces = Processor.class) + private RequiresPeerForwarding requiresPeerForwarding1; + + @Mock(extraInterfaces = Processor.class) + private RequiresPeerForwarding requiresPeerForwarding2; + @Mock(extraInterfaces = Processor.class) private RequiresPeerForwarding requiresPeerForwarding; @@ -115,16 +129,53 @@ class WithRegisteredPeerForwarder { void setUp() { identificationKeys = Set.of(TEST_IDENTIFICATION_KEY); - when(peerForwarderProvider.register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(peerForwarder); - when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(identificationKeys); processor = (Processor) requiresPeerForwarding; + lenient().when(peerForwarderProvider.register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(peerForwarder); + when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(identificationKeys); } @Test void PeerForwardingProcessingDecorator_should_have_interaction_with_getIdentificationKeys() { createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); verify(requiresPeerForwarding, times(2)).getIdentificationKeys(); - verify(peerForwarderProvider).register(pipelineName, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + verify(peerForwarderProvider).register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + } + + @Test + void PeerForwardingProcessingDecorator_with_localProcessingOnly() { + List processorList = new ArrayList<>(); + processorList.add((Processor) requiresPeerForwarding); + processorList.add((Processor) requiresPeerForwardingCopy); + + LocalPeerForwarder localPeerForwarder = mock(LocalPeerForwarder.class); + lenient().when(peerForwarderProvider.register(pipelineName, (Processor) requiresPeerForwarding, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(localPeerForwarder); + lenient().when(peerForwarderProvider.register(pipelineName, (Processor) requiresPeerForwardingCopy, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(localPeerForwarder); + Event event = mock(Event.class); + when(record.getData()).thenReturn(event); + List> testData = Collections.singletonList(record); + when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event)).thenReturn(false); + when(requiresPeerForwardingCopy.isApplicableEventForPeerForwarding(event)).thenReturn(false); + + processor1 = (Processor)requiresPeerForwarding; + processor2 = (Processor)requiresPeerForwardingCopy; + when(processor1.execute(testData)).thenReturn(testData); + when(processor2.execute(testData)).thenReturn(testData); + + when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(identificationKeys); + when(requiresPeerForwardingCopy.getIdentificationKeys()).thenReturn(identificationKeys); + + when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(true); + when(requiresPeerForwardingCopy.isForLocalProcessingOnly(any())).thenReturn(true); + + final List processors = createObjectUnderTesDecoratedProcessors(processorList); + assertThat(processors.size(), equalTo(2)); + verify(peerForwarderProvider, times(1)).register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + Collection> result = processors.get(0).execute(testData); + assertThat(result.size(), equalTo(testData.size())); + assertThat(result, equalTo(testData)); + result = processors.get(1).execute(testData); + assertThat(result.size(), equalTo(testData.size())); + assertThat(result, equalTo(testData)); } @Test @@ -195,7 +246,7 @@ void PeerForwardingProcessingDecorator_execute_will_call_inner_processors_execut Event event = mock(Event.class); when(record.getData()).thenReturn(event); when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event)).thenReturn(false); - when(requiresPeerForwarding.isForLocalProcessingOnly(event)).thenReturn(true); + when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(true); final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); Collection> testData = Collections.singletonList(record); @@ -220,9 +271,7 @@ void PeerForwardingProcessingDecorator_inner_processor_with_is_applicable_event_ when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event1)).thenReturn(false); when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event2)).thenReturn(false); when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event3)).thenReturn(false); - when(requiresPeerForwarding.isForLocalProcessingOnly(event1)).thenReturn(true); - when(requiresPeerForwarding.isForLocalProcessingOnly(event2)).thenReturn(true); - when(requiresPeerForwarding.isForLocalProcessingOnly(event3)).thenReturn(true); + when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(true); final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); when(record1.getData()).thenReturn(event1); when(record2.getData()).thenReturn(event2); @@ -253,7 +302,7 @@ void PeerForwardingProcessingDecorator_inner_processor_with_is_applicable_event_ when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event1)).thenReturn(true); when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event2)).thenReturn(false); when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event3)).thenReturn(true); - when(requiresPeerForwarding.isForLocalProcessingOnly(event2)).thenReturn(false); + when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(false); final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); when(record1.getData()).thenReturn(event1); when(record2.getData()).thenReturn(event2); From 513d02f151091d82a64e6800919afd1d5b454e32 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 14 Jun 2024 11:55:28 -0500 Subject: [PATCH 018/159] MAINT: bump io.confluent:* packages to match org.apache.kafka.* (#4626) Signed-off-by: George Chen --- data-prepper-plugins/kafka-plugins/build.gradle | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index c88fde1365..0032bed806 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -29,6 +29,8 @@ dependencies { implementation project(':data-prepper-plugins:buffer-common') implementation project(':data-prepper-plugins:blocking-buffer') implementation project(':data-prepper-plugins:aws-plugin-api') + // bump io.confluent:* dependencies correspondingly when bumping org.apache.kafka.* + // https://docs.confluent.io/platform/current/release-notes/index.html implementation 'org.apache.kafka:kafka-clients:3.6.1' implementation 'org.apache.kafka:connect-json:3.6.1' implementation project(':data-prepper-plugins:http-common') @@ -36,9 +38,9 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'io.micrometer:micrometer-core' implementation libs.commons.lang3 - implementation 'io.confluent:kafka-avro-serializer:7.4.0' - implementation 'io.confluent:kafka-json-schema-serializer:7.4.0' - implementation 'io.confluent:kafka-schema-registry-client:7.4.0' + implementation 'io.confluent:kafka-avro-serializer:7.6.0' + implementation 'io.confluent:kafka-json-schema-serializer:7.6.0' + implementation 'io.confluent:kafka-schema-registry-client:7.6.0' implementation 'software.amazon.awssdk:sts' implementation 'software.amazon.awssdk:auth' implementation 'software.amazon.awssdk:kafka' @@ -75,8 +77,8 @@ dependencies { testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' integrationTestImplementation testLibs.junit.vintage - integrationTestImplementation 'io.confluent:kafka-schema-registry:7.4.0' - integrationTestImplementation ('io.confluent:kafka-schema-registry:7.4.0:tests') { + integrationTestImplementation 'io.confluent:kafka-schema-registry:7.6.0' + integrationTestImplementation ('io.confluent:kafka-schema-registry:7.6.0:tests') { exclude group: 'org.glassfish.jersey.containers', module: 'jersey-container-servlet' exclude group: 'org.glassfish.jersey.inject', module: 'jersey-hk2' exclude group: 'org.glassfish.jersey.ext', module: 'jersey-bean-validation' From 67a027c7858e8021e9e1fc90232fdd00e2e4e807 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 17 Jun 2024 09:26:05 -0500 Subject: [PATCH 019/159] Updates to Armeria 1.29.0 which fixes a bug that may help with #4080. (#4629) Signed-off-by: David Venable --- settings.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/settings.gradle b/settings.gradle index c72986922e..8400ff98c2 100644 --- a/settings.gradle +++ b/settings.gradle @@ -30,7 +30,7 @@ dependencyResolutionManagement { libs { version('slf4j', '2.0.6') library('slf4j-api', 'org.slf4j', 'slf4j-api').versionRef('slf4j') - version('armeria', '1.28.2') + version('armeria', '1.29.0') library('armeria-core', 'com.linecorp.armeria', 'armeria').versionRef('armeria') library('armeria-grpc', 'com.linecorp.armeria', 'armeria-grpc').versionRef('armeria') library('armeria-junit', 'com.linecorp.armeria', 'armeria-junit5').versionRef('armeria') From 52d2f0ed1b48b3bfeef9d5eeb573f047215a9782 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 17 Jun 2024 10:40:28 -0500 Subject: [PATCH 020/159] Adds the EventKey and EventKeyFactory. (#4627) Adds the EventKey and EventKeyFactory. Resolves #1916. Signed-off-by: David Venable --- .../dataprepper/model/event/Event.java | 66 ++++++ .../dataprepper/model/event/EventKey.java | 21 ++ .../model/event/EventKeyFactory.java | 71 ++++++ .../dataprepper/model/event/JacksonEvent.java | 206 +++++++++--------- .../model/event/JacksonEventKey.java | 137 ++++++++++++ .../model/event/EventActionTest.java | 69 ++++++ .../model/event/EventKeyFactoryTest.java | 47 ++++ .../model/event/JacksonEventKeyTest.java | 179 +++++++++++++++ .../model/event/JacksonEventTest.java | 137 +++++++++++- .../integration/ProcessorPipelineIT.java | 118 ++++++++++ .../plugins/InMemorySourceAccessor.java | 17 +- .../dataprepper/plugins/SimpleProcessor.java | 56 +++++ .../plugins/SimpleProcessorConfig.java | 19 ++ .../pipeline/processor-pipeline.yaml | 14 ++ .../core/event/DefaultEventKeyFactory.java | 20 ++ .../event/InternalOnlyEventKeyBridge.java | 17 ++ .../event/DefaultEventKeyFactoryTest.java | 52 +++++ .../ApplicationContextToTypedSuppliers.java | 3 + ...pplicationContextToTypedSuppliersTest.java | 17 +- .../dataprepper/event/TestEventContext.java | 24 ++ .../dataprepper/event/TestEventFactory.java | 10 +- .../event/TestEventKeyFactory.java | 30 +++ .../event/TestEventKeyFactoryTest.java | 56 +++++ 23 files changed, 1268 insertions(+), 118 deletions(-) create mode 100644 data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKey.java create mode 100644 data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyFactory.java create mode 100644 data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java create mode 100644 data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventActionTest.java create mode 100644 data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventKeyFactoryTest.java create mode 100644 data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java create mode 100644 data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java create mode 100644 data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java create mode 100644 data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java create mode 100644 data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml create mode 100644 data-prepper-event/src/main/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactory.java create mode 100644 data-prepper-event/src/main/java/org/opensearch/dataprepper/model/event/InternalOnlyEventKeyBridge.java create mode 100644 data-prepper-event/src/test/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactoryTest.java create mode 100644 data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventContext.java create mode 100644 data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventKeyFactory.java create mode 100644 data-prepper-test-event/src/test/java/org/opensearch/dataprepper/event/TestEventKeyFactoryTest.java diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/Event.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/Event.java index 740447ecc0..e0e36d9237 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/Event.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/Event.java @@ -26,6 +26,15 @@ */ public interface Event extends Serializable { + /** + * Adds or updates the key with a given value in the Event + * + * @param key where the value will be set + * @param value value to set the key to + * @since 2.8 + */ + void put(EventKey key, Object value); + /** * Adds or updates the key with a given value in the Event * @@ -35,6 +44,17 @@ public interface Event extends Serializable { */ void put(String key, Object value); + /** + * Retrieves the given key from the Event + * + * @param key the value to retrieve from + * @param clazz the return type of the value + * @param The type + * @return T a clazz object from the key + * @since 2.8 + */ + T get(EventKey key, Class clazz); + /** * Retrieves the given key from the Event * @@ -46,6 +66,17 @@ public interface Event extends Serializable { */ T get(String key, Class clazz); + /** + * Retrieves the given key from the Event as a List + * + * @param key the value to retrieve from + * @param clazz the return type of elements in the list + * @param The type + * @return {@literal List} a list of clazz elements + * @since 2.8 + */ + List getList(EventKey key, Class clazz); + /** * Retrieves the given key from the Event as a List * @@ -57,6 +88,14 @@ public interface Event extends Serializable { */ List getList(String key, Class clazz); + /** + * Deletes the given key from the Event + * + * @param key the field to be deleted + * @since 2.8 + */ + void delete(EventKey key); + /** * Deletes the given key from the Event * @@ -87,6 +126,15 @@ public interface Event extends Serializable { */ JsonNode getJsonNode(); + /** + * Gets a serialized Json string of the specific key in the Event + * + * @param key the field to be returned + * @return Json string of the field + * @since 2.8 + */ + String getAsJsonString(EventKey key); + /** * Gets a serialized Json string of the specific key in the Event * @@ -104,6 +152,15 @@ public interface Event extends Serializable { */ EventMetadata getMetadata(); + /** + * Checks if the key exists. + * + * @param key name of the key to look for + * @return returns true if the key exists, otherwise false + * @since 2.8 + */ + boolean containsKey(EventKey key); + /** * Checks if the key exists. * @@ -113,6 +170,15 @@ public interface Event extends Serializable { */ boolean containsKey(String key); + /** + * Checks if the value stored for the key is list + * + * @param key name of the key to look for + * @return returns true if the key is a list, otherwise false + * @since 2.8 + */ + boolean isValueAList(EventKey key); + /** * Checks if the value stored for the key is list * diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKey.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKey.java new file mode 100644 index 0000000000..9086f0f641 --- /dev/null +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKey.java @@ -0,0 +1,21 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +/** + * Model class to represent a key into a Data Prepper {@link Event}. + * + * @since 2.9 + */ +public interface EventKey { + /** + * The original key provided as a string. + * + * @return The key as a string + * @since 2.9 + */ + String getKey(); +} diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyFactory.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyFactory.java new file mode 100644 index 0000000000..e7cbc25463 --- /dev/null +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyFactory.java @@ -0,0 +1,71 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +import java.util.Arrays; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.Set; + +/** + * A factory for producing {@link EventKey} objects. + * + * @since 2.9 + */ +public interface EventKeyFactory { + /** + * Creates an {@link EventKey} with given actions. + * + * @param key The key + * @param forActions Actions to support + * @return The EventKey + * @since 2.9 + */ + EventKey createEventKey(String key, EventAction... forActions); + + /** + * Creates an {@link EventKey} for the default actions, which are all. + * + * @param key The key + * @return The EventKey + * @since 2.9 + */ + default EventKey createEventKey(final String key) { + return createEventKey(key, EventAction.ALL); + } + + /** + * An action on an Event. + * + * @since 2.9 + */ + enum EventAction { + GET, + DELETE, + PUT, + ALL(GET, DELETE, PUT); + + private final List includedActions; + + EventAction(EventAction... eventActions) { + includedActions = Arrays.asList(eventActions); + + } + + boolean isMutableAction() { + return this != GET; + } + + Set getSupportedActions() { + final EnumSet supportedActions = EnumSet.noneOf(EventAction.class); + supportedActions.add(this); + supportedActions.addAll(includedActions); + + return Collections.unmodifiableSet(supportedActions); + } + } +} diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java index 9ef34bb82c..35e0dd863b 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java @@ -28,8 +28,8 @@ import java.io.ObjectInputStream; import java.time.Instant; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; +import java.util.Deque; import java.util.HashMap; import java.util.Iterator; import java.util.LinkedList; @@ -38,8 +38,8 @@ import java.util.Objects; import java.util.StringJoiner; -import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static org.opensearch.dataprepper.model.event.JacksonEventKey.trimTrailingSlashInKey; /** * A Jackson Implementation of {@link Event} interface. This implementation relies heavily on JsonNode to manage the keys of the event. @@ -137,20 +137,15 @@ public JsonNode getJsonNode() { return jsonNode; } - /** - * Adds or updates the key with a given value in the Event. - * - * @param key where the value will be set - * @param value value to set the key to - * @since 1.2 - */ @Override - public void put(final String key, final Object value) { - checkArgument(!key.isEmpty(), "key cannot be an empty string for put method"); + public void put(EventKey key, Object value) { + final JacksonEventKey jacksonEventKey = asJacksonEventKey(key); - final String trimmedKey = checkAndTrimKey(key); + if(!jacksonEventKey.supports(EventKeyFactory.EventAction.PUT)) { + throw new IllegalArgumentException("key cannot be an empty string for put method"); + } - final LinkedList keys = new LinkedList<>(Arrays.asList(trimmedKey.split(SEPARATOR, -1))); + final Deque keys = new LinkedList<>(jacksonEventKey.getKeyPathList()); JsonNode parentNode = jsonNode; @@ -166,6 +161,19 @@ public void put(final String key, final Object value) { } } + /** + * Adds or updates the key with a given value in the Event. + * + * @param key where the value will be set + * @param value value to set the key to + * @since 1.2 + */ + @Override + public void put(final String key, final Object value) { + final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.PUT); + put(jacksonEventKey, value); + } + @Override public EventHandle getEventHandle() { return eventHandle; @@ -189,6 +197,27 @@ private JsonNode getOrCreateNode(final JsonNode node, final String key) { return childNode; } + @Override + public T get(EventKey key, Class clazz) { + JacksonEventKey jacksonEventKey = asJacksonEventKey(key); + + final JsonNode node = getNode(jacksonEventKey); + if (node.isMissingNode()) { + return null; + } + + return mapNodeToObject(key.getKey(), node, clazz); + } + + private static JacksonEventKey asJacksonEventKey(EventKey key) { + if(!(key instanceof JacksonEventKey)) { + throw new IllegalArgumentException("The key provided must be obtained through the EventKeyFactory."); + } + + JacksonEventKey jacksonEventKey = (JacksonEventKey) key; + return jacksonEventKey; + } + /** * Retrieves the value of type clazz from the key. * @@ -200,15 +229,8 @@ private JsonNode getOrCreateNode(final JsonNode node, final String key) { */ @Override public T get(final String key, final Class clazz) { - - final String trimmedKey = checkAndTrimKey(key); - - final JsonNode node = getNode(trimmedKey); - if (node.isMissingNode()) { - return null; - } - - return mapNodeToObject(key, node, clazz); + final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + return get(jacksonEventKey, clazz); } private JsonNode getNode(final String key) { @@ -216,6 +238,10 @@ private JsonNode getNode(final String key) { return jsonNode.at(jsonPointer); } + private JsonNode getNode(final JacksonEventKey key) { + return jsonNode.at(key.getJsonPointer()); + } + private T mapNodeToObject(final String key, final JsonNode node, final Class clazz) { try { return mapper.treeToValue(node, clazz); @@ -225,6 +251,18 @@ private T mapNodeToObject(final String key, final JsonNode node, final Class } } + @Override + public List getList(EventKey key, Class clazz) { + JacksonEventKey jacksonEventKey = asJacksonEventKey(key); + + final JsonNode node = getNode(jacksonEventKey); + if (node.isMissingNode()) { + return null; + } + + return mapNodeToList(jacksonEventKey.getKey(), node, clazz); + } + /** * Retrieves the given key from the Event as a List * @@ -236,15 +274,8 @@ private T mapNodeToObject(final String key, final JsonNode node, final Class */ @Override public List getList(final String key, final Class clazz) { - - final String trimmedKey = checkAndTrimKey(key); - - final JsonNode node = getNode(trimmedKey); - if (node.isMissingNode()) { - return null; - } - - return mapNodeToList(key, node, clazz); + JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + return getList(jacksonEventKey, clazz); } private List mapNodeToList(final String key, final JsonNode node, final Class clazz) { @@ -267,16 +298,15 @@ private JsonPointer toJsonPointer(final String key) { return JsonPointer.compile(jsonPointerExpression); } - /** - * Deletes the key from the event. - * - * @param key the field to be deleted - */ @Override - public void delete(final String key) { + public void delete(final EventKey key) { + final JacksonEventKey jacksonEventKey = asJacksonEventKey(key); + + if(!jacksonEventKey.supports(EventKeyFactory.EventAction.DELETE)) { + throw new IllegalArgumentException("key cannot be an empty string for delete method"); + } - checkArgument(!key.isEmpty(), "key cannot be an empty string for delete method"); - final String trimmedKey = checkAndTrimKey(key); + final String trimmedKey = jacksonEventKey.getTrimmedKey(); final int index = trimmedKey.lastIndexOf(SEPARATOR); JsonNode baseNode = jsonNode; @@ -293,6 +323,17 @@ public void delete(final String key) { } } + /** + * Deletes the key from the event. + * + * @param key the field to be deleted + */ + @Override + public void delete(final String key) { + final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.DELETE); + delete(jacksonEventKey); + } + @Override public void clear() { // Delete all entries from the event @@ -309,16 +350,22 @@ public String toJsonString() { } @Override - public String getAsJsonString(final String key) { - final String trimmedKey = checkAndTrimKey(key); + public String getAsJsonString(EventKey key) { - final JsonNode node = getNode(trimmedKey); + JacksonEventKey jacksonEventKey = asJacksonEventKey(key); + final JsonNode node = getNode(jacksonEventKey); if (node.isMissingNode()) { return null; } return node.toString(); } + @Override + public String getAsJsonString(final String key) { + JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + return getAsJsonString(jacksonEventKey); + } + /** * returns a string with formatted parts replaced by their values. The input * string may contain parts with format "${.../.../...}" which are replaced @@ -402,24 +449,35 @@ public EventMetadata getMetadata() { } @Override - public boolean containsKey(final String key) { - - final String trimmedKey = checkAndTrimKey(key); + public boolean containsKey(EventKey key) { + JacksonEventKey jacksonEventKey = asJacksonEventKey(key); - final JsonNode node = getNode(trimmedKey); + final JsonNode node = getNode(jacksonEventKey); return !node.isMissingNode(); } @Override - public boolean isValueAList(final String key) { - final String trimmedKey = checkAndTrimKey(key); + public boolean containsKey(final String key) { + JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + return containsKey(jacksonEventKey); + } - final JsonNode node = getNode(trimmedKey); + @Override + public boolean isValueAList(EventKey key) { + JacksonEventKey jacksonEventKey = asJacksonEventKey(key); + + final JsonNode node = getNode(jacksonEventKey); return node.isArray(); } + @Override + public boolean isValueAList(final String key) { + JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + return isValueAList(jacksonEventKey); + } + @Override public Map toMap() { return mapper.convertValue(jsonNode, MAP_TYPE_REFERENCE); @@ -427,30 +485,7 @@ public Map toMap() { public static boolean isValidEventKey(final String key) { - try { - checkKey(key); - return true; - } catch (final Exception e) { - return false; - } - } - private String checkAndTrimKey(final String key) { - checkKey(key); - return trimTrailingSlashInKey(key); - } - - private static void checkKey(final String key) { - checkNotNull(key, "key cannot be null"); - if (key.isEmpty()) { - // Empty string key is valid - return; - } - if (key.length() > MAX_KEY_LENGTH) { - throw new IllegalArgumentException("key cannot be longer than " + MAX_KEY_LENGTH + " characters"); - } - if (!isValidKey(key)) { - throw new IllegalArgumentException("key " + key + " must contain only alphanumeric chars with .-_@/ and must follow JsonPointer (ie. 'field/to/key')"); - } + return JacksonEventKey.isValidEventKey(key); } private String trimKey(final String key) { @@ -459,31 +494,6 @@ private String trimKey(final String key) { return trimTrailingSlashInKey(trimmedLeadingSlash); } - private String trimTrailingSlashInKey(final String key) { - return key.length() > 1 && key.endsWith(SEPARATOR) ? key.substring(0, key.length() - 1) : key; - } - - private static boolean isValidKey(final String key) { - for (int i = 0; i < key.length(); i++) { - char c = key.charAt(i); - - if (!(c >= 48 && c <= 57 - || c >= 65 && c <= 90 - || c >= 97 && c <= 122 - || c == '.' - || c == '-' - || c == '_' - || c == '@' - || c == '/' - || c == '[' - || c == ']')) { - - return false; - } - } - return true; - } - /** * Constructs an empty builder. * diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java new file mode 100644 index 0000000000..172e6b1254 --- /dev/null +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java @@ -0,0 +1,137 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +import com.fasterxml.jackson.core.JsonPointer; + +import java.util.Arrays; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; + +import static com.google.common.base.Preconditions.checkNotNull; + +class JacksonEventKey implements EventKey { + private static final String SEPARATOR = "/"; + private static final int MAX_KEY_LENGTH = 2048; + private final String key; + private final EventKeyFactory.EventAction[] eventActions; + private final String trimmedKey; + private final List keyPathList; + private final JsonPointer jsonPointer; + private final Set supportedActions; + + JacksonEventKey(final String key, final EventKeyFactory.EventAction... eventActions) { + this.key = Objects.requireNonNull(key, "Parameter key cannot be null for EventKey."); + this.eventActions = eventActions.length == 0 ? new EventKeyFactory.EventAction[] { EventKeyFactory.EventAction.ALL } : eventActions; + + if(key.isEmpty()) { + for (final EventKeyFactory.EventAction action : this.eventActions) { + if (action.isMutableAction()) { + throw new IllegalArgumentException("Event key cannot be an empty string for " + action + " actions."); + } + } + } + + trimmedKey = checkAndTrimKey(key); + + keyPathList = Collections.unmodifiableList(Arrays.asList(trimmedKey.split(SEPARATOR, -1))); + jsonPointer = toJsonPointer(trimmedKey); + + supportedActions = EnumSet.noneOf(EventKeyFactory.EventAction.class); + for (final EventKeyFactory.EventAction eventAction : this.eventActions) { + supportedActions.addAll(eventAction.getSupportedActions()); + } + + } + + @Override + public String getKey() { + return key; + } + + String getTrimmedKey() { + return trimmedKey; + } + + List getKeyPathList() { + return keyPathList; + } + + JsonPointer getJsonPointer() { + return jsonPointer; + } + + boolean supports(final EventKeyFactory.EventAction eventAction) { + return supportedActions.contains(eventAction); + } + + private String checkAndTrimKey(final String key) { + checkKey(key); + return trimTrailingSlashInKey(key); + } + + private static void checkKey(final String key) { + checkNotNull(key, "key cannot be null"); + if (key.isEmpty()) { + // Empty string key is valid + return; + } + if (key.length() > MAX_KEY_LENGTH) { + throw new IllegalArgumentException("key cannot be longer than " + MAX_KEY_LENGTH + " characters"); + } + if (!isValidKey(key)) { + throw new IllegalArgumentException("key " + key + " must contain only alphanumeric chars with .-_@/ and must follow JsonPointer (ie. 'field/to/key')"); + } + } + + + static String trimTrailingSlashInKey(final String key) { + return key.length() > 1 && key.endsWith(SEPARATOR) ? key.substring(0, key.length() - 1) : key; + } + + private static boolean isValidKey(final String key) { + for (int i = 0; i < key.length(); i++) { + char c = key.charAt(i); + + if (!(c >= 48 && c <= 57 + || c >= 65 && c <= 90 + || c >= 97 && c <= 122 + || c == '.' + || c == '-' + || c == '_' + || c == '@' + || c == '/' + || c == '[' + || c == ']')) { + + return false; + } + } + return true; + } + + private JsonPointer toJsonPointer(final String key) { + final String jsonPointerExpression; + if (key.isEmpty() || key.startsWith("/")) { + jsonPointerExpression = key; + } else { + jsonPointerExpression = SEPARATOR + key; + } + return JsonPointer.compile(jsonPointerExpression); + } + + static boolean isValidEventKey(final String key) { + try { + checkKey(key); + return true; + } catch (final Exception e) { + return false; + } + } +} diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventActionTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventActionTest.java new file mode 100644 index 0000000000..edb63fa663 --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventActionTest.java @@ -0,0 +1,69 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +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.hasItem; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +class EventActionTest { + @ParameterizedTest + @EnumSource(value = EventKeyFactory.EventAction.class, mode = EnumSource.Mode.EXCLUDE, names = {"GET"}) + void isMutableAction_is_true_for_mutable_actions(final EventKeyFactory.EventAction eventAction) { + assertThat(eventAction.isMutableAction(), equalTo(true)); + } + + @ParameterizedTest + @EnumSource(value = EventKeyFactory.EventAction.class, mode = EnumSource.Mode.INCLUDE, names = {"GET"}) + void isMutableAction_is_false_for_mutable_actions(final EventKeyFactory.EventAction eventAction) { + assertThat(eventAction.isMutableAction(), equalTo(false)); + } + + @ParameterizedTest + @EnumSource(value = EventKeyFactory.EventAction.class) + void getSupportedActions_includes_self(final EventKeyFactory.EventAction eventAction) { + assertThat(eventAction.getSupportedActions(), hasItem(eventAction)); + } + + @ParameterizedTest + @EnumSource(value = EventKeyFactory.EventAction.class) + void getSupportedActions_includes_for_all_actions_when_ALL(final EventKeyFactory.EventAction eventAction) { + assertThat(EventKeyFactory.EventAction.ALL.getSupportedActions(), hasItem(eventAction)); + } + + @ParameterizedTest + @ArgumentsSource(SupportsArgumentsProvider.class) + void supports_returns_expected_value(final EventKeyFactory.EventAction eventAction, final EventKeyFactory.EventAction otherAction, final boolean expectedSupports) { + assertThat(eventAction.getSupportedActions().contains(otherAction), equalTo(expectedSupports)); + } + + static class SupportsArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext extensionContext) throws Exception { + return Stream.of( + arguments(EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.PUT, false), + arguments(EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.DELETE, false), + arguments(EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.ALL, false), + arguments(EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.GET, false), + arguments(EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.DELETE, false), + arguments(EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.ALL, false), + arguments(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.GET, false), + arguments(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.PUT, false), + arguments(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.ALL, false) + ); + } + } +} \ No newline at end of file diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventKeyFactoryTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventKeyFactoryTest.java new file mode 100644 index 0000000000..c2ed2d56f3 --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/EventKeyFactoryTest.java @@ -0,0 +1,47 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +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 java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class EventKeyFactoryTest { + + private String keyPath; + + @Mock + private EventKey eventKey; + + @BeforeEach + void setUp() { + keyPath = UUID.randomUUID().toString(); + } + + private EventKeyFactory createObjectUnderTest() { + return mock(EventKeyFactory.class); + } + + @Test + void createEventKey_calls_with_ALL_action() { + final EventKeyFactory objectUnderTest = createObjectUnderTest(); + when(objectUnderTest.createEventKey(anyString())).thenCallRealMethod(); + when(objectUnderTest.createEventKey(keyPath, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey); + + assertThat(objectUnderTest.createEventKey(keyPath), equalTo(eventKey)); + } +} \ No newline at end of file diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java new file mode 100644 index 0000000000..359db06278 --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java @@ -0,0 +1,179 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +import com.fasterxml.jackson.core.JsonPointer; +import org.junit.jupiter.api.Test; +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.CsvSource; +import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.List; +import java.util.UUID; +import java.util.stream.Stream; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +class JacksonEventKeyTest { + @Test + void constructor_throws_with_null_key() { + assertThrows(NullPointerException.class, () -> new JacksonEventKey(null)); + } + + @Test + void getKey_with_empty_string_for_GET() { + final JacksonEventKey objectUnderTest = new JacksonEventKey("", EventKeyFactory.EventAction.GET); + assertThat(objectUnderTest.getKey(), equalTo("")); + assertThat(objectUnderTest.getTrimmedKey(), equalTo("")); + assertThat(objectUnderTest.getKeyPathList(), notNullValue()); + assertThat(objectUnderTest.getKeyPathList(), equalTo(List.of(""))); + assertThat(objectUnderTest.getJsonPointer(), notNullValue()); + } + + @ParameterizedTest + @EnumSource(value = EventKeyFactory.EventAction.class, mode = EnumSource.Mode.EXCLUDE, names = {"GET"}) + void constructor_throws_with_empty_string_for_unsupported_actions(final EventKeyFactory.EventAction eventAction) { + assertThrows(IllegalArgumentException.class, () -> new JacksonEventKey("", eventAction)); + } + + + @ParameterizedTest + @ValueSource(strings = { + "inv(alid", + "getMetadata(\"test_key\")" + }) + void constructor_throws_with_invalid_key(final String key) { + assertThrows(IllegalArgumentException.class, () -> new JacksonEventKey(key)); + } + + @ParameterizedTest + @ValueSource(strings = { + "test_key", + "/test_key", + "key.with.dot", + "key-with-hyphen", + "key_with_underscore", + "key@with@at", + "key[with]brackets" + }) + void getKey_returns_expected_result(final String key) { + assertThat(new JacksonEventKey(key).getKey(), equalTo(key)); + } + + @ParameterizedTest + @CsvSource(value = { + "test_key, test_key", + "/test_key, /test_key", + "/test_key/, /test_key", + "key.with.dot, key.with.dot", + "key-with-hyphen, key-with-hyphen", + "key_with_underscore, key_with_underscore", + "key@with@at, key@with@at", + "key[with]brackets, key[with]brackets" + }) + void getTrimmedKey_returns_expected_result(final String key, final String expectedTrimmedKey) { + assertThat(new JacksonEventKey(key).getTrimmedKey(), equalTo(expectedTrimmedKey)); + } + + @ParameterizedTest + @ArgumentsSource(KeyPathListArgumentsProvider.class) + void getKeyPathList_returns_expected_value(final String key, final List expectedKeyPathList) { + assertThat(new JacksonEventKey(key).getKeyPathList(), equalTo(expectedKeyPathList)); + } + + @Test + void getJsonPointer_returns_a_valid_JsonPointer() { + final String testKey = UUID.randomUUID().toString(); + final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey); + + final JsonPointer jsonPointer = objectUnderTest.getJsonPointer(); + assertThat(jsonPointer, notNullValue()); + assertThat(jsonPointer.toString(), equalTo("/" + testKey)); + } + + @Test + void getJsonPointer_returns_the_same_instance_for_multiple_calls() { + final String testKey = UUID.randomUUID().toString(); + final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey); + + final JsonPointer jsonPointer = objectUnderTest.getJsonPointer(); + assertThat(objectUnderTest.getJsonPointer(), sameInstance(jsonPointer)); + assertThat(objectUnderTest.getJsonPointer(), sameInstance(jsonPointer)); + } + + @ParameterizedTest + @ArgumentsSource(SupportsArgumentsProvider.class) + void supports_returns_true_if_any_supports(final List eventActionsList, final EventKeyFactory.EventAction otherAction, final boolean expectedSupports) { + final String testKey = UUID.randomUUID().toString(); + final EventKeyFactory.EventAction[] eventActions = new EventKeyFactory.EventAction[eventActionsList.size()]; + eventActionsList.toArray(eventActions); + assertThat(new JacksonEventKey(testKey, eventActions).supports(otherAction), equalTo(expectedSupports)); + } + + @ParameterizedTest + @CsvSource(value = { + "test_key, true", + "/test_key, true", + "inv(alid, false", + "getMetadata(\"test_key\"), false", + "key.with.dot, true", + "key-with-hyphen, true", + "key_with_underscore, true", + "key@with@at, true", + "key[with]brackets, true" + }) + void isValidEventKey_returns_expected_result(final String key, final boolean isValid) { + assertThat(JacksonEventKey.isValidEventKey(key), equalTo(isValid)); + } + + + static class KeyPathListArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext extensionContext) { + return Stream.of( + arguments("test_key", List.of("test_key")), + arguments("a/b", List.of("a", "b")), + arguments("a/b/", List.of("a", "b")), + arguments("a/b/c", List.of("a", "b", "c")), + arguments("a/b/c/", List.of("a", "b", "c")) + ); + } + } + + static class SupportsArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext extensionContext) throws Exception { + return Stream.of( + arguments(List.of(), EventKeyFactory.EventAction.GET, true), + arguments(List.of(), EventKeyFactory.EventAction.PUT, true), + arguments(List.of(), EventKeyFactory.EventAction.DELETE, true), + arguments(List.of(), EventKeyFactory.EventAction.ALL, true), + arguments(List.of(EventKeyFactory.EventAction.GET), EventKeyFactory.EventAction.GET, true), + arguments(List.of(EventKeyFactory.EventAction.PUT), EventKeyFactory.EventAction.PUT, true), + arguments(List.of(EventKeyFactory.EventAction.DELETE), EventKeyFactory.EventAction.DELETE, true), + arguments(List.of(EventKeyFactory.EventAction.GET), EventKeyFactory.EventAction.PUT, false), + arguments(List.of(EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.PUT), EventKeyFactory.EventAction.PUT, true), + arguments(List.of(EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.GET), EventKeyFactory.EventAction.PUT, true), + arguments(List.of(EventKeyFactory.EventAction.DELETE), EventKeyFactory.EventAction.PUT, false), + arguments(List.of(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.GET), EventKeyFactory.EventAction.PUT, false), + arguments(List.of(EventKeyFactory.EventAction.DELETE, EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.PUT), EventKeyFactory.EventAction.PUT, true), + arguments(List.of(EventKeyFactory.EventAction.ALL), EventKeyFactory.EventAction.GET, true), + arguments(List.of(EventKeyFactory.EventAction.ALL), EventKeyFactory.EventAction.PUT, true), + arguments(List.of(EventKeyFactory.EventAction.ALL), EventKeyFactory.EventAction.DELETE, true) + ); + } + } +} \ No newline at end of file diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java index 1a7efb7467..90645d2961 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventTest.java @@ -74,6 +74,53 @@ public void testPutAndGet_withRandomString() { assertThat(result, is(equalTo(value))); } + @Test + public void testPutAndGet_withRandomString_eventKey() { + final EventKey key = new JacksonEventKey("aRandomKey" + UUID.randomUUID()); + final UUID value = UUID.randomUUID(); + + event.put(key, value); + final UUID result = event.get(key, UUID.class); + + assertThat(result, is(notNullValue())); + assertThat(result, is(equalTo(value))); + } + + @Test + public void testPutAndGet_withRandomString_eventKey_multiple_events() { + final EventKey key = new JacksonEventKey("aRandomKey" + UUID.randomUUID()); + final UUID value = UUID.randomUUID(); + + for(int i = 0; i < 10; i++) { + event = JacksonEvent.builder() + .withEventType(eventType) + .build(); + + event.put(key, value); + final UUID result = event.get(key, UUID.class); + + assertThat(result, is(notNullValue())); + assertThat(result, is(equalTo(value))); + } + } + + @Test + public void testPutAndGet_eventKey_with_non_JacksonEventKey_throws() { + final EventKey key = mock(EventKey.class); + final UUID value = UUID.randomUUID(); + + assertThrows(IllegalArgumentException.class, () -> event.put(key, value)); + assertThrows(IllegalArgumentException.class, () -> event.get(key, UUID.class)); + } + + @Test + public void testPut_eventKey_with_immutable_action() { + final EventKey key = new JacksonEventKey("aRandomKey" + UUID.randomUUID(), EventKeyFactory.EventAction.GET); + final UUID value = UUID.randomUUID(); + + assertThrows(IllegalArgumentException.class, () -> event.put(key, value)); + } + @ParameterizedTest @ValueSource(strings = {"/", "foo", "foo-bar", "foo_bar", "foo.bar", "/foo", "/foo/", "a1K.k3-01_02", "keyWithBrackets[]"}) void testPutAndGet_withStrings(final String key) { @@ -86,6 +133,19 @@ void testPutAndGet_withStrings(final String key) { assertThat(result, is(equalTo(value))); } + @ParameterizedTest + @ValueSource(strings = {"/", "foo", "foo-bar", "foo_bar", "foo.bar", "/foo", "/foo/", "a1K.k3-01_02", "keyWithBrackets[]"}) + void testPutAndGet_withStrings_eventKey(final String key) { + final UUID value = UUID.randomUUID(); + + final EventKey eventKey = new JacksonEventKey(key); + event.put(eventKey, value); + final UUID result = event.get(eventKey, UUID.class); + + assertThat(result, is(notNullValue())); + assertThat(result, is(equalTo(value))); + } + @Test public void testPutKeyCannotBeEmptyString() { Throwable exception = assertThrows(IllegalArgumentException.class, () -> event.put("", "value")); @@ -93,7 +153,7 @@ public void testPutKeyCannotBeEmptyString() { } @Test - public void testPutAndGet_withMultLevelKey() { + public void testPutAndGet_withMultiLevelKey() { final String key = "foo/bar"; final UUID value = UUID.randomUUID(); @@ -104,6 +164,18 @@ public void testPutAndGet_withMultLevelKey() { assertThat(result, is(equalTo(value))); } + @Test + public void testPutAndGet_withMultiLevelKey_eventKey() { + final EventKey key = new JacksonEventKey("foo/bar"); + final UUID value = UUID.randomUUID(); + + event.put(key, value); + final UUID result = event.get(key, UUID.class); + + assertThat(result, is(notNullValue())); + assertThat(result, is(equalTo(value))); + } + @Test public void testPutAndGet_withMultiLevelKeyTwice() { final String key = "foo/bar"; @@ -125,6 +197,27 @@ public void testPutAndGet_withMultiLevelKeyTwice() { assertThat(result2, is(equalTo(value2))); } + @Test + public void testPutAndGet_withMultiLevelKeyTwice_eventKey() { + final EventKey key = new JacksonEventKey("foo/bar"); + final UUID value = UUID.randomUUID(); + + event.put(key, value); + final UUID result = event.get(key, UUID.class); + + assertThat(result, is(notNullValue())); + assertThat(result, is(equalTo(value))); + + final EventKey key2 = new JacksonEventKey("foo/fizz"); + final UUID value2 = UUID.randomUUID(); + + event.put(key2, value2); + final UUID result2 = event.get(key2, UUID.class); + + assertThat(result2, is(notNullValue())); + assertThat(result2, is(equalTo(value2))); + } + @Test public void testPutAndGet_withMultiLevelKeyWithADash() { final String key = "foo/bar-bar"; @@ -137,6 +230,18 @@ public void testPutAndGet_withMultiLevelKeyWithADash() { assertThat(result, is(equalTo(value))); } + @Test + public void testPutAndGet_withMultiLevelKeyWithADash_eventKey() { + final EventKey key = new JacksonEventKey("foo/bar-bar"); + final UUID value = UUID.randomUUID(); + + event.put(key, value); + final UUID result = event.get(key, UUID.class); + + assertThat(result, is(notNullValue())); + assertThat(result, is(equalTo(value))); + } + @ParameterizedTest @ValueSource(strings = {"foo", "/foo", "/foo/", "foo/"}) void testGetAtRootLevel(final String key) { @@ -148,6 +253,17 @@ void testGetAtRootLevel(final String key) { assertThat(result, is(Map.of("foo", value))); } + @ParameterizedTest + @ValueSource(strings = {"foo", "/foo", "/foo/", "foo/"}) + void testGetAtRootLevel_eventKey(final String key) { + final String value = UUID.randomUUID().toString(); + + event.put(new JacksonEventKey(key), value); + final Map result = event.get(new JacksonEventKey("", EventKeyFactory.EventAction.GET), Map.class); + + assertThat(result, is(Map.of("foo", value))); + } + @ParameterizedTest @ValueSource(strings = {"/foo/bar", "foo/bar", "foo/bar/"}) void testGetAtRootLevelWithMultiLevelKey(final String key) { @@ -159,6 +275,17 @@ void testGetAtRootLevelWithMultiLevelKey(final String key) { assertThat(result, is(Map.of("foo", Map.of("bar", value)))); } + @ParameterizedTest + @ValueSource(strings = {"/foo/bar", "foo/bar", "foo/bar/"}) + void testGetAtRootLevelWithMultiLevelKey_eventKey(final String key) { + final String value = UUID.randomUUID().toString(); + + event.put(new JacksonEventKey(key), value); + final Map result = event.get( new JacksonEventKey("", EventKeyFactory.EventAction.GET), Map.class); + + assertThat(result, is(Map.of("foo", Map.of("bar", value)))); + } + @Test public void testPutUpdateAndGet_withPojo() { final String key = "foo/bar"; @@ -293,6 +420,14 @@ public void testDeleteKey(final String key) { assertThat(result, is(nullValue())); } + @Test + public void testDelete_eventKey_with_immutable_action() { + final EventKey key = new JacksonEventKey("aRandomKey" + UUID.randomUUID(), EventKeyFactory.EventAction.GET); + final UUID value = UUID.randomUUID(); + + assertThrows(IllegalArgumentException.class, () -> event.delete(key)); + } + @Test public void testClear() { event.put("key1", UUID.randomUUID()); diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java new file mode 100644 index 0000000000..8aaf401e15 --- /dev/null +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java @@ -0,0 +1,118 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.integration; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.InMemorySinkAccessor; +import org.opensearch.dataprepper.plugins.InMemorySourceAccessor; +import org.opensearch.dataprepper.test.framework.DataPrepperTestRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.awaitility.Awaitility.await; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; + +class ProcessorPipelineIT { + private static final Logger LOG = LoggerFactory.getLogger(ProcessorPipelineIT.class); + private static final String IN_MEMORY_IDENTIFIER = "ProcessorPipelineIT"; + private static final String PIPELINE_CONFIGURATION_UNDER_TEST = "processor-pipeline.yaml"; + private DataPrepperTestRunner dataPrepperTestRunner; + private InMemorySourceAccessor inMemorySourceAccessor; + private InMemorySinkAccessor inMemorySinkAccessor; + + @BeforeEach + void setUp() { + dataPrepperTestRunner = DataPrepperTestRunner.builder() + .withPipelinesDirectoryOrFile(PIPELINE_CONFIGURATION_UNDER_TEST) + .build(); + + inMemorySourceAccessor = dataPrepperTestRunner.getInMemorySourceAccessor(); + inMemorySinkAccessor = dataPrepperTestRunner.getInMemorySinkAccessor(); + dataPrepperTestRunner.start(); + LOG.info("Started test runner."); + } + + @AfterEach + void tearDown() { + LOG.info("Test tear down. Stop the test runner."); + dataPrepperTestRunner.stop(); + } + + @Test + void run_with_single_record() { + final String messageValue = UUID.randomUUID().toString(); + final Event event = JacksonEvent.fromMessage(messageValue); + final Record eventRecord = new Record<>(event); + + LOG.info("Submitting a single record."); + inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, Collections.singletonList(eventRecord)); + + await().atMost(400, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER), not(empty())); + }); + + final List> records = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); + + assertThat(records.size(), equalTo(1)); + + assertThat(records.get(0), notNullValue()); + assertThat(records.get(0).getData(), notNullValue()); + assertThat(records.get(0).getData().get("message", String.class), equalTo(messageValue)); + assertThat(records.get(0).getData().get("test1", String.class), equalTo("knownPrefix10")); + } + + @Test + void pipeline_with_single_batch_of_records() { + final int recordsToCreate = 200; + final List> inputRecords = IntStream.range(0, recordsToCreate) + .mapToObj(i -> UUID.randomUUID().toString()) + .map(JacksonEvent::fromMessage) + .map(Record::new) + .collect(Collectors.toList()); + + LOG.info("Submitting a batch of record."); + inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, inputRecords); + + await().atMost(400, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER), not(empty())); + }); + + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER).size(), equalTo(recordsToCreate)); + + final List> sinkRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); + + for (int i = 0; i < sinkRecords.size(); i++) { + final Record inputRecord = inputRecords.get(i); + final Record sinkRecord = sinkRecords.get(i); + assertThat(sinkRecord, notNullValue()); + final Event recordData = sinkRecord.getData(); + assertThat(recordData, notNullValue()); + assertThat( + recordData.get("message", String.class), + equalTo(inputRecord.getData().get("message", String.class))); + assertThat(recordData.get("test1", String.class), + equalTo("knownPrefix1" + i)); + } + } +} diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySourceAccessor.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySourceAccessor.java index 71151be22e..3957d259a9 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySourceAccessor.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySourceAccessor.java @@ -6,20 +6,19 @@ package org.opensearch.dataprepper.plugins; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.record.Record; import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.UUID; import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.locks.Lock; import java.util.concurrent.locks.ReentrantLock; -import java.util.concurrent.atomic.AtomicBoolean; /** * Provides a mechanism to write records to an in_memory source. This allows the pipeline to execute @@ -62,8 +61,8 @@ public void submit(final String testingKey, int numRecords) { for (int i = 0; i < numRecords; i++) { Map eventMap = Map.of("message", UUID.randomUUID().toString()); EventBuilder eventBuilder = (EventBuilder) eventFactory.eventBuilder(EventBuilder.class).withData(eventMap); - JacksonEvent event = (JacksonEvent) eventBuilder.build(); - records.add(new Record(event)); + Event event = eventBuilder.build(); + records.add(new Record<>(event)); } submit(testingKey, records); } @@ -79,8 +78,8 @@ public void submitWithStatus(final String testingKey, int numRecords) { int status = (int)(Math.random() * (max - min + 1) + min); Map eventMap = Map.of("message", UUID.randomUUID().toString(), "status", status); EventBuilder eventBuilder = (EventBuilder) eventFactory.eventBuilder(EventBuilder.class).withData(eventMap); - JacksonEvent event = (JacksonEvent) eventBuilder.build(); - records.add(new Record(event)); + Event event = eventBuilder.build(); + records.add(new Record<>(event)); } submit(testingKey, records); } diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java new file mode 100644 index 0000000000..bc59deb138 --- /dev/null +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java @@ -0,0 +1,56 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins; + +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.processor.Processor; +import org.opensearch.dataprepper.model.record.Record; + +import java.util.Collection; + +@DataPrepperPlugin(name = "simple_test", pluginType = Processor.class, pluginConfigurationType = SimpleProcessorConfig.class) +public class SimpleProcessor implements Processor, Record> { + private final EventKey eventKey1; + private final String valuePrefix1; + int count = 0; + + @DataPrepperPluginConstructor + public SimpleProcessor( + final SimpleProcessorConfig simpleProcessorConfig, + final EventKeyFactory eventKeyFactory) { + eventKey1 = eventKeyFactory.createEventKey(simpleProcessorConfig.getKey1()); + valuePrefix1 = simpleProcessorConfig.getValuePrefix1(); + } + + @Override + public Collection> execute(final Collection> records) { + for (final Record record : records) { + record.getData().put(eventKey1, valuePrefix1 + count); + count++; + } + + return records; + } + + @Override + public void prepareForShutdown() { + + } + + @Override + public boolean isReadyForShutdown() { + return false; + } + + @Override + public void shutdown() { + + } +} diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java new file mode 100644 index 0000000000..957202431f --- /dev/null +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java @@ -0,0 +1,19 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins; + +public class SimpleProcessorConfig { + private String key1; + private String valuePrefix1; + + public String getKey1() { + return key1; + } + + public String getValuePrefix1() { + return valuePrefix1; + } +} diff --git a/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml new file mode 100644 index 0000000000..2223a07c3e --- /dev/null +++ b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml @@ -0,0 +1,14 @@ +processor-pipeline: + delay: 10 + source: + in_memory: + testing_key: ProcessorPipelineIT + + processor: + - simple_test: + key1: /test1 + value_prefix1: knownPrefix1 + + sink: + - in_memory: + testing_key: ProcessorPipelineIT diff --git a/data-prepper-event/src/main/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactory.java b/data-prepper-event/src/main/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactory.java new file mode 100644 index 0000000000..605b5bcb41 --- /dev/null +++ b/data-prepper-event/src/main/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactory.java @@ -0,0 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.core.event; + +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.event.InternalOnlyEventKeyBridge; + +import javax.inject.Named; + +@Named +public class DefaultEventKeyFactory implements EventKeyFactory { + @Override + public EventKey createEventKey(final String key, final EventAction... forActions) { + return InternalOnlyEventKeyBridge.createEventKey(key, forActions); + } +} diff --git a/data-prepper-event/src/main/java/org/opensearch/dataprepper/model/event/InternalOnlyEventKeyBridge.java b/data-prepper-event/src/main/java/org/opensearch/dataprepper/model/event/InternalOnlyEventKeyBridge.java new file mode 100644 index 0000000000..130b94db0e --- /dev/null +++ b/data-prepper-event/src/main/java/org/opensearch/dataprepper/model/event/InternalOnlyEventKeyBridge.java @@ -0,0 +1,17 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +/** + * Until we remove {@link JacksonEvent} from data-prepper-api, + * we will need this class to give us access to the package-protected + * {@link JacksonEventKey}. + */ +public class InternalOnlyEventKeyBridge { + public static EventKey createEventKey(final String key, final EventKeyFactory.EventAction... forAction) { + return new JacksonEventKey(key, forAction); + } +} diff --git a/data-prepper-event/src/test/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactoryTest.java b/data-prepper-event/src/test/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactoryTest.java new file mode 100644 index 0000000000..8d034fcc83 --- /dev/null +++ b/data-prepper-event/src/test/java/org/opensearch/dataprepper/core/event/DefaultEventKeyFactoryTest.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.core.event; + +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +class DefaultEventKeyFactoryTest { + + private DefaultEventKeyFactory createObjectUnderTest() { + return new DefaultEventKeyFactory(); + } + + @Test + void createEventKey_returns_correct_EventKey() { + final String keyPath = UUID.randomUUID().toString(); + final EventKey eventKey = createObjectUnderTest().createEventKey(keyPath); + + assertThat(eventKey, notNullValue()); + assertThat(eventKey.getKey(), equalTo(keyPath)); + } + + @Test + void createEventKey_with_EventAction_returns_correct_EventKey() { + final String keyPath = UUID.randomUUID().toString(); + final EventKey eventKey = createObjectUnderTest().createEventKey(keyPath, EventKeyFactory.EventAction.GET); + + assertThat(eventKey, notNullValue()); + assertThat(eventKey.getKey(), equalTo(keyPath)); + } + + @Test + void createEventKey_returns_JacksonEventKey() { + final String keyPath = UUID.randomUUID().toString(); + final EventKey eventKey = createObjectUnderTest().createEventKey(keyPath); + + assertThat(eventKey, notNullValue()); + assertThat(eventKey.getClass().getSimpleName(), equalTo("JacksonEventKey")); + + assertThat(eventKey.getKey(), equalTo(keyPath)); + } +} \ No newline at end of file diff --git a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliers.java b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliers.java index f5ceebbde6..f9e1abddb7 100644 --- a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliers.java +++ b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliers.java @@ -8,6 +8,7 @@ import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.breaker.CircuitBreaker; import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.springframework.beans.factory.annotation.Autowired; import javax.inject.Inject; @@ -31,6 +32,7 @@ class ApplicationContextToTypedSuppliers { @Inject ApplicationContextToTypedSuppliers( final EventFactory eventFactory, + final EventKeyFactory eventKeyFactory, final AcknowledgementSetManager acknowledgementSetManager, @Autowired(required = false) final CircuitBreaker circuitBreaker ) { @@ -39,6 +41,7 @@ class ApplicationContextToTypedSuppliers { typedSuppliers = Map.of( EventFactory.class, () -> eventFactory, + EventKeyFactory.class, () -> eventKeyFactory, AcknowledgementSetManager.class, () -> acknowledgementSetManager, CircuitBreaker.class, () -> circuitBreaker ); diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliersTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliersTest.java index 0cd008559a..a12540a46a 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliersTest.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ApplicationContextToTypedSuppliersTest.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.breaker.CircuitBreaker; import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import java.util.Map; import java.util.function.Supplier; @@ -28,6 +29,9 @@ class ApplicationContextToTypedSuppliersTest { @Mock private EventFactory eventFactory; + @Mock + private EventKeyFactory eventKeyFactory; + @Mock private AcknowledgementSetManager acknowledgementSetManager; @@ -37,6 +41,7 @@ class ApplicationContextToTypedSuppliersTest { private ApplicationContextToTypedSuppliers createObjectUnderTest() { return new ApplicationContextToTypedSuppliers( eventFactory, + eventKeyFactory, acknowledgementSetManager, circuitBreaker ); @@ -58,12 +63,16 @@ void constructor_throws_with_null_AcknowledgementSetManager() { void getArgumentsSuppliers_returns_map_with_expected_classes() { final Map, Supplier> argumentsSuppliers = createObjectUnderTest().getArgumentsSuppliers(); - assertThat(argumentsSuppliers.size(), equalTo(3)); + assertThat(argumentsSuppliers.size(), equalTo(4)); assertThat(argumentsSuppliers, hasKey(EventFactory.class)); assertThat(argumentsSuppliers.get(EventFactory.class), notNullValue()); assertThat(argumentsSuppliers.get(EventFactory.class).get(), equalTo(eventFactory)); + assertThat(argumentsSuppliers, hasKey(EventKeyFactory.class)); + assertThat(argumentsSuppliers.get(EventKeyFactory.class), notNullValue()); + assertThat(argumentsSuppliers.get(EventKeyFactory.class).get(), equalTo(eventKeyFactory)); + assertThat(argumentsSuppliers, hasKey(AcknowledgementSetManager.class)); assertThat(argumentsSuppliers.get(AcknowledgementSetManager.class), notNullValue()); assertThat(argumentsSuppliers.get(AcknowledgementSetManager.class).get(), equalTo(acknowledgementSetManager)); @@ -79,12 +88,16 @@ void getArgumentsSuppliers_returns_map_with_null_optional_CircuitBreaker() { final Map, Supplier> argumentsSuppliers = createObjectUnderTest().getArgumentsSuppliers(); - assertThat(argumentsSuppliers.size(), equalTo(3)); + assertThat(argumentsSuppliers.size(), equalTo(4)); assertThat(argumentsSuppliers, hasKey(EventFactory.class)); assertThat(argumentsSuppliers.get(EventFactory.class), notNullValue()); assertThat(argumentsSuppliers.get(EventFactory.class).get(), equalTo(eventFactory)); + assertThat(argumentsSuppliers, hasKey(EventKeyFactory.class)); + assertThat(argumentsSuppliers.get(EventKeyFactory.class), notNullValue()); + assertThat(argumentsSuppliers.get(EventKeyFactory.class).get(), equalTo(eventKeyFactory)); + assertThat(argumentsSuppliers, hasKey(AcknowledgementSetManager.class)); assertThat(argumentsSuppliers.get(AcknowledgementSetManager.class), notNullValue()); assertThat(argumentsSuppliers.get(AcknowledgementSetManager.class).get(), equalTo(acknowledgementSetManager)); diff --git a/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventContext.java b/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventContext.java new file mode 100644 index 0000000000..6c5b001129 --- /dev/null +++ b/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventContext.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.event; + +import org.opensearch.dataprepper.core.event.EventFactoryApplicationContextMarker; +import org.springframework.context.annotation.AnnotationConfigApplicationContext; + +class TestEventContext { + private static AnnotationConfigApplicationContext APPLICATION_CONTEXT; + + private TestEventContext() {} + + static T getFromContext(final Class targetClass) { + if(APPLICATION_CONTEXT == null) { + APPLICATION_CONTEXT = new AnnotationConfigApplicationContext(); + APPLICATION_CONTEXT.scan(EventFactoryApplicationContextMarker.class.getPackageName()); + APPLICATION_CONTEXT.refresh(); + } + return APPLICATION_CONTEXT.getBean(targetClass); + } +} diff --git a/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventFactory.java b/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventFactory.java index 932c9ca66a..08a2cd2f29 100644 --- a/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventFactory.java +++ b/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventFactory.java @@ -5,18 +5,15 @@ package org.opensearch.dataprepper.event; -import org.opensearch.dataprepper.core.event.EventFactoryApplicationContextMarker; import org.opensearch.dataprepper.model.event.BaseEventBuilder; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventFactory; -import org.springframework.context.annotation.AnnotationConfigApplicationContext; /** * An implementation of {@link EventFactory} that is useful for integration and unit tests * in other projects. */ public class TestEventFactory implements EventFactory { - private static AnnotationConfigApplicationContext APPLICATION_CONTEXT; private static EventFactory DEFAULT_EVENT_FACTORY; private final EventFactory innerEventFactory; @@ -25,11 +22,8 @@ public class TestEventFactory implements EventFactory { } public static EventFactory getTestEventFactory() { - if(APPLICATION_CONTEXT == null) { - APPLICATION_CONTEXT = new AnnotationConfigApplicationContext(); - APPLICATION_CONTEXT.scan(EventFactoryApplicationContextMarker.class.getPackageName()); - APPLICATION_CONTEXT.refresh(); - DEFAULT_EVENT_FACTORY = APPLICATION_CONTEXT.getBean(EventFactory.class); + if(DEFAULT_EVENT_FACTORY == null) { + DEFAULT_EVENT_FACTORY = TestEventContext.getFromContext(EventFactory.class); } return new TestEventFactory(DEFAULT_EVENT_FACTORY); } diff --git a/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventKeyFactory.java b/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventKeyFactory.java new file mode 100644 index 0000000000..0cec742924 --- /dev/null +++ b/data-prepper-test-event/src/main/java/org/opensearch/dataprepper/event/TestEventKeyFactory.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.event; + +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +public class TestEventKeyFactory implements EventKeyFactory { + private static EventKeyFactory DEFAULT_EVENT_KEY_FACTORY; + private final EventKeyFactory innerEventKeyFactory; + + TestEventKeyFactory(final EventKeyFactory innerEventKeyFactory) { + this.innerEventKeyFactory = innerEventKeyFactory; + } + + public static EventKeyFactory getTestEventFactory() { + if(DEFAULT_EVENT_KEY_FACTORY == null) { + DEFAULT_EVENT_KEY_FACTORY = TestEventContext.getFromContext(EventKeyFactory.class); + } + return new TestEventKeyFactory(DEFAULT_EVENT_KEY_FACTORY); + } + + @Override + public EventKey createEventKey(final String key, final EventAction... forActions) { + return innerEventKeyFactory.createEventKey(key, forActions); + } +} diff --git a/data-prepper-test-event/src/test/java/org/opensearch/dataprepper/event/TestEventKeyFactoryTest.java b/data-prepper-test-event/src/test/java/org/opensearch/dataprepper/event/TestEventKeyFactoryTest.java new file mode 100644 index 0000000000..65b17819b8 --- /dev/null +++ b/data-prepper-test-event/src/test/java/org/opensearch/dataprepper/event/TestEventKeyFactoryTest.java @@ -0,0 +1,56 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.event; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class TestEventKeyFactoryTest { + + @Mock + private EventKeyFactory innerEventKeyFactory; + + @Mock + private EventKey eventKey; + + private TestEventKeyFactory createObjectUnderTest() { + return new TestEventKeyFactory(innerEventKeyFactory); + } + + @Test + void createEventKey_returns_from_inner_EventKeyFactory() { + final String keyPath = UUID.randomUUID().toString(); + when(innerEventKeyFactory.createEventKey(keyPath, EventKeyFactory.EventAction.ALL)) + .thenReturn(eventKey); + + assertThat(createObjectUnderTest().createEventKey(keyPath), + equalTo(eventKey)); + } + + @ParameterizedTest + @EnumSource(EventKeyFactory.EventAction.class) + void createEventKey_with_Actions_returns_from_inner_EventKeyFactory(final EventKeyFactory.EventAction eventAction) { + final String keyPath = UUID.randomUUID().toString(); + when(innerEventKeyFactory.createEventKey(keyPath, eventAction)) + .thenReturn(eventKey); + + assertThat(createObjectUnderTest().createEventKey(keyPath, eventAction), + equalTo(eventKey)); + } +} \ No newline at end of file From a85e05e60f9fd2d2a269aaf2c7d5291c59946b87 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Mon, 17 Jun 2024 12:34:33 -0700 Subject: [PATCH 021/159] Add Aggregate event handle (#4625) Aggregate event handle Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../AcknowledgementSetManager.java | 31 ------ .../model/event/AbstractEventHandle.java | 52 +++++++++ .../model/event/AggregateEventHandle.java | 77 +++++++++++++ .../model/event/DefaultEventHandle.java | 49 +++------ .../dataprepper/model/event/EventHandle.java | 3 +- .../model/event/InternalEventHandle.java | 21 ++-- .../event/AggregateEventHandleTests.java | 102 ++++++++++++++++++ .../model/event/DefaultEventHandleTests.java | 9 +- .../JacksonEvent_JavaSerializationTest.java | 10 +- .../dataprepper/plugins/InMemorySink.java | 2 +- .../AcknowledgementSetMonitor.java | 63 +---------- .../DefaultAcknowledgementSet.java | 2 +- .../DefaultAcknowledgementSetManager.java | 14 --- .../InactiveAcknowledgementSetManager.java | 13 --- .../dataprepper/pipeline/ProcessWorker.java | 2 +- .../router/RouterCopyRecordStrategy.java | 5 +- .../AcknowledgementSetMonitorTests.java | 54 ---------- ...DefaultAcknowledgementSetManagerTests.java | 86 ++++++++++++--- .../DefaultAcknowledgementSetTests.java | 10 +- ...nactiveAcknowledgementSetManagerTests.java | 24 ----- .../codec/JavaPeerForwarderCodecTest.java | 4 +- .../dataprepper/pipeline/PipelineTests.java | 2 +- .../pipeline/ProcessWorkerTest.java | 11 +- .../router/RouterCopyRecordStrategyTests.java | 9 +- .../plugins/sink/s3/S3SinkServiceTest.java | 4 +- .../splitevent/SplitEventProcessorTest.java | 2 +- 26 files changed, 375 insertions(+), 286 deletions(-) create mode 100644 data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AbstractEventHandle.java create mode 100644 data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AggregateEventHandle.java create mode 100644 data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/AggregateEventHandleTests.java diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/acknowledgements/AcknowledgementSetManager.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/acknowledgements/AcknowledgementSetManager.java index 69c07c4aa5..6afebeaa91 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/acknowledgements/AcknowledgementSetManager.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/acknowledgements/AcknowledgementSetManager.java @@ -5,9 +5,6 @@ package org.opensearch.dataprepper.model.acknowledgements; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.EventHandle; - import java.time.Duration; import java.util.function.Consumer; @@ -29,32 +26,4 @@ public interface AcknowledgementSetManager { * @since 2.2 */ AcknowledgementSet create(final Consumer callback, final Duration timeout); - - /** - * Releases an event's reference - * - * @param eventHandle event handle - * @param success indicates negative or positive acknowledgement - * - * @since 2.2 - */ - void releaseEventReference(final EventHandle eventHandle, boolean success); - - /** - * Acquires an event's reference - * - * @param eventHandle event handle - * - * @since 2.2 - */ - void acquireEventReference(final EventHandle eventHandle); - - /** - * Acquires an event's reference - * - * @param event event - * - * @since 2.2 - */ - void acquireEventReference(final Event event); } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AbstractEventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AbstractEventHandle.java new file mode 100644 index 0000000000..2ca40fbe59 --- /dev/null +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AbstractEventHandle.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +import java.util.ArrayList; +import java.util.List; +import java.time.Instant; +import java.util.function.BiConsumer; + +abstract class AbstractEventHandle implements EventHandle, InternalEventHandle { + private Instant externalOriginationTime; + private final Instant internalOriginationTime; + private List> releaseConsumers; + + AbstractEventHandle(final Instant internalOriginationTime) { + this.externalOriginationTime = null; + this.internalOriginationTime = internalOriginationTime; + this.releaseConsumers = new ArrayList<>(); + } + @Override + public void setExternalOriginationTime(final Instant externalOriginationTime) { + this.externalOriginationTime = externalOriginationTime; + } + + @Override + public Instant getInternalOriginationTime() { + return this.internalOriginationTime; + } + + @Override + public Instant getExternalOriginationTime() { + return this.externalOriginationTime; + } + + @Override + public void onRelease(BiConsumer releaseConsumer) { + synchronized (releaseConsumers) { + releaseConsumers.add(releaseConsumer); + } + } + + public void notifyReleaseConsumers(boolean result) { + synchronized (releaseConsumers) { + for (final BiConsumer consumer: releaseConsumers) { + consumer.accept(this, result); + } + } + } +} diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AggregateEventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AggregateEventHandle.java new file mode 100644 index 0000000000..921d689a3c --- /dev/null +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/AggregateEventHandle.java @@ -0,0 +1,77 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import java.lang.ref.WeakReference; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.time.Instant; +import java.io.Serializable; + +public class AggregateEventHandle extends AbstractEventHandle implements Serializable { + private List> acknowledgementSetRefList; + private Set acknowledgementSetHashes; + + public AggregateEventHandle(final Instant internalOriginationTime) { + super(internalOriginationTime); + this.acknowledgementSetRefList = new ArrayList<>(); + this.acknowledgementSetHashes = new HashSet<>(); + } + + @Override + public void addAcknowledgementSet(final AcknowledgementSet acknowledgementSet) { + int hashCode = acknowledgementSet.hashCode(); + if (!acknowledgementSetHashes.contains(hashCode)) { + this.acknowledgementSetRefList.add(new WeakReference<>(acknowledgementSet)); + acknowledgementSetHashes.add(hashCode); + } + } + + @Override + public boolean hasAcknowledgementSet() { + return acknowledgementSetRefList.size() != 0; + } + + @Override + public void acquireReference() { + synchronized (this) { + for (WeakReference acknowledgementSetRef: acknowledgementSetRefList) {; + AcknowledgementSet acknowledgementSet = acknowledgementSetRef.get(); + if (acknowledgementSet != null) { + acknowledgementSet.acquire(this); + } + } + } + } + + @Override + public boolean release(boolean result) { + notifyReleaseConsumers(result); + boolean returnValue = true; + synchronized (this) { + for (WeakReference acknowledgementSetRef: acknowledgementSetRefList) { + AcknowledgementSet acknowledgementSet = acknowledgementSetRef.get(); + if (acknowledgementSet != null) { + acknowledgementSet.release(this, result); + } else { + returnValue = false; + } + } + } + return returnValue; + } + + // For testing + List> getAcknowledgementSetRefs() { + return acknowledgementSetRefList; + } + +} + diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventHandle.java index 743309bf75..340c104a14 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventHandle.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/DefaultEventHandle.java @@ -8,35 +8,22 @@ import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import java.lang.ref.WeakReference; -import java.util.ArrayList; -import java.util.List; -import java.util.function.BiConsumer; import java.time.Instant; import java.io.Serializable; -public class DefaultEventHandle implements EventHandle, InternalEventHandle, Serializable { - private Instant externalOriginationTime; - private final Instant internalOriginationTime; +public class DefaultEventHandle extends AbstractEventHandle implements Serializable { private WeakReference acknowledgementSetRef; - private List> releaseConsumers; public DefaultEventHandle(final Instant internalOriginationTime) { + super(internalOriginationTime); this.acknowledgementSetRef = null; - this.externalOriginationTime = null; - this.internalOriginationTime = internalOriginationTime; - this.releaseConsumers = new ArrayList<>(); } @Override - public void setAcknowledgementSet(final AcknowledgementSet acknowledgementSet) { + public void addAcknowledgementSet(final AcknowledgementSet acknowledgementSet) { this.acknowledgementSetRef = new WeakReference<>(acknowledgementSet); } - @Override - public void setExternalOriginationTime(final Instant externalOriginationTime) { - this.externalOriginationTime = externalOriginationTime; - } - public AcknowledgementSet getAcknowledgementSet() { if (acknowledgementSetRef == null) { return null; @@ -45,32 +32,30 @@ public AcknowledgementSet getAcknowledgementSet() { } @Override - public Instant getInternalOriginationTime() { - return this.internalOriginationTime; + public boolean hasAcknowledgementSet() { + AcknowledgementSet acknowledgementSet = getAcknowledgementSet(); + return acknowledgementSet != null; } @Override - public Instant getExternalOriginationTime() { - return this.externalOriginationTime; + public void acquireReference() { + synchronized (this) { + AcknowledgementSet acknowledgementSet = getAcknowledgementSet(); + if (acknowledgementSet != null) { + acknowledgementSet.acquire(this); + } + } } @Override - public void release(boolean result) { - synchronized (releaseConsumers) { - for (final BiConsumer consumer: releaseConsumers) { - consumer.accept(this, result); - } - } + public boolean release(boolean result) { + notifyReleaseConsumers(result); AcknowledgementSet acknowledgementSet = getAcknowledgementSet(); if (acknowledgementSet != null) { acknowledgementSet.release(this, result); + return true; } + return false; } - @Override - public void onRelease(BiConsumer releaseConsumer) { - synchronized (releaseConsumers) { - releaseConsumers.add(releaseConsumer); - } - } } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventHandle.java index d05dd8e36c..898384c32e 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventHandle.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventHandle.java @@ -14,9 +14,10 @@ public interface EventHandle { * * @param result result to be used while releasing. This indicates if * the operation on the event handle is success or not + * @return returns true if the event handle is released successful, false otherwise * @since 2.2 */ - void release(boolean result); + boolean release(boolean result); /** * sets external origination time diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/InternalEventHandle.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/InternalEventHandle.java index 3817365f17..3ee88f698b 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/InternalEventHandle.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/InternalEventHandle.java @@ -9,20 +9,27 @@ public interface InternalEventHandle { /** - * sets acknowledgement set + * adds acknowledgement set * * @param acknowledgementSet acknowledgementSet to be set in the event handle - * @since 2.6 + * @since 2.9 */ - void setAcknowledgementSet(final AcknowledgementSet acknowledgementSet); + void addAcknowledgementSet(final AcknowledgementSet acknowledgementSet); /** - * gets acknowledgement set + * Indicates if the event handle has atleast one acknowledgement set * - * @return returns acknowledgementSet from the event handle - * @since 2.6 + * @return returns true if there is at least one acknowledgementSet in the event handle + * @since 2.9 */ - AcknowledgementSet getAcknowledgementSet(); + boolean hasAcknowledgementSet(); + + /** + * Acquires reference to acknowledgement set(s) in the event handle + * + * @since 2.9 + */ + void acquireReference(); } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/AggregateEventHandleTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/AggregateEventHandleTests.java new file mode 100644 index 0000000000..9998d6eb6d --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/AggregateEventHandleTests.java @@ -0,0 +1,102 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import org.junit.jupiter.api.Test; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.verify; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.times; +import org.mockito.Mock; + +import java.lang.ref.WeakReference; +import java.time.Instant; + +class AggregateEventHandleTests { + @Mock + private AcknowledgementSet acknowledgementSet1; + @Mock + private AcknowledgementSet acknowledgementSet2; + private int count; + + @Test + void testBasic() { + Instant now = Instant.now(); + AggregateEventHandle eventHandle = new AggregateEventHandle(now); + assertThat(eventHandle.getInternalOriginationTime(), equalTo(now)); + assertThat(eventHandle.getExternalOriginationTime(), equalTo(null)); + assertThat(eventHandle.hasAcknowledgementSet(), equalTo(false)); + eventHandle.acquireReference(); + eventHandle.release(true); + } + + @Test + void testWithAcknowledgementSet() { + acknowledgementSet1 = mock(AcknowledgementSet.class); + acknowledgementSet2 = mock(AcknowledgementSet.class); + when(acknowledgementSet1.release(any(EventHandle.class), any(Boolean.class))).thenReturn(true); + when(acknowledgementSet2.release(any(EventHandle.class), any(Boolean.class))).thenReturn(true); + Instant now = Instant.now(); + AggregateEventHandle eventHandle = new AggregateEventHandle(now); + assertThat(eventHandle.getInternalOriginationTime(), equalTo(now)); + assertThat(eventHandle.getExternalOriginationTime(), equalTo(null)); + eventHandle.addAcknowledgementSet(acknowledgementSet1); + // just do duplicate add + eventHandle.addAcknowledgementSet(acknowledgementSet1); + assertThat(eventHandle.hasAcknowledgementSet(), equalTo(true)); + eventHandle.addAcknowledgementSet(acknowledgementSet2); + eventHandle.acquireReference(); + verify(acknowledgementSet1).acquire(eventHandle); + verify(acknowledgementSet2).acquire(eventHandle); + eventHandle.release(true); + verify(acknowledgementSet1).release(eventHandle, true); + verify(acknowledgementSet2).release(eventHandle, true); + } + + @Test + void testWithExternalOriginationTime() { + Instant now = Instant.now(); + AggregateEventHandle eventHandle = new AggregateEventHandle(now); + assertThat(eventHandle.hasAcknowledgementSet(), equalTo(false)); + assertThat(eventHandle.getInternalOriginationTime(), equalTo(now)); + assertThat(eventHandle.getExternalOriginationTime(), equalTo(null)); + eventHandle.setExternalOriginationTime(now.minusSeconds(60)); + assertThat(eventHandle.getExternalOriginationTime(), equalTo(now.minusSeconds(60))); + eventHandle.release(true); + } + + @Test + void testWithOnReleaseHandler() { + Instant now = Instant.now(); + count = 0; + AggregateEventHandle eventHandle = new AggregateEventHandle(now); + acknowledgementSet1 = mock(AcknowledgementSet.class); + acknowledgementSet2 = mock(AcknowledgementSet.class); + eventHandle.onRelease((handle, result) -> {if (result) count++; }); + eventHandle.addAcknowledgementSet(acknowledgementSet1); + assertThat(eventHandle.hasAcknowledgementSet(), equalTo(true)); + eventHandle.addAcknowledgementSet(acknowledgementSet2); + // Simulate weak reference object not available for + // verification tests to pass 100% + for (WeakReference acknowledgementSetRef: eventHandle.getAcknowledgementSetRefs()) { + if (acknowledgementSetRef.get() == acknowledgementSet2 ) { + acknowledgementSetRef.clear(); + break; + } + } + eventHandle.release(true); + assertThat(count, equalTo(1)); + verify(acknowledgementSet1, times(1)).release(eventHandle, true); + verify(acknowledgementSet2, times(0)).release(eventHandle, true); + + } + +} + diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventHandleTests.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventHandleTests.java index b2a66b2d1d..f351febd11 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventHandleTests.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/DefaultEventHandleTests.java @@ -13,6 +13,7 @@ import static org.mockito.Mockito.when; import static org.mockito.Mockito.verify; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doNothing; import org.mockito.Mock; import java.time.Instant; @@ -29,6 +30,8 @@ void testBasic() { assertThat(eventHandle.getAcknowledgementSet(), equalTo(null)); assertThat(eventHandle.getInternalOriginationTime(), equalTo(now)); assertThat(eventHandle.getExternalOriginationTime(), equalTo(null)); + eventHandle.acquireReference(); + assertThat(eventHandle.hasAcknowledgementSet(), equalTo(false)); eventHandle.release(true); } @@ -36,12 +39,16 @@ void testBasic() { void testWithAcknowledgementSet() { acknowledgementSet = mock(AcknowledgementSet.class); when(acknowledgementSet.release(any(EventHandle.class), any(Boolean.class))).thenReturn(true); + doNothing().when(acknowledgementSet).acquire(any(EventHandle.class)); Instant now = Instant.now(); DefaultEventHandle eventHandle = new DefaultEventHandle(now); assertThat(eventHandle.getAcknowledgementSet(), equalTo(null)); assertThat(eventHandle.getInternalOriginationTime(), equalTo(now)); assertThat(eventHandle.getExternalOriginationTime(), equalTo(null)); - eventHandle.setAcknowledgementSet(acknowledgementSet); + eventHandle.addAcknowledgementSet(acknowledgementSet); + assertThat(eventHandle.hasAcknowledgementSet(), equalTo(true)); + eventHandle.acquireReference(); + verify(acknowledgementSet).acquire(eventHandle); eventHandle.release(true); verify(acknowledgementSet).release(eventHandle, true); } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEvent_JavaSerializationTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEvent_JavaSerializationTest.java index b3ee46b55c..160f08d673 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEvent_JavaSerializationTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEvent_JavaSerializationTest.java @@ -8,6 +8,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import static org.junit.jupiter.api.Assertions.assertFalse; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -20,7 +21,6 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.nullValue; import static org.mockito.Mockito.mock; class JacksonEvent_JavaSerializationTest { @@ -54,7 +54,7 @@ void serialize_without_acknowledgementSet_includes_data() throws IOException, Cl assertThat(deserializedEvent.getMetadata(), equalTo(objectUnderTest.getMetadata())); assertThat(deserializedEvent.getEventHandle(), instanceOf(InternalEventHandle.class)); - assertThat(((InternalEventHandle) deserializedEvent.getEventHandle()).getAcknowledgementSet(), nullValue()); + assertFalse(((InternalEventHandle) deserializedEvent.getEventHandle()).hasAcknowledgementSet()); assertThat(deserializedEvent.getEventHandle().getInternalOriginationTime(), equalTo(objectUnderTest.getMetadata().getTimeReceived())); } @@ -63,7 +63,7 @@ void serialize_without_acknowledgementSet_includes_data() throws IOException, Cl void serialize_with_acknowledgementSet_does_not_include_old_acknowledgement_set() throws IOException, ClassNotFoundException { final JacksonEvent objectUnderTest = createObjectUnderTest(); final InternalEventHandle internalEventHandle = (InternalEventHandle) objectUnderTest.getEventHandle(); - internalEventHandle.setAcknowledgementSet(mock(AcknowledgementSet.class)); + internalEventHandle.addAcknowledgementSet(mock(AcknowledgementSet.class)); final Object deserializedObject = serializeAndDeserialize(objectUnderTest); @@ -74,7 +74,7 @@ void serialize_with_acknowledgementSet_does_not_include_old_acknowledgement_set( assertThat(deserializedEvent.getMetadata(), equalTo(objectUnderTest.getMetadata())); assertThat(deserializedEvent.getEventHandle(), instanceOf(InternalEventHandle.class)); - assertThat(((InternalEventHandle) deserializedEvent.getEventHandle()).getAcknowledgementSet(), nullValue()); + assertFalse(((InternalEventHandle) deserializedEvent.getEventHandle()).hasAcknowledgementSet()); assertThat(deserializedEvent.getEventHandle().getInternalOriginationTime(), equalTo(objectUnderTest.getMetadata().getTimeReceived())); } @@ -84,4 +84,4 @@ private Object serializeAndDeserialize(final JacksonEvent objectUnderTest) throw return objectInputStream.readObject(); } -} \ No newline at end of file +} diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySink.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySink.java index dec7aa5c1f..360367a1e4 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySink.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySink.java @@ -40,7 +40,7 @@ public void output(final Collection> records) { records.stream().forEach((record) -> { EventHandle eventHandle = ((Event)record.getData()).getEventHandle(); if (acknowledgements) { - acknowledgementSetManager.releaseEventReference(eventHandle, result); + eventHandle.release(result); } }); } diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitor.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitor.java index af9860cc9a..8c911346db 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitor.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/AcknowledgementSetMonitor.java @@ -5,9 +5,6 @@ package org.opensearch.dataprepper.acknowledgements; -import org.opensearch.dataprepper.model.event.EventHandle; -import org.opensearch.dataprepper.model.event.DefaultEventHandle; -import org.opensearch.dataprepper.model.event.InternalEventHandle; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import java.util.concurrent.locks.ReentrantLock; @@ -33,15 +30,6 @@ class AcknowledgementSetMonitor implements Runnable { private final AtomicInteger numInvalidReleases; private final AtomicInteger numNullHandles; - private DefaultAcknowledgementSet getAcknowledgementSet(final EventHandle eventHandle) { - if (eventHandle instanceof DefaultEventHandle) { - InternalEventHandle internalEventHandle = (InternalEventHandle)(DefaultEventHandle)eventHandle; - return (DefaultAcknowledgementSet)internalEventHandle.getAcknowledgementSet(); - } else { - throw new RuntimeException("Unsupported event handle"); - } - } - public AcknowledgementSetMonitor() { this.acknowledgementSets = new HashSet<>(); this.lock = new ReentrantLock(true); @@ -67,55 +55,6 @@ public void add(final AcknowledgementSet acknowledgementSet) { } } - public void acquire(final EventHandle eventHandle) { - if (eventHandle == null) { - numNullHandles.incrementAndGet(); - return; - } - - DefaultAcknowledgementSet acknowledgementSet = getAcknowledgementSet(eventHandle); - lock.lock(); - boolean exists = false; - try { - exists = acknowledgementSets.contains(acknowledgementSet); - } finally { - lock.unlock(); - } - // if acknowledgementSet doesn't exist then it means that the - // event still active even after the acknowledgement set is - // cleaned up. - if (exists) { - acknowledgementSet.acquire(eventHandle); - } else { - LOG.warn("Trying acquire an event in an AcknowledgementSet that does not exist"); - numInvalidAcquires.incrementAndGet(); - } - } - - public void release(final EventHandle eventHandle, final boolean success) { - if (eventHandle == null) { - numNullHandles.incrementAndGet(); - return; - } - DefaultAcknowledgementSet acknowledgementSet = getAcknowledgementSet(eventHandle); - lock.lock(); - boolean exists = false; - try { - exists = acknowledgementSets.contains(acknowledgementSet); - } finally { - lock.unlock(); - } - // if acknowledgementSet doesn't exist then it means some late - // arrival of event handle release after the acknowledgement set - // is cleaned up. - if (exists) { - boolean b = acknowledgementSet.release(eventHandle, success); - } else { - LOG.warn("Trying to release from an AcknowledgementSet that does not exist"); - numInvalidReleases.incrementAndGet(); - } - } - /** * for testing * @return the size @@ -131,6 +70,8 @@ public void run() { if (acknowledgementSets.size() > 0) { acknowledgementSets.removeIf((ackSet) -> ((DefaultAcknowledgementSet) ackSet).isDone()); } + Thread.sleep(1000); + } catch (InterruptedException e) { } finally { lock.unlock(); } diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSet.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSet.java index c2823203fe..fd26d10c72 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSet.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSet.java @@ -82,7 +82,7 @@ public void add(Event event) { EventHandle eventHandle = event.getEventHandle(); if (eventHandle instanceof DefaultEventHandle) { InternalEventHandle internalEventHandle = (InternalEventHandle)(DefaultEventHandle)eventHandle; - internalEventHandle.setAcknowledgementSet(this); + internalEventHandle.addAcknowledgementSet(this); pendingAcknowledgments.put(eventHandle, new AtomicInteger(1)); totalEventsAdded.incrementAndGet(); } diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManager.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManager.java index 3f2e3761bd..b8f81dbfc1 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManager.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManager.java @@ -7,8 +7,6 @@ import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.metrics.PluginMetrics; import javax.inject.Inject; @@ -49,18 +47,6 @@ public AcknowledgementSet create(final Consumer callback, final Duratio return acknowledgementSet; } - public void acquireEventReference(final Event event) { - acquireEventReference(event.getEventHandle()); - } - - public void acquireEventReference(final EventHandle eventHandle) { - acknowledgementSetMonitor.acquire(eventHandle); - } - - public void releaseEventReference(final EventHandle eventHandle, final boolean success) { - acknowledgementSetMonitor.release(eventHandle, success); - } - public void shutdown() { acknowledgementSetMonitorThread.stop(); } diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManager.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManager.java index 2e112b4560..52f0e1978f 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManager.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManager.java @@ -5,8 +5,6 @@ package org.opensearch.dataprepper.acknowledgements; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import java.util.function.Consumer; @@ -26,15 +24,4 @@ public AcknowledgementSet create(final Consumer callback, final Duratio throw new UnsupportedOperationException("create operation not supported"); } - public void acquireEventReference(final Event event) { - throw new UnsupportedOperationException("acquire operation not supported"); - } - - public void acquireEventReference(final EventHandle eventHandle) { - throw new UnsupportedOperationException("acquire operation not supported"); - } - - public void releaseEventReference(final EventHandle eventHandle, boolean success) { - throw new UnsupportedOperationException("release operation not supported"); - } } diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/ProcessWorker.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/ProcessWorker.java index 2178fd6bcc..b5538dfe73 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/ProcessWorker.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/ProcessWorker.java @@ -100,7 +100,7 @@ private void processAcknowledgements(List inputEvents, Collection {return null; }).when(acknowledgementSet1).acquire(eventHandle1); - } catch (Exception e){} - acknowledgementSetMonitor.add(acknowledgementSet1); - acknowledgementSetMonitor.acquire(eventHandle1); - acknowledgementSetMonitor.release(eventHandle1, true); - Thread shutdownThread = new Thread(() -> { - try { - Thread.sleep(DEFAULT_WAIT_TIME_MS); - } catch (Exception e){} - }); - shutdownThread.start(); - acknowledgementSetMonitor.run(); - assertThat(acknowledgementSetMonitor.getSize(), equalTo(0)); - } - - @Test - public void testAcknowledgementSetInvalidAcquire() { - acknowledgementSet2 = mock(DefaultAcknowledgementSet.class); - when(eventHandle1.getAcknowledgementSet()).thenReturn(acknowledgementSet2); - acknowledgementSetMonitor.add(acknowledgementSet1); - acknowledgementSetMonitor.acquire(eventHandle1); - Thread shutdownThread = new Thread(() -> { - try { - Thread.sleep(DEFAULT_WAIT_TIME_MS); - } catch (Exception e){} - }); - shutdownThread.start(); - acknowledgementSetMonitor.run(); - assertThat(acknowledgementSetMonitor.getSize(), equalTo(0)); - assertThat(acknowledgementSetMonitor.getNumInvalidAcquires(), equalTo(1)); - } - - @Test - public void testAcknowledgementSetInvalidRelease() { - acknowledgementSet2 = mock(DefaultAcknowledgementSet.class); - when(eventHandle1.getAcknowledgementSet()).thenReturn(acknowledgementSet2); - acknowledgementSetMonitor.add(acknowledgementSet1); - acknowledgementSetMonitor.release(eventHandle1, true); - Thread shutdownThread = new Thread(() -> { - try { - Thread.sleep(DEFAULT_WAIT_TIME_MS); - } catch (Exception e){} - }); - shutdownThread.start(); - acknowledgementSetMonitor.run(); - assertThat(acknowledgementSetMonitor.getSize(), equalTo(0)); - assertThat(acknowledgementSetMonitor.getNumInvalidReleases(), equalTo(1)); - } } diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManagerTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManagerTests.java index 1b87d6c849..a083f5ea85 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManagerTests.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetManagerTests.java @@ -14,6 +14,8 @@ import org.junit.jupiter.api.Test; import org.mockito.junit.jupiter.MockitoExtension; import org.junit.jupiter.api.extension.ExtendWith; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.ArgumentMatchers.any; import org.mockito.Mock; import static org.awaitility.Awaitility.await; @@ -53,17 +55,27 @@ class DefaultAcknowledgementSetManagerTests { void setup() { currentRatio = 0; callbackExecutor = Executors.newScheduledThreadPool(2); + acknowledgementSetManager = createObjectUnderTest(); + AcknowledgementSet acknowledgementSet1 = acknowledgementSetManager.create((flag) -> { result = flag; }, TEST_TIMEOUT); event1 = mock(JacksonEvent.class); eventHandle1 = mock(DefaultEventHandle.class); + lenient().doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet1.release(eventHandle1, result); + return null; + }).when(eventHandle1).release(any(Boolean.class)); lenient().when(event1.getEventHandle()).thenReturn(eventHandle1); pluginMetrics = mock(PluginMetrics.class); event2 = mock(JacksonEvent.class); eventHandle2 = mock(DefaultEventHandle.class); + lenient().doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet1.release(eventHandle2, result); + return null; + }).when(eventHandle2).release(any(Boolean.class)); lenient().when(event2.getEventHandle()).thenReturn(eventHandle2); - acknowledgementSetManager = createObjectUnderTest(); - AcknowledgementSet acknowledgementSet1 = acknowledgementSetManager.create((flag) -> { result = flag; }, TEST_TIMEOUT); acknowledgementSet1.add(event1); acknowledgementSet1.add(event2); lenient().when(eventHandle1.getAcknowledgementSet()).thenReturn(acknowledgementSet1); @@ -77,8 +89,8 @@ DefaultAcknowledgementSetManager createObjectUnderTest() { @Test void testBasic() { - acknowledgementSetManager.releaseEventReference(eventHandle2, true); - acknowledgementSetManager.releaseEventReference(eventHandle1, true); + eventHandle2.release(true); + eventHandle1.release(true); await().atMost(TEST_TIMEOUT.multipliedBy(5)) .untilAsserted(() -> { assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0)); @@ -88,7 +100,7 @@ void testBasic() { @Test void testExpirations() throws InterruptedException { - acknowledgementSetManager.releaseEventReference(eventHandle2, true); + eventHandle2.release(true); Thread.sleep(TEST_TIMEOUT.multipliedBy(5).toMillis()); assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0)); await().atMost(TEST_TIMEOUT.multipliedBy(5)) @@ -99,17 +111,22 @@ void testExpirations() throws InterruptedException { @Test void testMultipleAcknowledgementSets() { + AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, TEST_TIMEOUT); event3 = mock(JacksonEvent.class); eventHandle3 = mock(DefaultEventHandle.class); + doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet2.release(eventHandle3, result); + return null; + }).when(eventHandle3).release(any(Boolean.class)); lenient().when(event3.getEventHandle()).thenReturn(eventHandle3); - AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, TEST_TIMEOUT); acknowledgementSet2.add(event3); lenient().when(eventHandle3.getAcknowledgementSet()).thenReturn(acknowledgementSet2); acknowledgementSet2.complete(); - acknowledgementSetManager.releaseEventReference(eventHandle2, true); - acknowledgementSetManager.releaseEventReference(eventHandle3, true); + eventHandle2.release(true); + eventHandle3.release(true); await().atMost(TEST_TIMEOUT.multipliedBy(5)) .untilAsserted(() -> { assertThat(acknowledgementSetManager.getAcknowledgementSetMonitor().getSize(), equalTo(0)); @@ -119,22 +136,42 @@ void testMultipleAcknowledgementSets() { @Test void testWithProgressCheckCallbacks() { + AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofMillis(10000)); eventHandle3 = mock(DefaultEventHandle.class); + doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet2.release(eventHandle3, result); + return null; + }).when(eventHandle3).release(any(Boolean.class)); lenient().when(event3.getEventHandle()).thenReturn(eventHandle3); eventHandle4 = mock(DefaultEventHandle.class); + doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet2.release(eventHandle4, result); + return null; + }).when(eventHandle4).release(any(Boolean.class)); JacksonEvent event4 = mock(JacksonEvent.class); lenient().when(event4.getEventHandle()).thenReturn(eventHandle4); eventHandle5 = mock(DefaultEventHandle.class); + doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet2.release(eventHandle5, result); + return null; + }).when(eventHandle5).release(any(Boolean.class)); JacksonEvent event5 = mock(JacksonEvent.class); lenient().when(event5.getEventHandle()).thenReturn(eventHandle5); eventHandle6 = mock(DefaultEventHandle.class); + doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet2.release(eventHandle6, result); + return null; + }).when(eventHandle6).release(any(Boolean.class)); JacksonEvent event6 = mock(JacksonEvent.class); lenient().when(event6.getEventHandle()).thenReturn(eventHandle6); - AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofMillis(10000)); acknowledgementSet2.addProgressCheck((progressCheck) -> {currentRatio = progressCheck.getRatio();}, Duration.ofSeconds(1)); acknowledgementSet2.add(event3); acknowledgementSet2.add(event4); @@ -145,22 +182,22 @@ void testWithProgressCheckCallbacks() { lenient().when(eventHandle5.getAcknowledgementSet()).thenReturn(acknowledgementSet2); lenient().when(eventHandle6.getAcknowledgementSet()).thenReturn(acknowledgementSet2); acknowledgementSet2.complete(); - acknowledgementSetManager.releaseEventReference(eventHandle3, true); + eventHandle3.release(true); await().atMost(TEST_TIMEOUT.multipliedBy(5)) .untilAsserted(() -> { assertThat(currentRatio, equalTo(0.75)); }); - acknowledgementSetManager.releaseEventReference(eventHandle4, true); + eventHandle4.release(true); await().atMost(TEST_TIMEOUT.multipliedBy(5)) .untilAsserted(() -> { assertThat(currentRatio, equalTo(0.5)); }); - acknowledgementSetManager.releaseEventReference(eventHandle5, true); + eventHandle5.release(true); await().atMost(TEST_TIMEOUT.multipliedBy(5)) .untilAsserted(() -> { assertThat(currentRatio, equalTo(0.25)); }); - acknowledgementSetManager.releaseEventReference(eventHandle6, true); + eventHandle6.release(true); await().atMost(TEST_TIMEOUT.multipliedBy(5)) .untilAsserted(() -> { assertThat(result, equalTo(true)); @@ -170,14 +207,30 @@ void testWithProgressCheckCallbacks() { @Test void testWithProgressCheckCallbacks_AcksExpire() { + AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofSeconds(10)); eventHandle3 = mock(DefaultEventHandle.class); + doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet2.release(eventHandle3, result); + return null; + }).when(eventHandle3).release(any(Boolean.class)); lenient().when(event3.getEventHandle()).thenReturn(eventHandle3); eventHandle4 = mock(DefaultEventHandle.class); + doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet2.release(eventHandle4, result); + return null; + }).when(eventHandle4).release(any(Boolean.class)); JacksonEvent event4 = mock(JacksonEvent.class); lenient().when(event4.getEventHandle()).thenReturn(eventHandle4); eventHandle5 = mock(DefaultEventHandle.class); + doAnswer(a -> { + Boolean result = (Boolean)a.getArgument(0); + acknowledgementSet2.release(eventHandle5, result); + return null; + }).when(eventHandle5).release(any(Boolean.class)); JacksonEvent event5 = mock(JacksonEvent.class); lenient().when(event5.getEventHandle()).thenReturn(eventHandle5); @@ -185,7 +238,6 @@ void testWithProgressCheckCallbacks_AcksExpire() { JacksonEvent event6 = mock(JacksonEvent.class); lenient().when(event6.getEventHandle()).thenReturn(eventHandle6); - AcknowledgementSet acknowledgementSet2 = acknowledgementSetManager.create((flag) -> { result = flag; }, Duration.ofSeconds(10)); acknowledgementSet2.addProgressCheck((progressCheck) -> {currentRatio = progressCheck.getRatio();}, Duration.ofSeconds(1)); acknowledgementSet2.add(event3); acknowledgementSet2.add(event4); @@ -196,17 +248,17 @@ void testWithProgressCheckCallbacks_AcksExpire() { lenient().when(eventHandle5.getAcknowledgementSet()).thenReturn(acknowledgementSet2); lenient().when(eventHandle6.getAcknowledgementSet()).thenReturn(acknowledgementSet2); acknowledgementSet2.complete(); - acknowledgementSetManager.releaseEventReference(eventHandle3, true); + eventHandle3.release(true); await().atMost(TEST_TIMEOUT.multipliedBy(5)) .untilAsserted(() -> { assertThat(currentRatio, equalTo(0.75)); }); - acknowledgementSetManager.releaseEventReference(eventHandle4, true); + eventHandle4.release(true); await().atMost(TEST_TIMEOUT.multipliedBy(5)) .untilAsserted(() -> { assertThat(currentRatio, equalTo(0.5)); }); - acknowledgementSetManager.releaseEventReference(eventHandle5, true); + eventHandle5.release(true); await().atMost(TEST_TIMEOUT.multipliedBy(5)) .untilAsserted(() -> { assertThat(currentRatio, equalTo(0.25)); diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetTests.java index 28e17d77cc..a3ee665adf 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetTests.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/DefaultAcknowledgementSetTests.java @@ -91,14 +91,14 @@ void setupEvent() { AcknowledgementSet acknowledgementSet = a.getArgument(0); lenient().when(handle.getAcknowledgementSet()).thenReturn(acknowledgementSet); return null; - }).when(handle).setAcknowledgementSet(any(AcknowledgementSet.class)); + }).when(handle).addAcknowledgementSet(any(AcknowledgementSet.class)); lenient().when(event.getEventHandle()).thenReturn(handle); event2 = mock(JacksonEvent.class); lenient().doAnswer(a -> { AcknowledgementSet acknowledgementSet = a.getArgument(0); lenient().when(handle2.getAcknowledgementSet()).thenReturn(acknowledgementSet); return null; - }).when(handle2).setAcknowledgementSet(any(AcknowledgementSet.class)); + }).when(handle2).addAcknowledgementSet(any(AcknowledgementSet.class)); handle2 = mock(DefaultEventHandle.class); lenient().when(event2.getEventHandle()).thenReturn(handle2); } @@ -186,7 +186,7 @@ void testDefaultAcknowledgementSetNegativeAcknowledgements() throws Exception { AcknowledgementSet acknowledgementSet = a.getArgument(0); lenient().when(handle.getAcknowledgementSet()).thenReturn(acknowledgementSet); return null; - }).when(handle).setAcknowledgementSet(any(AcknowledgementSet.class)); + }).when(handle).addAcknowledgementSet(any(AcknowledgementSet.class)); assertThat(handle.getAcknowledgementSet(), equalTo(defaultAcknowledgementSet)); defaultAcknowledgementSet.acquire(handle); assertThat(defaultAcknowledgementSet.release(handle, true), equalTo(false)); @@ -219,7 +219,7 @@ void testDefaultAcknowledgementSetExpirations() throws Exception { AcknowledgementSet acknowledgementSet = a.getArgument(0); lenient().when(handle.getAcknowledgementSet()).thenReturn(acknowledgementSet); return null; - }).when(handle).setAcknowledgementSet(any(AcknowledgementSet.class)); + }).when(handle).addAcknowledgementSet(any(AcknowledgementSet.class)); assertThat(handle, not(equalTo(null))); assertThat(handle.getAcknowledgementSet(), equalTo(defaultAcknowledgementSet)); assertThat(defaultAcknowledgementSet.release(handle, true), equalTo(true)); @@ -253,7 +253,7 @@ void testDefaultAcknowledgementSetWithProgressCheck() throws Exception { AcknowledgementSet acknowledgementSet = a.getArgument(0); lenient().when(handle.getAcknowledgementSet()).thenReturn(acknowledgementSet); return null; - }).when(handle).setAcknowledgementSet(any(AcknowledgementSet.class)); + }).when(handle).addAcknowledgementSet(any(AcknowledgementSet.class)); assertThat(handle, not(equalTo(null))); assertThat(handle.getAcknowledgementSet(), equalTo(defaultAcknowledgementSet)); await().atMost(Duration.ofSeconds(5)) diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManagerTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManagerTests.java index eb1303d487..8a0a4d2ffd 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManagerTests.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/acknowledgements/InactiveAcknowledgementSetManagerTests.java @@ -7,12 +7,9 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import static org.mockito.Mockito.mock; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.CoreMatchers.notNullValue; import static org.junit.jupiter.api.Assertions.assertThrows; -import org.opensearch.dataprepper.model.event.EventHandle; -import org.opensearch.dataprepper.model.event.Event; import java.time.Duration; @@ -30,25 +27,4 @@ void testCreateAPI() { assertThrows(UnsupportedOperationException.class, () -> acknowledgementSetManager.create((a)->{}, Duration.ofMillis(10))); } - @Test - void testEventAcquireAPI() { - assertThat(acknowledgementSetManager, notNullValue()); - Event event = mock(Event.class); - assertThrows(UnsupportedOperationException.class, () -> acknowledgementSetManager.acquireEventReference(event)); - } - - @Test - void testEventHandleAcquireAPI() { - assertThat(acknowledgementSetManager, notNullValue()); - EventHandle eventHandle = mock(EventHandle.class); - assertThrows(UnsupportedOperationException.class, () -> acknowledgementSetManager.acquireEventReference(eventHandle)); - } - - @Test - void testReleaseAPI() { - assertThat(acknowledgementSetManager, notNullValue()); - EventHandle eventHandle = mock(EventHandle.class); - assertThrows(UnsupportedOperationException.class, () -> acknowledgementSetManager.releaseEventReference(eventHandle, true)); - } - } diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/codec/JavaPeerForwarderCodecTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/codec/JavaPeerForwarderCodecTest.java index 70a1e737d8..bd0b26e05f 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/codec/JavaPeerForwarderCodecTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/codec/JavaPeerForwarderCodecTest.java @@ -78,7 +78,7 @@ void testCodec_with_acknowledgementSet() throws IOException, ClassNotFoundExcept inputEvents.getEvents().stream() .map(Event::getEventHandle) .map(handle -> (InternalEventHandle)handle) - .forEach(handle -> handle.setAcknowledgementSet(mock(AcknowledgementSet.class))); + .forEach(handle -> handle.addAcknowledgementSet(mock(AcknowledgementSet.class))); final byte[] bytes = createObjectUnderTest().serialize(inputEvents); final PeerForwardingEvents outputEvents = createObjectUnderTest().deserialize(bytes); assertThat(outputEvents.getDestinationPipelineName(), equalTo(inputEvents.getDestinationPipelineName())); @@ -119,4 +119,4 @@ private PeerForwardingEvents generatePeerForwardingEvents(final int numEvents) { } return new PeerForwardingEvents(events, pluginId, pipelineName); } -} \ No newline at end of file +} diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineTests.java index 5c0a9a974e..c2e0ad769f 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineTests.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineTests.java @@ -441,7 +441,7 @@ void publishToSinks_calls_route_with_Events_and_Sinks_verify_AcknowledgementSetM Pipeline pipeline = createObjectUnderTest(); when(mockSource.areAcknowledgementsEnabled()).thenReturn(true); pipeline.publishToSinks(records); - verify(acknowledgementSetManager).acquireEventReference(any(DefaultEventHandle.class)); + verify(eventHandle).acquireReference(); verify(router) .route(anyCollection(), eq(dataFlowComponents), any(RouterGetRecordStrategy.class), any(BiConsumer.class)); diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/ProcessWorkerTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/ProcessWorkerTest.java index 3d13c0d49f..455da07a93 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/ProcessWorkerTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/ProcessWorkerTest.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.event.DefaultEventHandle; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.event.InternalEventHandle; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.Source; @@ -27,7 +28,6 @@ import java.util.concurrent.Future; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.never; @@ -104,7 +104,6 @@ void testProcessWorkerHappyPathWithAcknowledgments() { final Record mockRecord = mock(Record.class); final Event mockEvent = mock(Event.class); final EventHandle eventHandle = mock(DefaultEventHandle.class); - when(((DefaultEventHandle) eventHandle).getAcknowledgementSet()).thenReturn(mock(AcknowledgementSet.class)); when(mockRecord.getData()).thenReturn(mockEvent); when(mockEvent.getEventHandle()).thenReturn(eventHandle); @@ -174,8 +173,8 @@ void testProcessWorkerWithProcessorThrowingExceptionAndAcknowledgmentsEnabledIsH final Record mockRecord = mock(Record.class); final Event mockEvent = mock(Event.class); final EventHandle eventHandle = mock(DefaultEventHandle.class); - when(((DefaultEventHandle) eventHandle).getAcknowledgementSet()).thenReturn(mock(AcknowledgementSet.class)); - doNothing().when(eventHandle).release(true); + final AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class); + ((InternalEventHandle)eventHandle).addAcknowledgementSet(acknowledgementSet); when(mockRecord.getData()).thenReturn(mockEvent); when(mockEvent.getEventHandle()).thenReturn(eventHandle); @@ -218,8 +217,8 @@ void testProcessWorkerWithProcessorDroppingAllRecordsAndAcknowledgmentsEnabledIs final Record mockRecord = mock(Record.class); final Event mockEvent = mock(Event.class); final EventHandle eventHandle = mock(DefaultEventHandle.class); - when(((DefaultEventHandle) eventHandle).getAcknowledgementSet()).thenReturn(mock(AcknowledgementSet.class)); - doNothing().when(eventHandle).release(true); + final AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class); + ((InternalEventHandle)eventHandle).addAcknowledgementSet(acknowledgementSet); when(mockRecord.getData()).thenReturn(mockEvent); when(mockEvent.getEventHandle()).thenReturn(eventHandle); diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategyTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategyTests.java index 4c56113323..c971cd5b8d 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategyTests.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/RouterCopyRecordStrategyTests.java @@ -83,7 +83,7 @@ void setUp() { int v = handleRefCount.getOrDefault(handle, 0); handleRefCount.put(handle, v+1); return null; - }).when(acknowledgementSetManager).acquireEventReference(any(DefaultEventHandle.class)); + }).when(acknowledgementSet1).acquire(any(DefaultEventHandle.class)); } catch (Exception e){} mockRecordsIn = IntStream.range(0, 10) .mapToObj(i -> mock(Record.class)) @@ -103,7 +103,7 @@ private void attachEventHandlesToRecordsIn(List eventHandles while (iter.hasNext()) { Record r = (Record) iter.next(); DefaultEventHandle handle = (DefaultEventHandle)((JacksonEvent)r.getData()).getEventHandle(); - handle.setAcknowledgementSet(acknowledgementSet1); + handle.addAcknowledgementSet(acknowledgementSet1); eventHandles.add(handle); } } @@ -195,6 +195,7 @@ void test_one_record_with_acknowledgements() { assertTrue(getRecordStrategy.getReferencedRecords().contains(firstRecord)); recordOut = getRecordStrategy.getRecord(firstRecord); assertThat(recordOut, sameInstance(firstRecord)); + firstHandle.addAcknowledgementSet(acknowledgementSet1); assertThat(handleRefCount.get(firstHandle), equalTo(1)); recordOut = getRecordStrategy.getRecord(firstRecord); assertThat(recordOut, sameInstance(firstRecord)); @@ -242,7 +243,7 @@ void test_one_record_with_acknowledgements_and_multi_components() { try { doAnswer((i) -> { JacksonEvent e1 = (JacksonEvent) i.getArgument(0); - ((DefaultEventHandle)e1.getEventHandle()).setAcknowledgementSet(acknowledgementSet1); + ((DefaultEventHandle)e1.getEventHandle()).addAcknowledgementSet(acknowledgementSet1); return null; }).when(acknowledgementSet1).add(any(JacksonEvent.class)); } catch (Exception e){} @@ -280,7 +281,7 @@ void test_multiple_records_with_acknowledgements_and_multi_components() { try { doAnswer((i) -> { JacksonEvent e1 = (JacksonEvent) i.getArgument(0); - ((DefaultEventHandle)e1.getEventHandle()).setAcknowledgementSet(acknowledgementSet1); + ((DefaultEventHandle)e1.getEventHandle()).addAcknowledgementSet(acknowledgementSet1); return null; }).when(acknowledgementSet1).add(any(JacksonEvent.class)); } catch (Exception 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 88c4df5202..c1f84d2bd4 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 @@ -565,8 +565,8 @@ void output_will_skip_and_drop_failed_records() throws IOException { DefaultEventHandle eventHandle1 = (DefaultEventHandle)event1.getEventHandle(); DefaultEventHandle eventHandle2 = (DefaultEventHandle)event2.getEventHandle(); - eventHandle1.setAcknowledgementSet(acknowledgementSet); - eventHandle2.setAcknowledgementSet(acknowledgementSet); + eventHandle1.addAcknowledgementSet(acknowledgementSet); + eventHandle2.addAcknowledgementSet(acknowledgementSet); doThrow(RuntimeException.class).when(codec).writeEvent(event1, outputStream); diff --git a/data-prepper-plugins/split-event-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorTest.java b/data-prepper-plugins/split-event-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorTest.java index 7fc126fdf5..4e8944ab91 100644 --- a/data-prepper-plugins/split-event-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorTest.java +++ b/data-prepper-plugins/split-event-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorTest.java @@ -69,7 +69,7 @@ private Record createTestRecord(final Map data) { DefaultEventHandle eventHandle = (DefaultEventHandle) event.getEventHandle(); - eventHandle.setAcknowledgementSet(mockAcknowledgementSet); + eventHandle.addAcknowledgementSet(mockAcknowledgementSet); return new Record<>(event); } From 2c53d7e8ddef713fd040a22cd51e22af9c964892 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 17 Jun 2024 14:55:38 -0500 Subject: [PATCH 022/159] Deprecates PluginSetting which should not be used for plugins anymore. (#4624) Signed-off-by: David Venable --- .../model/configuration/PluginSetting.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PluginSetting.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PluginSetting.java index a8ea4a3ee1..61db9a3c7e 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PluginSetting.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PluginSetting.java @@ -5,10 +5,22 @@ package org.opensearch.dataprepper.model.configuration; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; + import java.util.Collections; import java.util.List; import java.util.Map; +/** + * Deprecated class for getting plugin settings. + *

+ * Only projects within data-prepper-core should use this. It is currently used + * extensively in plugin framework to load plugins. In Data Prepper 3.0 this + * class will be moved into data-prepper-core and not exposed to plugins anymore. + * + * @deprecated Use {@link DataPrepperPlugin#pluginConfigurationType()} or {@link PipelineDescription} instead. + */ +@Deprecated public class PluginSetting implements PipelineDescription { private static final String UNEXPECTED_ATTRIBUTE_TYPE_MSG = "Unexpected type [%s] for attribute [%s]"; From 007aada588096bd00ecc591b08082e07cd344e82 Mon Sep 17 00:00:00 2001 From: David Venable Date: Tue, 18 Jun 2024 18:49:34 -0500 Subject: [PATCH 023/159] Implements equals, hashCode, and toString for JacksonEventKey. (#4633) Signed-off-by: David Venable --- .../model/event/JacksonEventKey.java | 20 +++++ .../model/event/JacksonEventKeyTest.java | 88 +++++++++++++++++++ 2 files changed, 108 insertions(+) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java index 172e6b1254..2df755492a 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java @@ -71,6 +71,26 @@ boolean supports(final EventKeyFactory.EventAction eventAction) { return supportedActions.contains(eventAction); } + @Override + public boolean equals(final Object other) { + if (this == other) + return true; + if (other == null || getClass() != other.getClass()) + return false; + final JacksonEventKey that = (JacksonEventKey) other; + return Objects.equals(key, that.key) && Arrays.equals(eventActions, that.eventActions); + } + + @Override + public int hashCode() { + return Objects.hash(key, Arrays.hashCode(eventActions)); + } + + @Override + public String toString() { + return key; + } + private String checkAndTrimKey(final String key) { checkKey(key); return trimTrailingSlashInKey(key); diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java index 359db06278..929151175b 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java @@ -21,6 +21,7 @@ import java.util.stream.Stream; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.sameInstance; import static org.hamcrest.MatcherAssert.assertThat; @@ -176,4 +177,91 @@ public Stream provideArguments(final ExtensionContext exten ); } } + + @ParameterizedTest + @EnumSource(EventKeyFactory.EventAction.class) + void equals_returns_true_for_same_key_and_actions(final EventKeyFactory.EventAction eventAction) { + final String testKey = UUID.randomUUID().toString(); + final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, eventAction); + final JacksonEventKey other = new JacksonEventKey(testKey, eventAction); + + assertThat(objectUnderTest.equals(other), equalTo(true)); + } + + @Test + void equals_returns_true_for_same_instance() { + final JacksonEventKey objectUnderTest = new JacksonEventKey(UUID.randomUUID().toString(), + EventKeyFactory.EventAction.PUT); + + assertThat(objectUnderTest.equals(objectUnderTest), equalTo(true)); + } + + @Test + void equals_returns_false_for_null() { + final JacksonEventKey objectUnderTest = new JacksonEventKey(UUID.randomUUID().toString(), + EventKeyFactory.EventAction.PUT); + + assertThat(objectUnderTest.equals(null), equalTo(false)); + } + + @Test + void equals_returns_false_for_non_EventKey() { + final String testKey = UUID.randomUUID().toString(); + final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, + EventKeyFactory.EventAction.PUT); + + assertThat(objectUnderTest.equals(testKey), equalTo(false)); + } + + @Test + void equals_returns_false_for_same_key_but_different_actions() { + final String testKey = UUID.randomUUID().toString(); + final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, EventKeyFactory.EventAction.PUT); + final JacksonEventKey other = new JacksonEventKey(testKey, EventKeyFactory.EventAction.GET); + + assertThat(objectUnderTest.equals(other), equalTo(false)); + } + + @ParameterizedTest + @EnumSource(EventKeyFactory.EventAction.class) + void equals_returns_false_for_different_key_but_same_actions(final EventKeyFactory.EventAction eventAction) { + final JacksonEventKey objectUnderTest = new JacksonEventKey(UUID.randomUUID().toString(), eventAction); + final JacksonEventKey other = new JacksonEventKey(UUID.randomUUID().toString(), eventAction); + + assertThat(objectUnderTest.equals(other), equalTo(false)); + } + + @ParameterizedTest + @EnumSource(EventKeyFactory.EventAction.class) + void hashCode_is_the_same_for_same_key_and_actions(final EventKeyFactory.EventAction eventAction) { + final String testKey = UUID.randomUUID().toString(); + final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, eventAction); + final JacksonEventKey other = new JacksonEventKey(testKey, eventAction); + + assertThat(objectUnderTest.hashCode(), equalTo(other.hashCode())); + } + + @ParameterizedTest + @CsvSource({ + "test, PUT, test2, PUT", + "test, PUT, test2, PUT", + "test, PUT, test, GET" + }) + void hashCode_is_the_different_for_same_key_and_actions( + final String testKey, final EventKeyFactory.EventAction eventAction, + final String testKeyOther, final EventKeyFactory.EventAction eventActionOther) { + final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, eventAction); + final JacksonEventKey other = new JacksonEventKey(testKeyOther, eventActionOther); + + assertThat(objectUnderTest.hashCode(), not(equalTo(other.hashCode()))); + } + + @ParameterizedTest + @EnumSource(EventKeyFactory.EventAction.class) + void toString_returns_the_key(final EventKeyFactory.EventAction eventAction) { + final String testKey = UUID.randomUUID().toString(); + final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, eventAction); + + assertThat(objectUnderTest.toString(), equalTo(testKey)); + } } \ No newline at end of file From f716e9cfc9f51c27fc43ddb5e88823541e097731 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 18 Jun 2024 16:50:42 -0700 Subject: [PATCH 024/159] Bump urllib3 in /examples/trace-analytics-sample-app/sample-app (#4631) Bumps [urllib3](https://github.com/urllib3/urllib3) from 2.0.7 to 2.2.2. - [Release notes](https://github.com/urllib3/urllib3/releases) - [Changelog](https://github.com/urllib3/urllib3/blob/main/CHANGES.rst) - [Commits](https://github.com/urllib3/urllib3/compare/2.0.7...2.2.2) --- updated-dependencies: - dependency-name: urllib3 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- examples/trace-analytics-sample-app/sample-app/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/trace-analytics-sample-app/sample-app/requirements.txt b/examples/trace-analytics-sample-app/sample-app/requirements.txt index 3f7f8b5f1d..df780b836b 100644 --- a/examples/trace-analytics-sample-app/sample-app/requirements.txt +++ b/examples/trace-analytics-sample-app/sample-app/requirements.txt @@ -6,5 +6,5 @@ opentelemetry-instrumentation-mysql==0.41b0 opentelemetry-instrumentation-requests==0.41b0 opentelemetry-sdk==1.20.0 protobuf==3.20.3 -urllib3==2.0.7 +urllib3==2.2.2 werkzeug==3.0.3 \ No newline at end of file From a3519ab5bb4f68b9c62cbd557a39649aa0a8adbc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 19 Jun 2024 07:23:44 -0700 Subject: [PATCH 025/159] Bump urllib3 in /release/smoke-tests/otel-span-exporter (#4640) Bumps [urllib3](https://github.com/urllib3/urllib3) from 1.26.18 to 1.26.19. - [Release notes](https://github.com/urllib3/urllib3/releases) - [Changelog](https://github.com/urllib3/urllib3/blob/1.26.19/CHANGES.rst) - [Commits](https://github.com/urllib3/urllib3/compare/1.26.18...1.26.19) --- updated-dependencies: - dependency-name: urllib3 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- release/smoke-tests/otel-span-exporter/requirements.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/release/smoke-tests/otel-span-exporter/requirements.txt b/release/smoke-tests/otel-span-exporter/requirements.txt index 3cd451baf4..6968658846 100644 --- a/release/smoke-tests/otel-span-exporter/requirements.txt +++ b/release/smoke-tests/otel-span-exporter/requirements.txt @@ -15,5 +15,5 @@ opentelemetry-semantic-conventions==0.26b1 protobuf==3.19.5 requests==2.32.3 six==1.16.0 -urllib3==1.26.18 +urllib3==1.26.19 wrapt==1.13.3 From 64f42a994dda7b8342f3cc21733f4090eabf730c Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Wed, 19 Jun 2024 14:55:10 -0500 Subject: [PATCH 026/159] Track last scan time from before scan starts instead of based on last Modified of objects (#4493) Signed-off-by: Taylor Gray --- .../s3/S3ScanPartitionCreationSupplier.java | 38 +++++-------------- .../S3ScanPartitionCreationSupplierTest.java | 26 ++++++++----- 2 files changed, 25 insertions(+), 39 deletions(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplier.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplier.java index 22e0a15678..66a0df271c 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplier.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplier.java @@ -87,6 +87,7 @@ public List apply(final Map globalStateMap) if (Objects.nonNull(s3ScanKeyPathOption) && Objects.nonNull(s3ScanKeyPathOption.getS3ScanExcludeSuffixOptions())) excludeItems.addAll(s3ScanKeyPathOption.getS3ScanExcludeSuffixOptions()); + final Instant updatedScanTime = Instant.now(); if (Objects.nonNull(s3ScanKeyPathOption) && Objects.nonNull(s3ScanKeyPathOption.getS3scanIncludePrefixOptions())) s3ScanKeyPathOption.getS3scanIncludePrefixOptions().forEach(includePath -> { listObjectsV2Request.prefix(includePath); @@ -96,6 +97,8 @@ public List apply(final Map globalStateMap) else objectsToProcess.addAll(listFilteredS3ObjectsForBucket(excludeItems, listObjectsV2Request, scanOptions.getBucketOption().getName(), scanOptions.getUseStartDateTime(), scanOptions.getUseEndDateTime(), globalStateMap)); + + globalStateMap.put(scanOptions.getBucketOption().getName(), updatedScanTime.toString()); } globalStateMap.put(SCAN_COUNT, (Integer) globalStateMap.get(SCAN_COUNT) + 1); @@ -110,13 +113,13 @@ private List listFilteredS3ObjectsForBucket(final List globalStateMap) { - Instant mostRecentLastModifiedTimestamp = globalStateMap.get(bucket) != null ? Instant.parse((String) globalStateMap.get(bucket)) : null; + final Instant previousScanTime = globalStateMap.get(bucket) != null ? 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)) + .filter(s3Object -> isLastModifiedTimeAfterMostRecentScanForBucket(previousScanTime, s3Object)) .map(s3Object -> Pair.of(s3Object.key(), instantToLocalDateTime(s3Object.lastModified()))) .filter(keyTimestampPair -> !keyTimestampPair.left().endsWith("/")) .filter(keyTimestampPair -> excludeKeyPaths.stream() @@ -127,12 +130,8 @@ private List listFilteredS3ObjectsForBucket(final List folderPartitions = allPartitionIdentifiers.stream() .map(partitionIdentifier -> { @@ -185,32 +184,13 @@ private void initializeGlobalStateMap(final Map globalStateMap) globalStateMap.put(SINGLE_SCAN_COMPLETE, false); } - private boolean isLastModifiedTimeAfterMostRecentScanForBucket(final String bucketName, - final S3Object s3Object, - final Map globalStateMap) { - if (!globalStateMap.containsKey(bucketName) || Objects.isNull(globalStateMap.get(bucketName))) { + private boolean isLastModifiedTimeAfterMostRecentScanForBucket(final Instant previousScanTime, + final S3Object s3Object) { + if (previousScanTime == null) { return true; } - final Instant lastProcessedObjectTimestamp = Instant.parse((String) globalStateMap.get(bucketName)); - - return s3Object.lastModified().compareTo(lastProcessedObjectTimestamp.minusSeconds(1)) >= 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; + return s3Object.lastModified().compareTo(previousScanTime) >= 0; } private boolean shouldScanBeSkipped(final Map globalStateMap) { diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplierTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplierTest.java index e320981b9d..0545a49459 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplierTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplierTest.java @@ -40,6 +40,9 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.notNullValue; import static org.mockito.BDDMockito.given; import static org.mockito.Mockito.mock; @@ -213,7 +216,7 @@ void getNextPartition_supplier_with_scheduling_options_returns_expected_Partitio s3ObjectsList.add(invalidForFirstBucketSuffixObject); expectedPartitionIdentifiers.add(PartitionIdentifier.builder().withPartitionKey(secondBucket + "|" + invalidForFirstBucketSuffixObject.key()).build()); - final Instant mostRecentFirstScan = Instant.now().plusSeconds(1); + final Instant mostRecentFirstScan = Instant.now().plusSeconds(2); final S3Object validObject = mock(S3Object.class); given(validObject.key()).willReturn("valid"); given(validObject.lastModified()).willReturn(mostRecentFirstScan); @@ -237,10 +240,6 @@ void getNextPartition_supplier_with_scheduling_options_returns_expected_Partitio given(listObjectsResponse.contents()) .willReturn(s3ObjectsList) .willReturn(s3ObjectsList) - .willReturn(s3ObjectsList) - .willReturn(s3ObjectsList) - .willReturn(secondScanObjects) - .willReturn(secondScanObjects) .willReturn(secondScanObjects) .willReturn(secondScanObjects); @@ -248,6 +247,8 @@ void getNextPartition_supplier_with_scheduling_options_returns_expected_Partitio given(s3Client.listObjectsV2(listObjectsV2RequestArgumentCaptor.capture())).willReturn(listObjectsResponse); final Map globalStateMap = new HashMap<>(); + + final Instant beforeFirstScan = Instant.now(); final List resultingPartitions = partitionCreationSupplier.apply(globalStateMap); assertThat(resultingPartitions, notNullValue()); @@ -260,10 +261,13 @@ void getNextPartition_supplier_with_scheduling_options_returns_expected_Partitio 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(Instant.parse((CharSequence) globalStateMap.get(firstBucket)), lessThanOrEqualTo(mostRecentFirstScan)); + assertThat(Instant.parse((CharSequence) globalStateMap.get(firstBucket)), greaterThanOrEqualTo(beforeFirstScan)); assertThat(globalStateMap.containsKey(secondBucket), equalTo(true)); - assertThat(globalStateMap.get(secondBucket), equalTo(mostRecentFirstScan.toString())); + assertThat(Instant.parse((CharSequence) globalStateMap.get(secondBucket)), lessThanOrEqualTo(mostRecentFirstScan)); + assertThat(Instant.parse((CharSequence) globalStateMap.get(secondBucket)), greaterThanOrEqualTo(beforeFirstScan)); + final Instant beforeSecondScan = Instant.now(); final List secondScanPartitions = partitionCreationSupplier.apply(globalStateMap); assertThat(secondScanPartitions.size(), equalTo(expectedPartitionIdentifiersSecondScan.size())); assertThat(secondScanPartitions.stream().map(PartitionIdentifier::getPartitionKey).collect(Collectors.toList()), @@ -273,14 +277,16 @@ void getNextPartition_supplier_with_scheduling_options_returns_expected_Partitio 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(Instant.parse((CharSequence) globalStateMap.get(firstBucket)), lessThanOrEqualTo(mostRecentSecondScan)); + assertThat(Instant.parse((CharSequence) globalStateMap.get(firstBucket)), greaterThanOrEqualTo(beforeSecondScan)); assertThat(globalStateMap.containsKey(secondBucket), equalTo(true)); - assertThat(globalStateMap.get(secondBucket), equalTo(mostRecentSecondScan.toString())); + assertThat(Instant.parse((CharSequence) globalStateMap.get(secondBucket)), lessThanOrEqualTo(mostRecentSecondScan)); + assertThat(Instant.parse((CharSequence) globalStateMap.get(secondBucket)), greaterThan(beforeSecondScan)); 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(); + verify(listObjectsResponse, times(4)).contents(); } @Test From 4e947c245c9ee68d31184562b8daf04f9e2920df Mon Sep 17 00:00:00 2001 From: Dinu John <86094133+dinujoh@users.noreply.github.com> Date: Wed, 19 Jun 2024 16:09:54 -0500 Subject: [PATCH 027/159] DocumentDB Source improvements (#4645) * Extend the export partition ownership during query partition creation Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> * Add support to shutdown task refresher that starts export and stream scheduler/worker on data prepper shutdown Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> * Add AcknowledgmentStatus enum and code refactor to fail negative ack right away Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> --------- Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> --- .../mongo/documentdb/DocumentDBService.java | 8 ++++++- .../mongo/documentdb/MongoTasksRefresher.java | 13 ++++++++++- .../MongoDBExportPartitionSupplier.java | 8 +++++++ .../mongo/stream/CheckpointStatus.java | 22 ++++++++++++++----- .../stream/StreamAcknowledgementManager.java | 16 +++++++------- .../DocumentDBSourceAggregateMetrics.java | 9 ++++++-- .../documentdb/MongoTasksRefresherTest.java | 14 ++++++++++++ .../MongoDBExportPartitionSupplierTest.java | 12 +++++++++- .../StreamAcknowledgementManagerTest.java | 8 +++---- 9 files changed, 87 insertions(+), 23 deletions(-) diff --git a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/documentdb/DocumentDBService.java b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/documentdb/DocumentDBService.java index 73567b8605..1acf21620b 100644 --- a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/documentdb/DocumentDBService.java +++ b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/documentdb/DocumentDBService.java @@ -33,6 +33,7 @@ public class DocumentDBService { private final PluginConfigObservable pluginConfigObservable; private final DocumentDBSourceAggregateMetrics documentDBAggregateMetrics; private ExecutorService leaderExecutor; + private MongoTasksRefresher mongoTasksRefresher; public DocumentDBService(final EnhancedSourceCoordinator sourceCoordinator, final MongoDBSourceConfig sourceConfig, final PluginMetrics pluginMetrics, @@ -68,7 +69,7 @@ public void start(Buffer> buffer) { BackgroundThreadFactory.defaultExecutorThreadFactory("documentdb-source")); runnableList.forEach(leaderExecutor::submit); - final MongoTasksRefresher mongoTasksRefresher = new MongoTasksRefresher( + mongoTasksRefresher = new MongoTasksRefresher( buffer, sourceCoordinator, pluginMetrics, acknowledgementSetManager, numThread -> Executors.newFixedThreadPool( numThread, BackgroundThreadFactory.defaultExecutorThreadFactory("documentdb-source")), @@ -105,5 +106,10 @@ public void shutdown() { LOG.info("shutdown DocumentDB Service scheduler and worker"); leaderExecutor.shutdownNow(); } + + if (mongoTasksRefresher != null) { + LOG.info("shutdown DocumentDB Task refresher"); + mongoTasksRefresher.shutdown(); + } } } diff --git a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/documentdb/MongoTasksRefresher.java b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/documentdb/MongoTasksRefresher.java index 19c988f285..3fea4680e8 100644 --- a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/documentdb/MongoTasksRefresher.java +++ b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/documentdb/MongoTasksRefresher.java @@ -91,7 +91,7 @@ public void update(MongoDBSourceConfig pluginConfig) { private void refreshJobs(MongoDBSourceConfig pluginConfig) { final List runnables = new ArrayList<>(); if (pluginConfig.getCollections().stream().anyMatch(CollectionConfig::isExport)) { - currentMongoDBExportPartitionSupplier = new MongoDBExportPartitionSupplier(pluginConfig, documentDBAggregateMetrics); + currentMongoDBExportPartitionSupplier = new MongoDBExportPartitionSupplier(pluginConfig, sourceCoordinator, documentDBAggregateMetrics); runnables.add(new ExportScheduler(sourceCoordinator, currentMongoDBExportPartitionSupplier, pluginMetrics)); runnables.add(new ExportWorker( sourceCoordinator, buffer, pluginMetrics, acknowledgementSetManager, pluginConfig, s3PathPrefix, documentDBAggregateMetrics)); @@ -110,4 +110,15 @@ private boolean basicAuthChanged(final MongoDBSourceConfig.AuthenticationConfig return !Objects.equals(currentAuthConfig.getUsername(), newAuthConfig.getUsername()) || !Objects.equals(currentAuthConfig.getPassword(), newAuthConfig.getPassword()); } + + /** + * Interrupt the running of schedulers. + * Each scheduler must implement logic for gracefully shutdown. + */ + public void shutdown() { + if (currentExecutor != null) { + LOG.info("shutdown down export worker and stream worker"); + currentExecutor.shutdownNow(); + } + } } diff --git a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/export/MongoDBExportPartitionSupplier.java b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/export/MongoDBExportPartitionSupplier.java index d3fb3aac3c..dfbf518318 100644 --- a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/export/MongoDBExportPartitionSupplier.java +++ b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/export/MongoDBExportPartitionSupplier.java @@ -14,6 +14,7 @@ import com.mongodb.client.model.Filters; import org.bson.Document; import org.opensearch.dataprepper.model.source.coordinator.PartitionIdentifier; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.mongo.client.BsonHelper; import org.opensearch.dataprepper.plugins.mongo.client.MongoDBConnection; import org.opensearch.dataprepper.plugins.mongo.configuration.MongoDBSourceConfig; @@ -38,11 +39,14 @@ public class MongoDBExportPartitionSupplier implements Function= checkPointIntervalInMs) { long ackCount = 0; do { @@ -69,17 +69,16 @@ private void monitorAcknowledgment(final ExecutorService executorService, final if (ackCount % CHECKPOINT_RECORD_INTERVAL == 0) { checkpoint(lastCheckpointStatus.getResumeToken(), lastCheckpointStatus.getRecordCount()); } - } while (checkpointStatus != null && checkpointStatus.isAcknowledged()); + } while (checkpointStatus != null && checkpointStatus.isPositiveAcknowledgement()); checkpoint(lastCheckpointStatus.getResumeToken(), lastCheckpointStatus.getRecordCount()); lastCheckpointTime = System.currentTimeMillis(); } } else { LOG.debug("Checkpoint not complete for resume token {}", checkpointStatus.getResumeToken()); final Duration ackWaitDuration = Duration.between(Instant.ofEpochMilli(checkpointStatus.getCreateTimestamp()), Instant.now()); - // Acknowledgement not received for the checkpoint after twice ack wait time - if (ackWaitDuration.getSeconds() >= partitionAcknowledgmentTimeout.getSeconds() * 2) { + if (checkpointStatus.isNegativeAcknowledgement()) { // Give up partition and should interrupt parent thread to stop processing stream - if (lastCheckpointStatus != null && lastCheckpointStatus.isAcknowledged()) { + if (lastCheckpointStatus != null && lastCheckpointStatus.isPositiveAcknowledgement()) { partitionCheckpoint.checkpoint(lastCheckpointStatus.getResumeToken(), lastCheckpointStatus.getRecordCount()); } LOG.warn("Acknowledgement not received for the checkpoint {} past wait time. Giving up partition.", checkpointStatus.getResumeToken()); @@ -124,12 +123,13 @@ Optional createAcknowledgementSet(final String resumeToken, ackStatus.put(resumeToken, checkpointStatus); LOG.debug("Creating acknowledgment for resumeToken {}", checkpointStatus.getResumeToken()); return Optional.of(acknowledgementSetManager.create((result) -> { + final CheckpointStatus ackCheckpointStatus = ackStatus.get(resumeToken); + ackCheckpointStatus.setAcknowledgedTimestamp(Instant.now().toEpochMilli()); if (result) { - final CheckpointStatus ackCheckpointStatus = ackStatus.get(resumeToken); - ackCheckpointStatus.setAcknowledgedTimestamp(Instant.now().toEpochMilli()); - ackCheckpointStatus.setAcknowledged(true); + ackCheckpointStatus.setAcknowledged(CheckpointStatus.AcknowledgmentStatus.POSITIVE_ACK); LOG.debug("Received acknowledgment of completion from sink for checkpoint {}", resumeToken); } else { + ackCheckpointStatus.setAcknowledged(CheckpointStatus.AcknowledgmentStatus.NEGATIVE_ACK); LOG.warn("Negative acknowledgment received for checkpoint {}, resetting checkpoint", resumeToken); // default CheckpointStatus acknowledged value is false. The monitorCheckpoints method will time out // and reprocess stream from last successful checkpoint in the order. diff --git a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/utils/DocumentDBSourceAggregateMetrics.java b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/utils/DocumentDBSourceAggregateMetrics.java index db85260a52..c5a09a45e7 100644 --- a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/utils/DocumentDBSourceAggregateMetrics.java +++ b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/utils/DocumentDBSourceAggregateMetrics.java @@ -17,8 +17,7 @@ public class DocumentDBSourceAggregateMetrics { private static final String DOCUMENT_DB_EXPORT_5XX_ERRORS = "export5xxErrors"; private static final String DOCUMENT_DB_EXPORT_4XX_ERRORS = "export4xxErrors"; private static final String DOCUMENT_DB_EXPORT_API_INVOCATIONS = "exportApiInvocations"; - - + private static final String DOCUMENT_DB_EXPORT_PARTITION_QUERY_COUNT = "exportPartitionQueryCount"; private final PluginMetrics pluginMetrics; @@ -28,6 +27,7 @@ public class DocumentDBSourceAggregateMetrics { private final Counter export5xxErrors; private final Counter export4xxErrors; private final Counter exportApiInvocations; + private final Counter exportPartitionQueryCount; public DocumentDBSourceAggregateMetrics() { this.pluginMetrics = PluginMetrics.fromPrefix(DOCUMENT_DB); @@ -37,6 +37,7 @@ public DocumentDBSourceAggregateMetrics() { this.export5xxErrors = pluginMetrics.counter(DOCUMENT_DB_EXPORT_5XX_ERRORS); this.export4xxErrors = pluginMetrics.counter(DOCUMENT_DB_EXPORT_4XX_ERRORS); this.exportApiInvocations = pluginMetrics.counter(DOCUMENT_DB_EXPORT_API_INVOCATIONS); + this.exportPartitionQueryCount = pluginMetrics.counter(DOCUMENT_DB_EXPORT_PARTITION_QUERY_COUNT); } public Counter getStream5xxErrors() { @@ -62,4 +63,8 @@ public Counter getExport4xxErrors() { public Counter getExportApiInvocations() { return exportApiInvocations; } + + public Counter getExportPartitionQueryCount() { + return exportPartitionQueryCount; + } } diff --git a/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/documentdb/MongoTasksRefresherTest.java b/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/documentdb/MongoTasksRefresherTest.java index b48e097cc0..9ce93c8ded 100644 --- a/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/documentdb/MongoTasksRefresherTest.java +++ b/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/documentdb/MongoTasksRefresherTest.java @@ -254,4 +254,18 @@ void testTaskRefreshWithNullS3PathPrefix() { buffer, enhancedSourceCoordinator, pluginMetrics, acknowledgementSetManager, executorServiceFunction, null, documentDBSourceAggregateMetrics)); } + + @Test + void testTaskRefreshShutdown() { + final MongoTasksRefresher objectUnderTest = createObjectUnderTest(); + objectUnderTest.initialize(sourceConfig); + objectUnderTest.shutdown(); + verify(executorServiceFunction).apply(eq(3)); + verify(executorService).submit(any(ExportScheduler.class)); + verify(executorService).submit(any(ExportWorker.class)); + verify(executorService).submit(any(StreamScheduler.class)); + verify(executorService).shutdownNow(); + verifyNoMoreInteractions(executorServiceFunction); + + } } \ No newline at end of file diff --git a/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/export/MongoDBExportPartitionSupplierTest.java b/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/export/MongoDBExportPartitionSupplierTest.java index e8307e2d6d..0329cf7b72 100644 --- a/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/export/MongoDBExportPartitionSupplierTest.java +++ b/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/export/MongoDBExportPartitionSupplierTest.java @@ -21,6 +21,7 @@ import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.model.source.coordinator.PartitionIdentifier; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.mongo.client.MongoDBConnection; import org.opensearch.dataprepper.plugins.mongo.configuration.CollectionConfig; import org.opensearch.dataprepper.plugins.mongo.configuration.MongoDBSourceConfig; @@ -53,6 +54,9 @@ public class MongoDBExportPartitionSupplierTest { @Mock private MongoDBSourceConfig mongoDBConfig; + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + @Mock private DocumentDBSourceAggregateMetrics documentDBSourceAggregateMetrics; @@ -65,6 +69,8 @@ public class MongoDBExportPartitionSupplierTest { @Mock private Counter exportApiInvocations; @Mock + private Counter exportPartitionQueryCount; + @Mock private Counter export4xxErrors; @Mock private Counter export5xxErrors; @@ -77,9 +83,10 @@ public void setup() { lenient().when(collectionConfig.getCollectionName()).thenReturn(TEST_COLLECTION_NAME); lenient().when(mongoDBConfig.getCollections()).thenReturn(Collections.singletonList(collectionConfig)); when(documentDBSourceAggregateMetrics.getExportApiInvocations()).thenReturn(exportApiInvocations); + lenient().when(documentDBSourceAggregateMetrics.getExportPartitionQueryCount()).thenReturn(exportPartitionQueryCount); lenient().when(documentDBSourceAggregateMetrics.getExport4xxErrors()).thenReturn(export4xxErrors); lenient().when(documentDBSourceAggregateMetrics.getExport5xxErrors()).thenReturn(export5xxErrors); - testSupplier = new MongoDBExportPartitionSupplier(mongoDBConfig, documentDBSourceAggregateMetrics); + testSupplier = new MongoDBExportPartitionSupplier(mongoDBConfig, sourceCoordinator, documentDBSourceAggregateMetrics); } @Test @@ -121,6 +128,7 @@ public void test_buildPartitionsCollection() { verify(mongoClient, times(1)).close(); verify(mongoDatabase).getCollection(eq("collection")); verify(exportApiInvocations).increment(); + verify(exportPartitionQueryCount, times(2)).increment(); verify(export4xxErrors, never()).increment(); verify(export5xxErrors, never()).increment(); // And partitions are created @@ -135,6 +143,7 @@ public void test_buildPartitionsForCollection_error() { when(exportPartition.getCollection()).thenReturn("invalidDBName"); assertThrows(IllegalArgumentException.class, () -> testSupplier.apply(exportPartition)); verify(exportApiInvocations).increment(); + verify(exportPartitionQueryCount, never()).increment(); verify(export4xxErrors).increment(); verify(export5xxErrors, never()).increment(); } @@ -146,6 +155,7 @@ public void test_buildPartitions_dbException() { .thenThrow(MongoClientException.class); assertThrows(RuntimeException.class, () -> testSupplier.apply(exportPartition)); verify(exportApiInvocations).increment(); + verify(exportPartitionQueryCount, never()).increment(); verify(export4xxErrors).increment(); verify(export5xxErrors, never()).increment(); } diff --git a/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/stream/StreamAcknowledgementManagerTest.java b/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/stream/StreamAcknowledgementManagerTest.java index 78e2a51503..4e41008627 100644 --- a/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/stream/StreamAcknowledgementManagerTest.java +++ b/data-prepper-plugins/mongodb/src/test/java/org/opensearch/dataprepper/plugins/mongo/stream/StreamAcknowledgementManagerTest.java @@ -73,7 +73,7 @@ public void createAcknowledgementSet_enabled_ackSetWithAck() { consumer.accept(true); final ConcurrentHashMap ackStatus = streamAckManager.getAcknowledgementStatus(); final CheckpointStatus ackCheckpointStatus = ackStatus.get(resumeToken); - assertThat(ackCheckpointStatus.isAcknowledged(), is(true)); + assertThat(ackCheckpointStatus.isPositiveAcknowledgement(), is(true)); await() .atMost(Duration.ofSeconds(10)).untilAsserted(() -> verify(partitionCheckpoint).checkpoint(resumeToken, recordCount)); @@ -109,7 +109,7 @@ public void createAcknowledgementSet_enabled_multipleAckSetWithAck() { consumers.get(1).accept(true); ConcurrentHashMap ackStatus = streamAckManager.getAcknowledgementStatus(); CheckpointStatus ackCheckpointStatus = ackStatus.get(resumeToken2); - assertThat(ackCheckpointStatus.isAcknowledged(), is(true)); + assertThat(ackCheckpointStatus.isPositiveAcknowledgement(), is(true)); await() .atMost(Duration.ofSeconds(10)).untilAsserted(() -> verify(partitionCheckpoint).checkpoint(resumeToken2, recordCount2)); @@ -143,7 +143,7 @@ public void createAcknowledgementSet_enabled_multipleAckSetWithAckFailure() { consumers.get(1).accept(true); ConcurrentHashMap ackStatus = streamAckManager.getAcknowledgementStatus(); CheckpointStatus ackCheckpointStatus = ackStatus.get(resumeToken2); - assertThat(ackCheckpointStatus.isAcknowledged(), is(true)); + assertThat(ackCheckpointStatus.isPositiveAcknowledgement(), is(true)); await() .atMost(Duration.ofSeconds(10)).untilAsserted(() -> verify(partitionCheckpoint).giveUpPartition()); @@ -169,7 +169,7 @@ public void createAcknowledgementSet_enabled_ackSetWithNoAck() { consumer.accept(false); final ConcurrentHashMap ackStatus = streamAckManager.getAcknowledgementStatus(); final CheckpointStatus ackCheckpointStatus = ackStatus.get(resumeToken); - assertThat(ackCheckpointStatus.isAcknowledged(), is(false)); + assertThat(ackCheckpointStatus.isPositiveAcknowledgement(), is(false)); await() .atMost(Duration.ofSeconds(10)).untilAsserted(() -> verify(stopWorkerConsumer).accept(null)); From 7d16ea1ecf489328d61fb308f94c6811ad12e558 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 19 Jun 2024 17:08:12 -0500 Subject: [PATCH 028/159] Support plugins defining the EventKey in the plugin configuration classes. Data Prepper will deserialize the EventKey from the pipeline configuration and validate @NotEmpty validations. Builds on the #1916. (#4635) Signed-off-by: David Venable --- .../model/event/EventKeyConfiguration.java | 34 +++++ .../integration/ProcessorPipelineIT.java | 3 + .../plugins/SimpleCopyProcessor.java | 51 +++++++ .../plugins/SimpleCopyProcessorConfig.java | 24 +++ .../dataprepper/plugins/SimpleProcessor.java | 7 +- .../plugins/SimpleProcessorConfig.java | 9 +- .../pipeline/processor-pipeline.yaml | 3 + data-prepper-pipeline-parser/build.gradle | 1 + .../NotEmptyValidatorForEventKey.java | 22 +++ .../pipeline/parser/EventKeyDeserializer.java | 60 ++++++++ .../jakarta.validation.ConstraintValidator | 6 + .../NotEmptyValidatorForEventKeyTest.java | 50 ++++++ .../parser/EventKeyDeserializerTest.java | 142 ++++++++++++++++++ .../plugin/ObjectMapperConfiguration.java | 8 +- .../plugin/ObjectMapperConfigurationTest.java | 21 ++- 15 files changed, 431 insertions(+), 10 deletions(-) create mode 100644 data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyConfiguration.java create mode 100644 data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessor.java create mode 100644 data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessorConfig.java create mode 100644 data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKey.java create mode 100644 data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializer.java create mode 100644 data-prepper-pipeline-parser/src/main/resources/META-INF/services/jakarta.validation.ConstraintValidator create mode 100644 data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKeyTest.java create mode 100644 data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyConfiguration.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyConfiguration.java new file mode 100644 index 0000000000..c35e8db38c --- /dev/null +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/EventKeyConfiguration.java @@ -0,0 +1,34 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +/** + * An annotation for an {@link EventKey} used in a Data Prepper pipeline configuration. + *

+ * Unless you need all actions on a configuration, you should use this annotation to + * provide the most appropriate validation. + * + * @since 2.9 + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.FIELD}) +public @interface EventKeyConfiguration { + /** + * Defines the {@link EventKeyFactory.EventAction}s to use when creating the {@link EventKey} + * for the configuration. + * + * @return The desired event actions. + * @since 2.9 + */ + EventKeyFactory.EventAction[] value(); +} diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java index 8aaf401e15..8673fd9f21 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/ProcessorPipelineIT.java @@ -79,6 +79,7 @@ void run_with_single_record() { assertThat(records.get(0).getData(), notNullValue()); assertThat(records.get(0).getData().get("message", String.class), equalTo(messageValue)); assertThat(records.get(0).getData().get("test1", String.class), equalTo("knownPrefix10")); + assertThat(records.get(0).getData().get("test1_copy", String.class), equalTo("knownPrefix10")); } @Test @@ -113,6 +114,8 @@ void pipeline_with_single_batch_of_records() { equalTo(inputRecord.getData().get("message", String.class))); assertThat(recordData.get("test1", String.class), equalTo("knownPrefix1" + i)); + assertThat(recordData.get("test1_copy", String.class), + equalTo("knownPrefix1" + i)); } } } diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessor.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessor.java new file mode 100644 index 0000000000..a786f09128 --- /dev/null +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessor.java @@ -0,0 +1,51 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins; + +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.processor.Processor; +import org.opensearch.dataprepper.model.record.Record; + +import java.util.Collection; + +@DataPrepperPlugin(name = "simple_copy_test", pluginType = Processor.class, pluginConfigurationType = SimpleCopyProcessorConfig.class) +public class SimpleCopyProcessor implements Processor, Record> { + private final SimpleCopyProcessorConfig simpleCopyProcessorConfig; + int count = 0; + + @DataPrepperPluginConstructor + public SimpleCopyProcessor(final SimpleCopyProcessorConfig simpleCopyProcessorConfig) { + this.simpleCopyProcessorConfig = simpleCopyProcessorConfig; + } + + @Override + public Collection> execute(final Collection> records) { + for (final Record record : records) { + final Object value = record.getData().get(simpleCopyProcessorConfig.getSource(), Object.class); + record.getData().put(simpleCopyProcessorConfig.getTarget(), value); + count++; + } + + return records; + } + + @Override + public void prepareForShutdown() { + + } + + @Override + public boolean isReadyForShutdown() { + return false; + } + + @Override + public void shutdown() { + + } +} diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessorConfig.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessorConfig.java new file mode 100644 index 0000000000..ded7f6212f --- /dev/null +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleCopyProcessorConfig.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins; + +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +public class SimpleCopyProcessorConfig { + @EventKeyConfiguration(EventKeyFactory.EventAction.GET) + private EventKey source; + private EventKey target; + + public EventKey getSource() { + return source; + } + + public EventKey getTarget() { + return target; + } +} diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java index bc59deb138..b0450d06d1 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessor.java @@ -9,7 +9,6 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventKey; -import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -22,10 +21,8 @@ public class SimpleProcessor implements Processor, Record> int count = 0; @DataPrepperPluginConstructor - public SimpleProcessor( - final SimpleProcessorConfig simpleProcessorConfig, - final EventKeyFactory eventKeyFactory) { - eventKey1 = eventKeyFactory.createEventKey(simpleProcessorConfig.getKey1()); + public SimpleProcessor(final SimpleProcessorConfig simpleProcessorConfig) { + eventKey1 = simpleProcessorConfig.getKey1(); valuePrefix1 = simpleProcessorConfig.getValuePrefix1(); } diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java index 957202431f..932d91c936 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/SimpleProcessorConfig.java @@ -5,11 +5,16 @@ package org.opensearch.dataprepper.plugins; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + public class SimpleProcessorConfig { - private String key1; + @EventKeyConfiguration(EventKeyFactory.EventAction.PUT) + private EventKey key1; private String valuePrefix1; - public String getKey1() { + public EventKey getKey1() { return key1; } diff --git a/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml index 2223a07c3e..be0e18a283 100644 --- a/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml +++ b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/processor-pipeline.yaml @@ -8,6 +8,9 @@ processor-pipeline: - simple_test: key1: /test1 value_prefix1: knownPrefix1 + - simple_copy_test: + source: /test1 + target: /test1_copy sink: - in_memory: diff --git a/data-prepper-pipeline-parser/build.gradle b/data-prepper-pipeline-parser/build.gradle index 09c89eb15c..53b27d1e99 100644 --- a/data-prepper-pipeline-parser/build.gradle +++ b/data-prepper-pipeline-parser/build.gradle @@ -18,6 +18,7 @@ dependencies { implementation 'org.projectlombok:lombok:1.18.22' implementation 'com.jayway.jsonpath:json-path:2.6.0' implementation 'javax.inject:javax.inject:1' + implementation 'javax.annotation:javax.annotation-api:1.3.2' implementation(libs.spring.core) { exclude group: 'commons-logging', module: 'commons-logging' } diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKey.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKey.java new file mode 100644 index 0000000000..507d2e9637 --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKey.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.core.validators; + +import jakarta.validation.ConstraintValidator; +import jakarta.validation.ConstraintValidatorContext; +import org.opensearch.dataprepper.model.event.EventKey; + +import jakarta.validation.constraints.NotEmpty; + +public class NotEmptyValidatorForEventKey implements ConstraintValidator { + @Override + public boolean isValid(final EventKey eventKey, final ConstraintValidatorContext constraintValidatorContext) { + if(eventKey == null) { + return false; + } + return !eventKey.getKey().isEmpty(); + } +} diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializer.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializer.java new file mode 100644 index 0000000000..fbc27edc8b --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializer.java @@ -0,0 +1,60 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.pipeline.parser; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.deser.ContextualDeserializer; +import com.fasterxml.jackson.databind.deser.std.StdDeserializer; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import java.io.IOException; + +public class EventKeyDeserializer extends StdDeserializer implements ContextualDeserializer { + private final EventKeyFactory eventKeyFactory; + private final EventKeyFactory.EventAction[] eventAction; + + /** + * Constructs a new {@link EventKeyDeserializer} from an {@link EventKeyFactory}. + * + * @param eventKeyFactory The factory for creating {@link EventKey} objects. + */ + public EventKeyDeserializer(final EventKeyFactory eventKeyFactory) { + this(eventKeyFactory, new EventKeyFactory.EventAction[] {EventKeyFactory.EventAction.ALL}); + } + + private EventKeyDeserializer(final EventKeyFactory eventKeyFactory, final EventKeyFactory.EventAction[] eventAction) { + super(EventKey.class); + this.eventKeyFactory = eventKeyFactory; + this.eventAction = eventAction; + } + + @Override + public EventKey deserialize(final JsonParser parser, final DeserializationContext ctxt) throws IOException { + final String eventKeyString = parser.getValueAsString(); + + return eventKeyFactory.createEventKey(eventKeyString, eventAction); + } + + @Override + public JsonDeserializer createContextual(final DeserializationContext deserializationContext, final BeanProperty property) { + if(property == null) + return this; + + final EventKeyConfiguration eventKeyConfiguration = property.getAnnotation(EventKeyConfiguration.class); + + if(eventKeyConfiguration == null) + return this; + + final EventKeyFactory.EventAction[] eventAction = eventKeyConfiguration.value(); + + return new EventKeyDeserializer(eventKeyFactory, eventAction); + } +} diff --git a/data-prepper-pipeline-parser/src/main/resources/META-INF/services/jakarta.validation.ConstraintValidator b/data-prepper-pipeline-parser/src/main/resources/META-INF/services/jakarta.validation.ConstraintValidator new file mode 100644 index 0000000000..ab6fb40c08 --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/resources/META-INF/services/jakarta.validation.ConstraintValidator @@ -0,0 +1,6 @@ +# +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +# + +org.opensearch.dataprepper.core.validators.NotEmptyValidatorForEventKey \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKeyTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKeyTest.java new file mode 100644 index 0000000000..d49ca2c161 --- /dev/null +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/core/validators/NotEmptyValidatorForEventKeyTest.java @@ -0,0 +1,50 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.core.validators; + +import jakarta.validation.ConstraintValidatorContext; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.event.EventKey; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class NotEmptyValidatorForEventKeyTest { + @Mock + private EventKey eventKey; + + @Mock + private ConstraintValidatorContext context; + + private NotEmptyValidatorForEventKey createObjectUnderTest() { + return new NotEmptyValidatorForEventKey(); + } + + @Test + void isValid_returns_false_if_EventKey_is_empty() { + assertThat(createObjectUnderTest().isValid(null, context), equalTo(false)); + } + + @Test + void isValid_returns_false_if_EventKey_getKey_is_empty() { + when(eventKey.getKey()).thenReturn(""); + assertThat(createObjectUnderTest().isValid(eventKey, context), equalTo(false)); + } + + @ParameterizedTest + @ValueSource(strings = {"/", "a", "/abcdefghijklmnopqrstuvwxyz"}) + void isValid_returns_true_if_EventKey_getKey_is_not_empty(final String key) { + when(eventKey.getKey()).thenReturn(key); + assertThat(createObjectUnderTest().isValid(eventKey, context), equalTo(true)); + } +} \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java new file mode 100644 index 0000000000..c727f0529a --- /dev/null +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java @@ -0,0 +1,142 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.pipeline.parser; + +import com.fasterxml.jackson.core.JsonParser; +import com.fasterxml.jackson.databind.BeanProperty; +import com.fasterxml.jackson.databind.DeserializationContext; +import com.fasterxml.jackson.databind.JsonDeserializer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +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.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import java.io.IOException; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.powermock.api.mockito.PowerMockito.mock; +import static org.powermock.api.mockito.PowerMockito.when; + +@ExtendWith(MockitoExtension.class) +class EventKeyDeserializerTest { + + @Mock + private EventKeyFactory eventKeyFactory; + + @Mock + private DeserializationContext deserializationContext; + @Mock + private BeanProperty property; + @Mock(lenient = true) + private JsonParser parser; + @Mock + private EventKey eventKey; + + private String eventKeyString; + + @BeforeEach + void setUp() throws IOException { + eventKeyString = UUID.randomUUID().toString(); + + when(parser.getValueAsString()).thenReturn(eventKeyString); + } + + private EventKeyDeserializer createObjectUnderTest() { + return new EventKeyDeserializer(eventKeyFactory); + } + + @Test + void createContextual_returns_EventKeyDeserializer_that_deserializes_with_ALL_when_no_BeanProperty() throws IOException { + when(eventKeyFactory.createEventKey(eventKeyString, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey); + final JsonDeserializer contextualDeserializer = createObjectUnderTest().createContextual(deserializationContext, null); + assertThat(contextualDeserializer, notNullValue()); + assertThat(contextualDeserializer.deserialize(parser, deserializationContext), equalTo(eventKey)); + } + + @Test + void createContextual_returns_EventKeyDeserializer_that_deserializes_with_ALL_when_no_annotation() throws IOException { + when(eventKeyFactory.createEventKey(eventKeyString, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey); + final JsonDeserializer contextualDeserializer = createObjectUnderTest().createContextual(deserializationContext, property); + assertThat(contextualDeserializer, notNullValue()); + assertThat(contextualDeserializer.deserialize(parser, deserializationContext), equalTo(eventKey)); + } + + @Test + void createContextual_returns_same_EventKeyDeserializer_as_self_when_no_BeanProperty() { + final EventKeyDeserializer objectUnderTest = createObjectUnderTest(); + final JsonDeserializer contextualDeserializer = objectUnderTest.createContextual(deserializationContext, null); + assertThat(contextualDeserializer, sameInstance(objectUnderTest)); + } + + @Test + void createContextual_returns_same_EventKeyDeserializer_as_self_when_no_annotation() { + final EventKeyDeserializer objectUnderTest = createObjectUnderTest(); + final JsonDeserializer contextualDeserializer = objectUnderTest.createContextual(deserializationContext, property); + assertThat(contextualDeserializer, sameInstance(objectUnderTest)); + } + + @ParameterizedTest + @EnumSource(value = EventKeyFactory.EventAction.class) + void createContextual_returns_EventKeyDeserializer_that_deserializes_with_action_from_annotated_Event(final EventKeyFactory.EventAction eventAction) throws IOException { + final EventKeyConfiguration eventKeyConfiguration = mock(EventKeyConfiguration.class); + when(eventKeyConfiguration.value()).thenReturn(new EventKeyFactory.EventAction[] { eventAction }); + when(property.getAnnotation(EventKeyConfiguration.class)).thenReturn(eventKeyConfiguration); + when(eventKeyFactory.createEventKey(eventKeyString, eventAction)).thenReturn(eventKey); + + final JsonDeserializer contextualDeserializer = createObjectUnderTest().createContextual(deserializationContext, property); + + assertThat(contextualDeserializer, notNullValue()); + assertThat(contextualDeserializer.deserialize(parser, deserializationContext), equalTo(eventKey)); + } + + @Test + void createContextual_returns_EventKeyDeserializer_that_deserializes_with_action_from_annotated_Event_when_multiple() throws IOException { + final EventKeyConfiguration eventKeyConfiguration = mock(EventKeyConfiguration.class); + when(eventKeyConfiguration.value()).thenReturn(new EventKeyFactory.EventAction[] { EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.DELETE }); + when(property.getAnnotation(EventKeyConfiguration.class)).thenReturn(eventKeyConfiguration); + when(eventKeyFactory.createEventKey(eventKeyString, EventKeyFactory.EventAction.PUT, EventKeyFactory.EventAction.DELETE)).thenReturn(eventKey); + + final JsonDeserializer contextualDeserializer = createObjectUnderTest().createContextual(deserializationContext, property); + + assertThat(contextualDeserializer, notNullValue()); + assertThat(contextualDeserializer.deserialize(parser, deserializationContext), equalTo(eventKey)); + } + + @Nested + class UsingRealObjectMapper { + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + + final SimpleModule simpleModule = new SimpleModule(); + simpleModule.addDeserializer(EventKey.class, createObjectUnderTest()); + objectMapper.registerModule(simpleModule); + } + + @Test + void quick() { + when(eventKeyFactory.createEventKey(eventKeyString, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey); + + assertThat(objectMapper.convertValue(eventKeyString, EventKey.class), + equalTo(eventKey)); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ObjectMapperConfiguration.java b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ObjectMapperConfiguration.java index 5865d5b29a..ca2cea4ee8 100644 --- a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ObjectMapperConfiguration.java +++ b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ObjectMapperConfiguration.java @@ -8,9 +8,12 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.PropertyNamingStrategies; import com.fasterxml.jackson.databind.module.SimpleModule; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.types.ByteCount; import org.opensearch.dataprepper.pipeline.parser.ByteCountDeserializer; import org.opensearch.dataprepper.pipeline.parser.DataPrepperDurationDeserializer; +import org.opensearch.dataprepper.pipeline.parser.EventKeyDeserializer; import org.springframework.context.annotation.Bean; import javax.inject.Named; @@ -38,10 +41,13 @@ ObjectMapper extensionPluginConfigObjectMapper() { } @Bean(name = "pluginConfigObjectMapper") - ObjectMapper pluginConfigObjectMapper(final VariableExpander variableExpander) { + ObjectMapper pluginConfigObjectMapper( + final VariableExpander variableExpander, + final EventKeyFactory eventKeyFactory) { final SimpleModule simpleModule = new SimpleModule(); simpleModule.addDeserializer(Duration.class, new DataPrepperDurationDeserializer()); simpleModule.addDeserializer(ByteCount.class, new ByteCountDeserializer()); + simpleModule.addDeserializer(EventKey.class, new EventKeyDeserializer(eventKeyFactory)); TRANSLATE_VALUE_SUPPORTED_JAVA_TYPES.stream().forEach(clazz -> simpleModule.addDeserializer( clazz, new DataPrepperScalarTypeDeserializer<>(variableExpander, clazz))); diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ObjectMapperConfigurationTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ObjectMapperConfigurationTest.java index d839566680..594d3a47c2 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ObjectMapperConfigurationTest.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ObjectMapperConfigurationTest.java @@ -11,6 +11,8 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import java.time.Duration; import java.util.Arrays; @@ -20,6 +22,8 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) class ObjectMapperConfigurationTest { @@ -28,10 +32,13 @@ class ObjectMapperConfigurationTest { @Mock private VariableExpander variableExpander; + @Mock + private EventKeyFactory eventKeyFactory; + @Test void test_duration_with_pluginConfigObjectMapper() { final String durationTestString = "10s"; - final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander); + final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander, eventKeyFactory); final Duration duration = objectMapper.convertValue(durationTestString, Duration.class); assertThat(duration, equalTo(Duration.ofSeconds(10))); } @@ -39,7 +46,7 @@ void test_duration_with_pluginConfigObjectMapper() { @Test void test_enum_with_pluginConfigObjectMapper() { final String testString = "test"; - final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander); + final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander, eventKeyFactory); final TestType duration = objectMapper.convertValue(testString, TestType.class); assertThat(duration, equalTo(TestType.fromOptionValue(testString))); } @@ -60,6 +67,16 @@ void test_enum_with_extensionPluginConfigObjectMapper() { assertThat(duration, equalTo(TestType.fromOptionValue(testString))); } + @Test + void test_eventKey_with_pluginConfigObjectMapper() { + final String testKey = "test"; + final EventKey eventKey = mock(EventKey.class); + when(eventKeyFactory.createEventKey(testKey, EventKeyFactory.EventAction.ALL)).thenReturn(eventKey); + final ObjectMapper objectMapper = objectMapperConfiguration.pluginConfigObjectMapper(variableExpander, eventKeyFactory); + final EventKey actualEventKey = objectMapper.convertValue(testKey, EventKey.class); + assertThat(actualEventKey, equalTo(eventKey)); + } + private enum TestType { TEST("test"); From 9db4bc49a6055185e5bd52dfef7042e258fcfdfb Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Thu, 20 Jun 2024 11:05:40 -0700 Subject: [PATCH 029/159] Add support to configure metric name for count and histogram actions (#4642) * rebased to latest Signed-off-by: Krishna Kondaka * renamed name to metric_name Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../aggregate/actions/CountAggregateAction.java | 5 +++-- .../aggregate/actions/CountAggregateActionConfig.java | 8 ++++++++ .../aggregate/actions/HistogramAggregateAction.java | 7 ++++--- .../actions/HistogramAggregateActionConfig.java | 8 ++++++++ .../processor/aggregate/AggregateProcessorIT.java | 2 +- .../actions/CountAggregateActionConfigTests.java | 4 ++++ .../aggregate/actions/CountAggregateActionTest.java | 10 ++++++++-- .../actions/HistogramAggregateActionConfigTests.java | 6 ++++++ .../actions/HistogramAggregateActionTests.java | 10 ++++++---- 9 files changed, 48 insertions(+), 12 deletions(-) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java index 30ccccb2d5..f87cd5a7f0 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java @@ -40,7 +40,6 @@ public class CountAggregateAction implements AggregateAction { private static final String DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"; private static final String exemplarKey = "__exemplar"; static final String EVENT_TYPE = "event"; - static final String SUM_METRIC_NAME = "count"; static final String SUM_METRIC_DESCRIPTION = "Number of events"; static final String SUM_METRIC_UNIT = "1"; static final boolean SUM_METRIC_IS_MONOTONIC = true; @@ -49,6 +48,7 @@ public class CountAggregateAction implements AggregateAction { public final String endTimeKey; public final String outputFormat; private long startTimeNanos; + private final String metricName; @DataPrepperPluginConstructor public CountAggregateAction(final CountAggregateActionConfig countAggregateActionConfig) { @@ -56,6 +56,7 @@ public CountAggregateAction(final CountAggregateActionConfig countAggregateActio this.startTimeKey = countAggregateActionConfig.getStartTimeKey(); this.endTimeKey = countAggregateActionConfig.getEndTimeKey(); this.outputFormat = countAggregateActionConfig.getOutputFormat(); + this.metricName = countAggregateActionConfig.getMetricName(); } public Exemplar createExemplar(final Event event) { @@ -133,7 +134,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA Map attr = new HashMap(); groupState.forEach((k, v) -> attr.put((String)k, v)); JacksonSum sum = JacksonSum.builder() - .withName(SUM_METRIC_NAME) + .withName(this.metricName) .withDescription(SUM_METRIC_DESCRIPTION) .withTime(OTelProtoCodec.convertUnixNanosToISO8601(endTimeNanos)) .withStartTime(OTelProtoCodec.convertUnixNanosToISO8601(startTimeNanos)) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java index 3577e035c4..f7a2e6a48d 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java @@ -10,6 +10,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; public class CountAggregateActionConfig { + static final String SUM_METRIC_NAME = "count"; public static final String DEFAULT_COUNT_KEY = "aggr._count"; public static final String DEFAULT_START_TIME_KEY = "aggr._start_time"; public static final String DEFAULT_END_TIME_KEY = "aggr._end_time"; @@ -18,6 +19,9 @@ public class CountAggregateActionConfig { @JsonProperty("count_key") String countKey = DEFAULT_COUNT_KEY; + @JsonProperty("metric_name") + String metricName = SUM_METRIC_NAME; + @JsonProperty("start_time_key") String startTimeKey = DEFAULT_START_TIME_KEY; @@ -27,6 +31,10 @@ public class CountAggregateActionConfig { @JsonProperty("output_format") String outputFormat = OutputFormat.OTEL_METRICS.toString(); + public String getMetricName() { + return metricName; + } + public String getCountKey() { return countKey; } diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java index fd4d7542f5..bdb9a3fad6 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java @@ -44,7 +44,6 @@ public class HistogramAggregateAction implements AggregateAction { private static final String DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"; private static final String EVENT_TYPE = "event"; - public static final String HISTOGRAM_METRIC_NAME = "histogram"; private final String countKey; private final String bucketCountsKey; private final String bucketsKey; @@ -62,6 +61,7 @@ public class HistogramAggregateAction implements AggregateAction { private Event maxEvent; private double minValue; private double maxValue; + private final String metricName; private long startTimeNanos; private double[] buckets; @@ -72,6 +72,7 @@ public HistogramAggregateAction(final HistogramAggregateActionConfig histogramAg List bucketList = histogramAggregateActionConfig.getBuckets(); this.buckets = new double[bucketList.size()+2]; int bucketIdx = 0; + this.metricName = histogramAggregateActionConfig.getMetricName(); this.buckets[bucketIdx++] = -Float.MAX_VALUE; for (int i = 0; i < bucketList.size(); i++) { this.buckets[bucketIdx++] = convertToDouble(bucketList.get(i)); @@ -212,7 +213,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA Instant endTime = (Instant)groupState.get(endTimeKey); long startTimeNanos = getTimeNanos(startTime); long endTimeNanos = getTimeNanos(endTime); - String histogramKey = HISTOGRAM_METRIC_NAME + "_key"; + String histogramKey = this.metricName + "_key"; List exemplarList = new ArrayList<>(); exemplarList.add(createExemplar("min", minEvent, minValue)); exemplarList.add(createExemplar("max", maxEvent, maxValue)); @@ -245,7 +246,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA Integer count = (Integer)groupState.get(countKey); String description = String.format("Histogram of %s in the events", key); JacksonHistogram histogram = JacksonHistogram.builder() - .withName(HISTOGRAM_METRIC_NAME) + .withName(this.metricName) .withDescription(description) .withTime(OTelProtoCodec.convertUnixNanosToISO8601(endTimeNanos)) .withStartTime(OTelProtoCodec.convertUnixNanosToISO8601(startTimeNanos)) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java index a173671836..7c998c123d 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java @@ -12,6 +12,7 @@ import jakarta.validation.constraints.NotNull; public class HistogramAggregateActionConfig { + public static final String HISTOGRAM_METRIC_NAME = "histogram"; public static final String DEFAULT_GENERATED_KEY_PREFIX = "aggr._"; public static final String SUM_KEY = "sum"; public static final String COUNT_KEY = "count"; @@ -32,6 +33,9 @@ public class HistogramAggregateActionConfig { @NotNull String units; + @JsonProperty("metric_name") + String metricName = HISTOGRAM_METRIC_NAME; + @JsonProperty("generated_key_prefix") String generatedKeyPrefix = DEFAULT_GENERATED_KEY_PREFIX; @@ -45,6 +49,10 @@ public class HistogramAggregateActionConfig { @JsonProperty("record_minmax") boolean recordMinMax = false; + public String getMetricName() { + return metricName; + } + public boolean getRecordMinMax() { return recordMinMax; } diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java index ea7b6eb416..fc416b0e45 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java @@ -533,7 +533,7 @@ void aggregateWithHistogramAggregateAction() throws InterruptedException, NoSuch countDownLatch.countDown(); }); } - Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1000); + Thread.sleep(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE * 1500); boolean allThreadsFinished = countDownLatch.await(5L, TimeUnit.SECONDS); assertThat(allThreadsFinished, equalTo(true)); diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java index f022ac9148..53b7c846b6 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java @@ -38,6 +38,7 @@ void testDefault() { assertThat(countAggregateActionConfig.getCountKey(), equalTo(DEFAULT_COUNT_KEY)); assertThat(countAggregateActionConfig.getStartTimeKey(), equalTo(DEFAULT_START_TIME_KEY)); assertThat(countAggregateActionConfig.getOutputFormat(), equalTo(OutputFormat.OTEL_METRICS.toString())); + assertThat(countAggregateActionConfig.getMetricName(), equalTo(CountAggregateActionConfig.SUM_METRIC_NAME)); } @Test @@ -51,6 +52,9 @@ void testValidConfig() throws NoSuchFieldException, IllegalAccessException { final String testOutputFormat = OutputFormat.OTEL_METRICS.toString(); setField(CountAggregateActionConfig.class, countAggregateActionConfig, "outputFormat", testOutputFormat); assertThat(countAggregateActionConfig.getOutputFormat(), equalTo(OutputFormat.OTEL_METRICS.toString())); + final String testName = UUID.randomUUID().toString(); + setField(CountAggregateActionConfig.class, countAggregateActionConfig, "metricName", testName); + assertThat(countAggregateActionConfig.getMetricName(), equalTo(testName)); } @Test diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java index f98fe0028a..b91311c1c6 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java @@ -50,6 +50,7 @@ private AggregateAction createObjectUnderTest(CountAggregateActionConfig config) @ParameterizedTest @ValueSource(ints = {1, 2, 10, 100}) void testCountAggregate(int testCount) throws NoSuchFieldException, IllegalAccessException { + final String testName = UUID.randomUUID().toString(); CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig(); setField(CountAggregateActionConfig.class, countAggregateActionConfig, "outputFormat", OutputFormat.RAW.toString()); countAggregateAction = createObjectUnderTest(countAggregateActionConfig); @@ -79,8 +80,10 @@ void testCountAggregate(int testCount) throws NoSuchFieldException, IllegalAcces @ParameterizedTest @ValueSource(ints = {1, 2, 10, 100}) - void testCountAggregateOTelFormat(int testCount) { + void testCountAggregateOTelFormat(int testCount) throws NoSuchFieldException, IllegalAccessException { CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig(); + final String testName = UUID.randomUUID().toString(); + setField(CountAggregateActionConfig.class, countAggregateActionConfig, "metricName", testName); countAggregateAction = createObjectUnderTest(countAggregateActionConfig); final String key1 = "key-"+UUID.randomUUID().toString(); final String value1 = UUID.randomUUID().toString(); @@ -119,6 +122,7 @@ void testCountAggregateOTelFormat(int testCount) { expectedEventMap.put("isMonotonic", true); expectedEventMap.put("aggregationTemporality", "AGGREGATION_TEMPORALITY_DELTA"); expectedEventMap.put("unit", "1"); + expectedEventMap.put("name", testName); expectedEventMap.forEach((k, v) -> assertThat(result.get(0).toMap(), hasEntry(k,v))); assertThat(result.get(0).toMap().get("attributes"), equalTo(eventMap)); JacksonMetric metric = (JacksonMetric) result.get(0); @@ -149,6 +153,8 @@ void testCountAggregateOTelFormat(int testCount) { void testCountAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCount) { CountAggregateActionConfig mockConfig = mock(CountAggregateActionConfig.class); when(mockConfig.getCountKey()).thenReturn(CountAggregateActionConfig.DEFAULT_COUNT_KEY); + final String testName = UUID.randomUUID().toString(); + when(mockConfig.getMetricName()).thenReturn(testName); String startTimeKey = UUID.randomUUID().toString(); String endTimeKey = UUID.randomUUID().toString(); when(mockConfig.getStartTimeKey()).thenReturn(startTimeKey); @@ -195,7 +201,7 @@ void testCountAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCount) { assertThat(result.size(), equalTo(1)); Map expectedEventMap = new HashMap<>(); expectedEventMap.put("value", (double)testCount); - expectedEventMap.put("name", "count"); + expectedEventMap.put("name", testName); expectedEventMap.put("description", "Number of events"); expectedEventMap.put("isMonotonic", true); expectedEventMap.put("aggregationTemporality", "AGGREGATION_TEMPORALITY_DELTA"); diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfigTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfigTests.java index f3e1e19d25..60ba8dc202 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfigTests.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfigTests.java @@ -11,6 +11,8 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.apache.commons.lang3.RandomStringUtils; +import java.util.UUID; + import static org.opensearch.dataprepper.plugins.processor.aggregate.actions.HistogramAggregateActionConfig.DEFAULT_GENERATED_KEY_PREFIX; import java.util.concurrent.ThreadLocalRandom; @@ -41,6 +43,7 @@ void testDefault() { assertThat(histogramAggregateActionConfig.getGeneratedKeyPrefix(), equalTo(DEFAULT_GENERATED_KEY_PREFIX)); assertThat(histogramAggregateActionConfig.getRecordMinMax(), equalTo(false)); assertThat(histogramAggregateActionConfig.getOutputFormat(), equalTo(OutputFormat.OTEL_METRICS.toString())); + assertThat(histogramAggregateActionConfig.getMetricName(), equalTo(HistogramAggregateActionConfig.HISTOGRAM_METRIC_NAME)); } @Test @@ -106,6 +109,9 @@ void testValidConfig() throws NoSuchFieldException, IllegalAccessException { longBuckets.add(longValue2); setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "buckets", longBuckets); assertThat(histogramAggregateActionConfig.getBuckets(), containsInAnyOrder(longBuckets.toArray())); + final String testName = UUID.randomUUID().toString(); + setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "metricName", testName); + assertThat(histogramAggregateActionConfig.getMetricName(), equalTo(testName)); } @Test diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java index 1f5ca5e52c..155acee918 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java @@ -198,7 +198,7 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException final String expectedStartTimeKey = histogramAggregateActionConfig.getStartTimeKey(); Map expectedEventMap = new HashMap<>(Collections.singletonMap("count", (long)testCount)); expectedEventMap.put("unit", testUnits); - expectedEventMap.put("name", HistogramAggregateAction.HISTOGRAM_METRIC_NAME); + expectedEventMap.put("name", HistogramAggregateActionConfig.HISTOGRAM_METRIC_NAME); expectedEventMap.put("sum", expectedSum); expectedEventMap.put("min", expectedMin); expectedEventMap.put("max", expectedMax); @@ -212,7 +212,7 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException for (int i = 0; i < expectedBucketCounts.length; i++) { assertThat(expectedBucketCounts[i], equalTo(bucketCountsFromResult.get(i))); } - assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(HistogramAggregateAction.HISTOGRAM_METRIC_NAME+"_key", testKey)); + assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(HistogramAggregateActionConfig.HISTOGRAM_METRIC_NAME+"_key", testKey)); List exemplars = (List )result.get(0).toMap().get("exemplars"); assertThat(exemplars.size(), equalTo(2)); assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(dataKey, dataValue)); @@ -250,6 +250,8 @@ void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCoun final String testKeyPrefix = RandomStringUtils.randomAlphabetic(5)+"_"; when(mockConfig.getStartTimeKey()).thenReturn(startTimeKey); when(mockConfig.getEndTimeKey()).thenReturn(endTimeKey); + final String testName = UUID.randomUUID().toString(); + when(mockConfig.getMetricName()).thenReturn(testName); when(mockConfig.getOutputFormat()).thenReturn(OutputFormat.OTEL_METRICS.toString()); String keyPrefix = UUID.randomUUID().toString(); final String testUnits = "ms"; @@ -323,7 +325,7 @@ void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCoun final String expectedStartTimeKey = mockConfig.getStartTimeKey(); Map expectedEventMap = new HashMap<>(Collections.singletonMap("count", (long)testCount)); expectedEventMap.put("unit", testUnits); - expectedEventMap.put("name", HistogramAggregateAction.HISTOGRAM_METRIC_NAME); + expectedEventMap.put("name", testName); expectedEventMap.put("sum", expectedSum); expectedEventMap.put("min", expectedMin); expectedEventMap.put("max", expectedMax); @@ -337,7 +339,7 @@ void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCoun for (int i = 0; i < expectedBucketCounts.length; i++) { assertThat(expectedBucketCounts[i], equalTo(bucketCountsFromResult.get(i))); } - assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(HistogramAggregateAction.HISTOGRAM_METRIC_NAME+"_key", testKey)); + assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(testName+"_key", testKey)); List exemplars = (List )result.get(0).toMap().get("exemplars"); assertThat(exemplars.size(), equalTo(2)); assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(dataKey, dataValue)); From 0aba83b7dda738e3701af6c8959261390027d252 Mon Sep 17 00:00:00 2001 From: timo-mue Date: Fri, 21 Jun 2024 17:04:04 +0200 Subject: [PATCH 030/159] add remove_brackets option to flatten-processor (#4616) (#4653) Signed-off-by: Timo Mueller --- .../processor/flatten/FlattenProcessor.java | 7 ++- .../flatten/FlattenProcessorConfig.java | 14 +++++ .../flatten/FlattenProcessorConfigTest.java | 2 +- .../flatten/FlattenProcessorTest.java | 61 +++++++++++++++++++ 4 files changed, 80 insertions(+), 4 deletions(-) diff --git a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessor.java b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessor.java index 2a07fd6d99..9e3218be88 100644 --- a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessor.java +++ b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessor.java @@ -119,14 +119,15 @@ private Map removeListIndicesInKeys(final Map in final Map resultMap = new HashMap<>(); for (final Map.Entry entry : inputMap.entrySet()) { - final String keyWithoutIndices = removeListIndices(entry.getKey()); + final String keyWithoutIndices = removeListIndices(entry.getKey(), config.isRemoveBrackets()); addFieldsToMapWithMerge(keyWithoutIndices, entry.getValue(), resultMap); } return resultMap; } - private String removeListIndices(final String key) { - return key.replaceAll("\\[\\d+\\]", "[]"); + private String removeListIndices(final String key, final boolean removeBrackets) { + final String replacement = removeBrackets ? "" : "[]"; + return key.replaceAll("\\[\\d+\\]", replacement); } private void addFieldsToMapWithMerge(String key, Object value, Map map) { diff --git a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java index 96c9d2e024..c1208f5f40 100644 --- a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java +++ b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java @@ -6,6 +6,8 @@ package org.opensearch.dataprepper.plugins.processor.flatten; import com.fasterxml.jackson.annotation.JsonProperty; + +import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotNull; import java.util.ArrayList; @@ -29,6 +31,9 @@ public class FlattenProcessorConfig { @JsonProperty("remove_list_indices") private boolean removeListIndices = false; + @JsonProperty("remove_brackets") + private boolean removeBrackets = false; + @JsonProperty("exclude_keys") private List excludeKeys = DEFAULT_EXCLUDE_KEYS; @@ -54,6 +59,10 @@ public boolean isRemoveListIndices() { return removeListIndices; } + public boolean isRemoveBrackets() { + return removeBrackets; + } + public List getExcludeKeys() { return excludeKeys; } @@ -65,4 +74,9 @@ public String getFlattenWhen() { public List getTagsOnFailure() { return tagsOnFailure; } + + @AssertTrue(message = "remove_brackets can not be true if remove_list_indices is false.") + boolean removeBracketsNotTrueWhenRemoveListIndicesFalse() { + return (!removeBrackets || removeListIndices); + } } diff --git a/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfigTest.java b/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfigTest.java index d11860df0e..960db201d5 100644 --- a/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfigTest.java +++ b/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfigTest.java @@ -20,7 +20,7 @@ void testDefaultConfig() { assertThat(FlattenProcessorConfig.getSource(), equalTo(null)); assertThat(FlattenProcessorConfig.getTarget(), equalTo(null)); assertThat(FlattenProcessorConfig.isRemoveListIndices(), equalTo(false)); - assertThat(FlattenProcessorConfig.isRemoveListIndices(), equalTo(false)); + assertThat(FlattenProcessorConfig.isRemoveBrackets(), equalTo(false)); assertThat(FlattenProcessorConfig.getFlattenWhen(), equalTo(null)); assertThat(FlattenProcessorConfig.getTagsOnFailure(), equalTo(null)); assertThat(FlattenProcessorConfig.getExcludeKeys(), equalTo(List.of())); diff --git a/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorTest.java b/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorTest.java index 737d245ff5..df693f7f6f 100644 --- a/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorTest.java +++ b/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorTest.java @@ -52,6 +52,7 @@ void setUp() { lenient().when(mockConfig.getTarget()).thenReturn(""); lenient().when(mockConfig.isRemoveProcessedFields()).thenReturn(false); lenient().when(mockConfig.isRemoveListIndices()).thenReturn(false); + lenient().when(mockConfig.isRemoveBrackets()).thenReturn(false); lenient().when(mockConfig.getFlattenWhen()).thenReturn(null); lenient().when(mockConfig.getTagsOnFailure()).thenReturn(new ArrayList<>()); lenient().when(mockConfig.getExcludeKeys()).thenReturn(new ArrayList<>()); @@ -119,6 +120,35 @@ void testFlattenEntireEventDataAndRemoveListIndices() { assertThat(resultData.get("list1[].list2[].value"), is(List.of("value1", "value2"))); } + @Test + void testFlattenEntireEventDataAndRemoveListIndicesAndRemoveBrackets() { + when(mockConfig.isRemoveListIndices()).thenReturn(true); + when(mockConfig.isRemoveBrackets()).thenReturn(true); + + final FlattenProcessor processor = createObjectUnderTest(); + final Record testRecord = createTestRecord(createTestData()); + final List> resultRecord = (List>) processor.doExecute(Collections.singletonList(testRecord)); + + assertThat(resultRecord.size(), is(1)); + + final Event resultEvent = resultRecord.get(0).getData(); + Map resultData = resultEvent.get("", Map.class); + + assertThat(resultData.containsKey("key1"), is(true)); + assertThat(resultData.get("key1"), is("val1")); + + assertThat(resultData.containsKey("key1"), is(true)); + assertThat(resultData.get("key2.key3.key.4"), is("val2")); + + assertThat(resultData.containsKey("list1[].list2[].name"), is(false)); + assertThat(resultData.containsKey("list1.list2.name"), is(true)); + assertThat(resultData.get("list1.list2.name"), is(List.of("name1", "name2"))); + + assertThat(resultData.containsKey("list1[].list2[].value"), is(false)); + assertThat(resultData.containsKey("list1.list2.value"), is(true)); + assertThat(resultData.get("list1.list2.value"), is(List.of("value1", "value2"))); + } + @Test void testFlattenWithSpecificFieldsAsSourceAndTarget() { when(mockConfig.getSource()).thenReturn(SOURCE_KEY); @@ -187,6 +217,37 @@ void testFlattenWithSpecificFieldsAsSourceAndTargetAndRemoveListIndices() { assertThat(resultData.get("list1[].list2[].value"), is(List.of("value1", "value2"))); } + @Test + void testFlattenWithSpecificFieldsAsSourceAndTargetAndRemoveListIndicesAndRemoveBrackets() { + when(mockConfig.getSource()).thenReturn(SOURCE_KEY); + when(mockConfig.getTarget()).thenReturn(TARGET_KEY); + when(mockConfig.isRemoveListIndices()).thenReturn(true); + when(mockConfig.isRemoveBrackets()).thenReturn(true); + + final FlattenProcessor processor = createObjectUnderTest(); + final Record testRecord = createTestRecord(Map.of(SOURCE_KEY, createTestData())); + final List> resultRecord = (List>) processor.doExecute(Collections.singletonList(testRecord)); + + assertThat(resultRecord.size(), is(1)); + + final Event resultEvent = resultRecord.get(0).getData(); + Map resultData = resultEvent.get(TARGET_KEY, Map.class); + + assertThat(resultData.containsKey("key1"), is(true)); + assertThat(resultData.get("key1"), is("val1")); + + assertThat(resultData.containsKey("key1"), is(true)); + assertThat(resultData.get("key2.key3.key.4"), is("val2")); + + assertThat(resultData.containsKey("list1[].list2[].name"), is(false)); + assertThat(resultData.containsKey("list1.list2.name"), is(true)); + assertThat(resultData.get("list1.list2.name"), is(List.of("name1", "name2"))); + + assertThat(resultData.containsKey("list1[].list2[].value"), is(false)); + assertThat(resultData.containsKey("list1.list2.value"), is(true)); + assertThat(resultData.get("list1.list2.value"), is(List.of("value1", "value2"))); + } + @Test public void testEventNotProcessedWhenTheWhenConditionIsFalse() { final String whenCondition = UUID.randomUUID().toString(); From b4b71e279a8bad9310d57c3cc476bd1536a7a0d2 Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 21 Jun 2024 10:37:03 -0500 Subject: [PATCH 031/159] Fixes performance regression with JacksonEvent put/delete operations. (#4650) With the addition of the EventKey, JacksonEvent always creates a JacksonEventKey in order to use the same code for all paths. However, when put/delete calls are made with a String key, JacksonEvent does not need the JSON Pointer. But, it is created anyway. This adds more work to the put/delete calls that have not yet migrated to the String version. This fixes regression by adding a lazy initialization option when used in JacksonEvent. We should not be lazy when used with the EventKeyFactory since we may lose some up-front validations. Signed-off-by: David Venable --- .../dataprepper/model/event/JacksonEvent.java | 14 ++--- .../model/event/JacksonEventKey.java | 57 +++++++++++++++---- .../model/event/JacksonEventKeyTest.java | 17 ++++++ 3 files changed, 71 insertions(+), 17 deletions(-) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java index 35e0dd863b..25ef31ec8b 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEvent.java @@ -170,7 +170,7 @@ public void put(EventKey key, Object value) { */ @Override public void put(final String key, final Object value) { - final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.PUT); + final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.PUT); put(jacksonEventKey, value); } @@ -229,7 +229,7 @@ private static JacksonEventKey asJacksonEventKey(EventKey key) { */ @Override public T get(final String key, final Class clazz) { - final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET); return get(jacksonEventKey, clazz); } @@ -274,7 +274,7 @@ public List getList(EventKey key, Class clazz) { */ @Override public List getList(final String key, final Class clazz) { - JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET); return getList(jacksonEventKey, clazz); } @@ -330,7 +330,7 @@ public void delete(final EventKey key) { */ @Override public void delete(final String key) { - final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.DELETE); + final JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.DELETE); delete(jacksonEventKey); } @@ -362,7 +362,7 @@ public String getAsJsonString(EventKey key) { @Override public String getAsJsonString(final String key) { - JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET); return getAsJsonString(jacksonEventKey); } @@ -459,7 +459,7 @@ public boolean containsKey(EventKey key) { @Override public boolean containsKey(final String key) { - JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET); return containsKey(jacksonEventKey); } @@ -474,7 +474,7 @@ public boolean isValueAList(EventKey key) { @Override public boolean isValueAList(final String key) { - JacksonEventKey jacksonEventKey = new JacksonEventKey(key, EventKeyFactory.EventAction.GET); + JacksonEventKey jacksonEventKey = new JacksonEventKey(key, true, EventKeyFactory.EventAction.GET); return isValueAList(jacksonEventKey); } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java index 2df755492a..50d59a6585 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/JacksonEventKey.java @@ -22,14 +22,45 @@ class JacksonEventKey implements EventKey { private final String key; private final EventKeyFactory.EventAction[] eventActions; private final String trimmedKey; - private final List keyPathList; - private final JsonPointer jsonPointer; + private List keyPathList; + private JsonPointer jsonPointer; private final Set supportedActions; + /** + * Constructor for the JacksonEventKey which should only be used by implementation + * of {@link EventKeyFactory} in Data Prepper core. + * + * @param key The key + * @param eventActions Event actions to support + */ JacksonEventKey(final String key, final EventKeyFactory.EventAction... eventActions) { + this(key, false, eventActions); + } + + /** + * Constructs a new JacksonEventKey. + *

+ * This overload should only be used by {@link JacksonEvent} directly. It allows for skipping creating + * some resources knowing they will not be needed. The {@link JacksonEvent} only needs a JSON pointer + * when performing GET event actions. So we can optimize PUT/DELETE actions when called with a string + * key instead of an EventKey by not creating the JSON Pointer at all. + *

+ * For EventKey's constructed through the factory, we should not perform lazy initialization since + * we may lose some possible validations. + * + * @param key the key + * @param lazy Use true to lazily initialize. This will not be thread-safe, however. + * @param eventActions Event actions to support + */ + JacksonEventKey(final String key, final boolean lazy, final EventKeyFactory.EventAction... eventActions) { this.key = Objects.requireNonNull(key, "Parameter key cannot be null for EventKey."); this.eventActions = eventActions.length == 0 ? new EventKeyFactory.EventAction[] { EventKeyFactory.EventAction.ALL } : eventActions; + supportedActions = EnumSet.noneOf(EventKeyFactory.EventAction.class); + for (final EventKeyFactory.EventAction eventAction : this.eventActions) { + supportedActions.addAll(eventAction.getSupportedActions()); + } + if(key.isEmpty()) { for (final EventKeyFactory.EventAction action : this.eventActions) { if (action.isMutableAction()) { @@ -40,14 +71,10 @@ class JacksonEventKey implements EventKey { trimmedKey = checkAndTrimKey(key); - keyPathList = Collections.unmodifiableList(Arrays.asList(trimmedKey.split(SEPARATOR, -1))); - jsonPointer = toJsonPointer(trimmedKey); - - supportedActions = EnumSet.noneOf(EventKeyFactory.EventAction.class); - for (final EventKeyFactory.EventAction eventAction : this.eventActions) { - supportedActions.addAll(eventAction.getSupportedActions()); + if(!lazy) { + keyPathList = toKeyPathList(); + jsonPointer = toJsonPointer(trimmedKey); } - } @Override @@ -60,10 +87,16 @@ String getTrimmedKey() { } List getKeyPathList() { + if(keyPathList == null) { + keyPathList = toKeyPathList(); + } return keyPathList; } JsonPointer getJsonPointer() { + if(jsonPointer == null) { + jsonPointer = toJsonPointer(trimmedKey); + } return jsonPointer; } @@ -136,7 +169,11 @@ private static boolean isValidKey(final String key) { return true; } - private JsonPointer toJsonPointer(final String key) { + private List toKeyPathList() { + return Collections.unmodifiableList(Arrays.asList(trimmedKey.split(SEPARATOR, -1))); + } + + private static JsonPointer toJsonPointer(final String key) { final String jsonPointerExpression; if (key.isEmpty() || key.startsWith("/")) { jsonPointerExpression = key; diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java index 929151175b..5eb696a374 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/JacksonEventKeyTest.java @@ -115,6 +115,23 @@ void getJsonPointer_returns_the_same_instance_for_multiple_calls() { assertThat(objectUnderTest.getJsonPointer(), sameInstance(jsonPointer)); } + @ParameterizedTest + @EnumSource(value = EventKeyFactory.EventAction.class) + void getJsonPointer_returns_valid_JsonPointer_when_constructed_with_fromJacksonEvent(final EventKeyFactory.EventAction eventAction) { + final String testKey = UUID.randomUUID().toString(); + final JacksonEventKey objectUnderTest = new JacksonEventKey(testKey, true, eventAction); + + final JsonPointer jsonPointer = objectUnderTest.getJsonPointer(); + assertThat(jsonPointer, notNullValue()); + assertThat(jsonPointer.toString(), equalTo("/" + testKey)); + } + + @ParameterizedTest + @ArgumentsSource(KeyPathListArgumentsProvider.class) + void getKeyPathList_returns_expected_value_when_constructed_with_fromJacksonEvent(final String key, final List expectedKeyPathList) { + assertThat(new JacksonEventKey(key, true).getKeyPathList(), equalTo(expectedKeyPathList)); + } + @ParameterizedTest @ArgumentsSource(SupportsArgumentsProvider.class) void supports_returns_true_if_any_supports(final List eventActionsList, final EventKeyFactory.EventAction otherAction, final boolean expectedSupports) { From bbac579ba58b2399136e376d8cd016f04f241791 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Mon, 24 Jun 2024 16:56:23 -0500 Subject: [PATCH 032/159] MAINT: change log level for consumer properties in kafka source (#4658) Signed-off-by: George Chen --- .../dataprepper/plugins/kafka/source/KafkaSource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 6a01a91bf0..9a385c836f 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 @@ -241,7 +241,7 @@ private Properties getConsumerProperties(final TopicConsumerConfig topicConfig, } setConsumerTopicProperties(properties, topicConfig); setSchemaRegistryProperties(properties, topicConfig); - LOG.info("Starting consumer with the properties : {}", properties); + LOG.debug("Starting consumer with the properties : {}", properties); return properties; } From ab9700a70e4ff512ad456f6b9895d66c2d3e9bdc Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Mon, 24 Jun 2024 15:46:42 -0700 Subject: [PATCH 033/159] Add an option to count unique values of specified key(s) to CountAggregateAction (#4652) Add an option to count unique values of specified key(s) to CountAggregateAction Signed-off-by: Krishna Kondaka --- .../actions/CountAggregateAction.java | 28 +++++++++- .../actions/CountAggregateActionConfig.java | 10 +++- .../CountAggregateActionConfigTests.java | 8 +++ .../actions/CountAggregateActionTest.java | 54 +++++++++++++++++++ 4 files changed, 97 insertions(+), 3 deletions(-) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java index f87cd5a7f0..c8fd772336 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateAction.java @@ -22,13 +22,17 @@ import static org.opensearch.dataprepper.plugins.processor.aggregate.AggregateProcessor.getTimeNanos; import org.opensearch.dataprepper.plugins.processor.aggregate.GroupState; import io.opentelemetry.proto.metrics.v1.AggregationTemporality; +import org.opensearch.dataprepper.plugins.hasher.IdentificationKeysHasher; import java.time.Instant; -import java.util.List; import java.time.ZoneId; import java.time.format.DateTimeFormatter; -import java.util.Map; + import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; /** * An AggregateAction that combines multiple Events into a single Event. This action will count the number of events with same keys and will create a combined event @@ -38,6 +42,7 @@ @DataPrepperPlugin(name = "count", pluginType = AggregateAction.class, pluginConfigurationType = CountAggregateActionConfig.class) public class CountAggregateAction implements AggregateAction { private static final String DATE_FORMAT = "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"; + private static final String UNIQUE_KEYS_SETKEY = "__unique_keys"; private static final String exemplarKey = "__exemplar"; static final String EVENT_TYPE = "event"; static final String SUM_METRIC_DESCRIPTION = "Number of events"; @@ -49,6 +54,7 @@ public class CountAggregateAction implements AggregateAction { public final String outputFormat; private long startTimeNanos; private final String metricName; + private final IdentificationKeysHasher uniqueKeysHasher; @DataPrepperPluginConstructor public CountAggregateAction(final CountAggregateActionConfig countAggregateActionConfig) { @@ -57,6 +63,11 @@ public CountAggregateAction(final CountAggregateActionConfig countAggregateActio this.endTimeKey = countAggregateActionConfig.getEndTimeKey(); this.outputFormat = countAggregateActionConfig.getOutputFormat(); this.metricName = countAggregateActionConfig.getMetricName(); + if (countAggregateActionConfig.getUniqueKeys() != null) { + this.uniqueKeysHasher = new IdentificationKeysHasher(countAggregateActionConfig.getUniqueKeys()); + } else { + this.uniqueKeysHasher = null; + } } public Exemplar createExemplar(final Event event) { @@ -93,12 +104,24 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct } if (groupState.get(countKey) == null) { groupState.putAll(aggregateActionInput.getIdentificationKeys()); + if (uniqueKeysHasher != null) { + Set uniqueKeysMapSet = new HashSet<>(); + + uniqueKeysMapSet.add(uniqueKeysHasher.createIdentificationKeysMapFromEvent(event)); + groupState.put(UNIQUE_KEYS_SETKEY, uniqueKeysMapSet); + } groupState.put(countKey, 1); groupState.put(exemplarKey, createExemplar(event)); groupState.put(startTimeKey, eventStartTime); groupState.put(endTimeKey, eventEndTime); } else { Integer v = (Integer)groupState.get(countKey) + 1; + + if (uniqueKeysHasher != null) { + Set uniqueKeysMapSet = (Set) groupState.get(UNIQUE_KEYS_SETKEY); + uniqueKeysMapSet.add(uniqueKeysHasher.createIdentificationKeysMapFromEvent(event)); + v = uniqueKeysMapSet.size(); + } groupState.put(countKey, v); Instant groupStartTime = (Instant)groupState.get(startTimeKey); Instant groupEndTime = (Instant)groupState.get(endTimeKey); @@ -117,6 +140,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA Instant startTime = (Instant)groupState.get(startTimeKey); Instant endTime = (Instant)groupState.get(endTimeKey); groupState.remove(endTimeKey); + groupState.remove(UNIQUE_KEYS_SETKEY); if (outputFormat.equals(OutputFormat.RAW.toString())) { groupState.put(startTimeKey, startTime.atZone(ZoneId.of(ZoneId.systemDefault().toString())).format(DateTimeFormatter.ofPattern(DATE_FORMAT))); event = JacksonEvent.builder() diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java index f7a2e6a48d..1144aee261 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java @@ -5,8 +5,9 @@ package org.opensearch.dataprepper.plugins.processor.aggregate.actions; -import java.util.Set; import java.util.HashSet; +import java.util.List; +import java.util.Set; import com.fasterxml.jackson.annotation.JsonProperty; public class CountAggregateActionConfig { @@ -22,6 +23,9 @@ public class CountAggregateActionConfig { @JsonProperty("metric_name") String metricName = SUM_METRIC_NAME; + @JsonProperty("unique_keys") + List uniqueKeys = null; + @JsonProperty("start_time_key") String startTimeKey = DEFAULT_START_TIME_KEY; @@ -35,6 +39,10 @@ public String getMetricName() { return metricName; } + public List getUniqueKeys() { + return uniqueKeys; + } + public String getCountKey() { return countKey; } diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java index 53b7c846b6..1975918e37 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfigTests.java @@ -13,6 +13,8 @@ import static org.opensearch.dataprepper.plugins.processor.aggregate.actions.CountAggregateActionConfig.DEFAULT_COUNT_KEY; import static org.opensearch.dataprepper.plugins.processor.aggregate.actions.CountAggregateActionConfig.DEFAULT_START_TIME_KEY; +import java.util.ArrayList; +import java.util.List; import java.util.UUID; import static org.hamcrest.CoreMatchers.equalTo; @@ -39,6 +41,7 @@ void testDefault() { assertThat(countAggregateActionConfig.getStartTimeKey(), equalTo(DEFAULT_START_TIME_KEY)); assertThat(countAggregateActionConfig.getOutputFormat(), equalTo(OutputFormat.OTEL_METRICS.toString())); assertThat(countAggregateActionConfig.getMetricName(), equalTo(CountAggregateActionConfig.SUM_METRIC_NAME)); + assertThat(countAggregateActionConfig.getUniqueKeys(), equalTo(null)); } @Test @@ -55,6 +58,11 @@ void testValidConfig() throws NoSuchFieldException, IllegalAccessException { final String testName = UUID.randomUUID().toString(); setField(CountAggregateActionConfig.class, countAggregateActionConfig, "metricName", testName); assertThat(countAggregateActionConfig.getMetricName(), equalTo(testName)); + final List uniqueKeys = new ArrayList<>(); + uniqueKeys.add(UUID.randomUUID().toString()); + uniqueKeys.add(UUID.randomUUID().toString()); + setField(CountAggregateActionConfig.class, countAggregateActionConfig, "uniqueKeys", uniqueKeys); + assertThat(countAggregateActionConfig.getUniqueKeys(), equalTo(uniqueKeys)); } @Test diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java index b91311c1c6..af81ca001f 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionTest.java @@ -153,6 +153,7 @@ void testCountAggregateOTelFormat(int testCount) throws NoSuchFieldException, Il void testCountAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCount) { CountAggregateActionConfig mockConfig = mock(CountAggregateActionConfig.class); when(mockConfig.getCountKey()).thenReturn(CountAggregateActionConfig.DEFAULT_COUNT_KEY); + when(mockConfig.getUniqueKeys()).thenReturn(null); final String testName = UUID.randomUUID().toString(); when(mockConfig.getMetricName()).thenReturn(testName); String startTimeKey = UUID.randomUUID().toString(); @@ -233,4 +234,57 @@ void testCountAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCount) { assertThat(attributes.get(key2), equalTo(value2)); assertTrue(attributes.containsKey(dataKey2)); } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 3, 10, 20}) + void testCountAggregateOTelFormatUniqueKeys(int testCount) throws NoSuchFieldException, IllegalAccessException { + CountAggregateActionConfig countAggregateActionConfig = new CountAggregateActionConfig(); + final String testName = UUID.randomUUID().toString(); + setField(CountAggregateActionConfig.class, countAggregateActionConfig, "metricName", testName); + final String key1 = "key-"+UUID.randomUUID().toString(); + final String value1 = UUID.randomUUID().toString(); + final String dataKey1 = "datakey-"+UUID.randomUUID().toString(); + setField(CountAggregateActionConfig.class, countAggregateActionConfig, "uniqueKeys", List.of(dataKey1)); + countAggregateAction = createObjectUnderTest(countAggregateActionConfig); + Map eventMap = Collections.singletonMap(key1, value1); + Event testEvent = JacksonEvent.builder() + .withEventType("event") + .withData(eventMap) + .build(); + AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(eventMap); + final String dataKey1_1 = UUID.randomUUID().toString(); + final String dataKey1_2 = UUID.randomUUID().toString(); + final String dataKey1_3 = UUID.randomUUID().toString(); + final String[] dataKeysList = {dataKey1_1, dataKey1_2, dataKey1_3}; + for (int i = 0; i < testCount; i++) { + testEvent.put(dataKey1, dataKeysList[i % 3]); + AggregateActionResponse aggregateActionResponse = countAggregateAction.handleEvent(testEvent, aggregateActionInput); + assertThat(aggregateActionResponse.getEvent(), equalTo(null)); + } + + AggregateActionOutput actionOutput = countAggregateAction.concludeGroup(aggregateActionInput); + final List result = actionOutput.getEvents(); + assertThat(result.size(), equalTo(1)); + Map expectedEventMap = new HashMap<>(); + double expectedCount = (testCount >= 3) ? 3 : testCount; + expectedEventMap.put("value", expectedCount); + expectedEventMap.put("description", "Number of events"); + expectedEventMap.put("isMonotonic", true); + expectedEventMap.put("aggregationTemporality", "AGGREGATION_TEMPORALITY_DELTA"); + expectedEventMap.put("unit", "1"); + expectedEventMap.put("name", testName); + expectedEventMap.forEach((k, v) -> assertThat(result.get(0).toMap(), hasEntry(k,v))); + assertThat(result.get(0).toMap().get("attributes"), equalTo(eventMap)); + JacksonMetric metric = (JacksonMetric) result.get(0); + assertThat(metric.toJsonString().indexOf("attributes"), not(-1)); + assertThat(result.get(0).toMap(), hasKey("startTime")); + assertThat(result.get(0).toMap(), hasKey("time")); + List> exemplars = (List >)result.get(0).toMap().get("exemplars"); + assertThat(exemplars.size(), equalTo(1)); + Map exemplar = exemplars.get(0); + Map attributes = (Map)exemplar.get("attributes"); + assertThat(attributes.get(key1), equalTo(value1)); + assertTrue(attributes.containsKey(dataKey1)); + + } } From dc77ba5519962cc0d45bf69f7363304de75ec7fb Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Tue, 25 Jun 2024 09:11:40 -0500 Subject: [PATCH 034/159] FIX: remove logging that includes credential info on kafka (#4659) * FIX: use sensitive marker Signed-off-by: George Chen --- .../plugins/kafka/consumer/KafkaCustomConsumerFactory.java | 4 +++- .../dataprepper/plugins/kafka/source/KafkaSource.java | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java index d703538e42..0d091b8af7 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java @@ -55,6 +55,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.IntStream; +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.SENSITIVE; + public class KafkaCustomConsumerFactory { private static final Logger LOG = LoggerFactory.getLogger(KafkaCustomConsumerFactory.class); @@ -136,7 +138,7 @@ private Properties getConsumerProperties(final KafkaConsumerConfig sourceConfig, } setConsumerTopicProperties(properties, topicConfig, topicConfig.getGroupId()); setSchemaRegistryProperties(sourceConfig, properties, topicConfig); - LOG.debug("Starting consumer with the properties : {}", properties); + LOG.debug(SENSITIVE, "Starting consumer with the properties : {}", properties); return properties; } 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 9a385c836f..525c754929 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 @@ -61,6 +61,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.IntStream; +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.SENSITIVE; + /** * The starting point of the Kafka-source plugin and the Kafka consumer * properties and kafka multithreaded consumers are being handled here. @@ -241,7 +243,7 @@ private Properties getConsumerProperties(final TopicConsumerConfig topicConfig, } setConsumerTopicProperties(properties, topicConfig); setSchemaRegistryProperties(properties, topicConfig); - LOG.debug("Starting consumer with the properties : {}", properties); + LOG.debug(SENSITIVE, "Starting consumer with the properties : {}", properties); return properties; } From 970f51f6d3a08bfcd789af3a7e517eb7ea3c4ce0 Mon Sep 17 00:00:00 2001 From: David Venable Date: Tue, 25 Jun 2024 10:07:40 -0500 Subject: [PATCH 035/159] Fixes the loading of peer-forwarders when using multiple workers. This fixes a bug where the service_map processor would not load in a pipeline with multiple workers. Resolves #4660. (#4661) Signed-off-by: David Venable --- .../PeerForwardingProcessorDecorator.java | 9 +-- ...PeerForwardingProcessingDecoratorTest.java | 66 ++++++++++--------- 2 files changed, 37 insertions(+), 38 deletions(-) diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java index 097b2b6552..038bdb28c5 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessorDecorator.java @@ -67,13 +67,10 @@ public static List decorateProcessors( "Peer Forwarder Plugin: %s cannot have empty identification keys." + pluginId); } + final PeerForwarder peerForwarder = peerForwarderProvider.register(pipelineName, firstInnerProcessor, pluginId, identificationKeys, pipelineWorkerThreads); - return processors.stream() - .map(processor -> { - PeerForwarder peerForwarder = peerForwarderProvider.register(pipelineName, processor, pluginId, identificationKeys, pipelineWorkerThreads); - return new PeerForwardingProcessorDecorator(peerForwarder, processor); - }) - .collect(Collectors.toList()); + return processors.stream().map(processor -> new PeerForwardingProcessorDecorator(peerForwarder, processor)) + .collect(Collectors.toList()); } private PeerForwardingProcessorDecorator(final PeerForwarder peerForwarder, final Processor innerProcessor) { diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java index ceb424b0cb..7a85033842 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/PeerForwardingProcessingDecoratorTest.java @@ -39,6 +39,7 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -51,18 +52,6 @@ class PeerForwardingProcessingDecoratorTest { @Mock private Processor processor; - @Mock - private Processor processor1; - - @Mock - private Processor processor2; - - @Mock(extraInterfaces = Processor.class) - private RequiresPeerForwarding requiresPeerForwarding1; - - @Mock(extraInterfaces = Processor.class) - private RequiresPeerForwarding requiresPeerForwarding2; - @Mock(extraInterfaces = Processor.class) private RequiresPeerForwarding requiresPeerForwarding; @@ -82,13 +71,13 @@ record = mock(Record.class); pluginId = UUID.randomUUID().toString(); } - private List createObjectUnderTesDecoratedProcessors(final List processors) { + private List createObjectUnderTestDecoratedProcessors(final List processors) { return PeerForwardingProcessorDecorator.decorateProcessors(processors, peerForwarderProvider, pipelineName, pluginId, PIPELINE_WORKER_THREADS); } @Test void PeerForwardingProcessingDecorator_should_not_have_any_interactions_if_its_not_an_instance_of_RequiresPeerForwarding() { - assertThrows(UnsupportedPeerForwarderPluginException.class, () -> createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor))); + assertThrows(UnsupportedPeerForwarderPluginException.class, () -> createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor))); verifyNoInteractions(peerForwarderProvider); } @@ -97,7 +86,7 @@ void PeerForwardingProcessingDecorator_should_not_have_any_interactions_if_its_n void PeerForwardingProcessingDecorator_execute_with_empty_identification_keys_should_throw() { when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(Collections.emptySet()); - assertThrows(EmptyPeerForwarderPluginIdentificationKeysException.class, () -> createObjectUnderTesDecoratedProcessors(Collections.singletonList((Processor) requiresPeerForwarding))); + assertThrows(EmptyPeerForwarderPluginIdentificationKeysException.class, () -> createObjectUnderTestDecoratedProcessors(Collections.singletonList((Processor) requiresPeerForwarding))); } @Test @@ -109,12 +98,12 @@ void decorateProcessors_with_different_identification_key_should_throw() { when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(Set.of(UUID.randomUUID().toString())); when(requiresPeerForwardingCopy.getIdentificationKeys()).thenReturn(Set.of(UUID.randomUUID().toString())); - assertThrows(RuntimeException.class, () -> createObjectUnderTesDecoratedProcessors(List.of(((Processor) requiresPeerForwarding), (Processor) requiresPeerForwardingCopy))); + assertThrows(RuntimeException.class, () -> createObjectUnderTestDecoratedProcessors(List.of(((Processor) requiresPeerForwarding), (Processor) requiresPeerForwardingCopy))); } @Test void decorateProcessors_with_empty_processors_should_return_empty_list_of_processors() { - final List processors = createObjectUnderTesDecoratedProcessors(Collections.emptyList()); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.emptyList()); assertThat(processors.size(), equalTo(0)); } @@ -136,9 +125,22 @@ void setUp() { @Test void PeerForwardingProcessingDecorator_should_have_interaction_with_getIdentificationKeys() { - createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); + createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)); + verify(requiresPeerForwarding, times(2)).getIdentificationKeys(); + verify(peerForwarderProvider).register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + verifyNoMoreInteractions(peerForwarderProvider); + } + + @Test + void PeerForwardingProcessingDecorator_should_have_interaction_with_getIdentificationKeys_when_list_of_processors() { + when(requiresPeerForwarding.getIdentificationKeys()).thenReturn(identificationKeys); + when(requiresPeerForwardingCopy.getIdentificationKeys()).thenReturn(identificationKeys); + + createObjectUnderTestDecoratedProcessors(List.of((Processor) requiresPeerForwarding, (Processor) requiresPeerForwardingCopy)); + verify(requiresPeerForwarding, times(2)).getIdentificationKeys(); verify(peerForwarderProvider).register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + verifyNoMoreInteractions(peerForwarderProvider); } @Test @@ -148,16 +150,15 @@ void PeerForwardingProcessingDecorator_with_localProcessingOnly() { processorList.add((Processor) requiresPeerForwardingCopy); LocalPeerForwarder localPeerForwarder = mock(LocalPeerForwarder.class); - lenient().when(peerForwarderProvider.register(pipelineName, (Processor) requiresPeerForwarding, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(localPeerForwarder); - lenient().when(peerForwarderProvider.register(pipelineName, (Processor) requiresPeerForwardingCopy, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(localPeerForwarder); + when(peerForwarderProvider.register(pipelineName, (Processor) requiresPeerForwarding, pluginId, identificationKeys, PIPELINE_WORKER_THREADS)).thenReturn(localPeerForwarder); Event event = mock(Event.class); when(record.getData()).thenReturn(event); List> testData = Collections.singletonList(record); when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event)).thenReturn(false); when(requiresPeerForwardingCopy.isApplicableEventForPeerForwarding(event)).thenReturn(false); - processor1 = (Processor)requiresPeerForwarding; - processor2 = (Processor)requiresPeerForwardingCopy; + Processor processor1 = (Processor)requiresPeerForwarding; + Processor processor2 = (Processor)requiresPeerForwardingCopy; when(processor1.execute(testData)).thenReturn(testData); when(processor2.execute(testData)).thenReturn(testData); @@ -167,9 +168,10 @@ void PeerForwardingProcessingDecorator_with_localProcessingOnly() { when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(true); when(requiresPeerForwardingCopy.isForLocalProcessingOnly(any())).thenReturn(true); - final List processors = createObjectUnderTesDecoratedProcessors(processorList); + final List processors = createObjectUnderTestDecoratedProcessors(processorList); assertThat(processors.size(), equalTo(2)); verify(peerForwarderProvider, times(1)).register(pipelineName, processor, pluginId, identificationKeys, PIPELINE_WORKER_THREADS); + verifyNoMoreInteractions(peerForwarderProvider); Collection> result = processors.get(0).execute(testData); assertThat(result.size(), equalTo(testData.size())); assertThat(result, equalTo(testData)); @@ -189,7 +191,7 @@ void PeerForwardingProcessingDecorator_execute_should_forwardRecords_with_correc when(processor.execute(testData)).thenReturn(testData); - final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)); assertThat(processors.size(), equalTo(1)); final Collection> records = processors.get(0).execute(testData); @@ -215,7 +217,7 @@ void PeerForwardingProcessingDecorator_execute_should_receiveRecords() { when(((Processor) requiresPeerForwarding).execute(anyCollection())).thenReturn(expectedRecordsToProcessLocally); - final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList((Processor) requiresPeerForwarding)); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList((Processor) requiresPeerForwarding)); assertThat(processors.size(), equalTo(1)); final Collection> records = processors.get(0).execute(forwardTestData); @@ -232,7 +234,7 @@ void PeerForwardingProcessingDecorator_execute_will_call_inner_processors_execut Event event = mock(Event.class); when(record.getData()).thenReturn(event); when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event)).thenReturn(true); - final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)); Collection> testData = Collections.singletonList(record); assertThat(processors.size(), equalTo(1)); @@ -248,7 +250,7 @@ void PeerForwardingProcessingDecorator_execute_will_call_inner_processors_execut when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event)).thenReturn(false); when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(true); - final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)); Collection> testData = Collections.singletonList(record); assertThat(processors.size(), equalTo(1)); @@ -272,7 +274,7 @@ void PeerForwardingProcessingDecorator_inner_processor_with_is_applicable_event_ when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event2)).thenReturn(false); when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event3)).thenReturn(false); when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(true); - final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)); when(record1.getData()).thenReturn(event1); when(record2.getData()).thenReturn(event2); when(record3.getData()).thenReturn(event3); @@ -303,7 +305,7 @@ void PeerForwardingProcessingDecorator_inner_processor_with_is_applicable_event_ when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event2)).thenReturn(false); when(requiresPeerForwarding.isApplicableEventForPeerForwarding(event3)).thenReturn(true); when(requiresPeerForwarding.isForLocalProcessingOnly(any())).thenReturn(false); - final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)); when(record1.getData()).thenReturn(event1); when(record2.getData()).thenReturn(event2); when(record3.getData()).thenReturn(event3); @@ -322,7 +324,7 @@ void PeerForwardingProcessingDecorator_inner_processor_with_is_applicable_event_ @Test void PeerForwardingProcessingDecorator_prepareForShutdown_will_call_inner_processors_prepareForShutdown() { - final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)); assertThat(processors.size(), equalTo(1)); processors.get(0).prepareForShutdown(); @@ -331,7 +333,7 @@ void PeerForwardingProcessingDecorator_prepareForShutdown_will_call_inner_proces @Test void PeerForwardingProcessingDecorator_isReadyForShutdown_will_call_inner_processors_isReadyForShutdown() { - final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)); assertThat(processors.size(), equalTo(1)); processors.get(0).isReadyForShutdown(); @@ -340,7 +342,7 @@ void PeerForwardingProcessingDecorator_isReadyForShutdown_will_call_inner_proces @Test void PeerForwardingProcessingDecorator_shutdown_will_call_inner_processors_shutdown() { - final List processors = createObjectUnderTesDecoratedProcessors(Collections.singletonList(processor)); + final List processors = createObjectUnderTestDecoratedProcessors(Collections.singletonList(processor)); assertThat(processors.size(), equalTo(1)); processors.get(0).shutdown(); From 980b0ec50f0e7a905fab9a79896b413d9e23230e Mon Sep 17 00:00:00 2001 From: Srikanth Govindarajan Date: Tue, 25 Jun 2024 09:51:28 -0700 Subject: [PATCH 036/159] Refactor lambda plugin (#4643) * Refactor lambda plugin Signed-off-by: Srikanth Govindarajan * Address comments Signed-off-by: Srikanth Govindarajan * Address comments 2 Signed-off-by: Srikanth Govindarajan --------- Signed-off-by: Srikanth Govindarajan Signed-off-by: Srikanth Govindarajan --- .../{lambda-sink => lambda}/README.md | 0 .../{lambda-sink => lambda}/build.gradle | 5 ++ .../sink/lambda/LambdaSinkServiceIT.java | 17 +++--- .../lambda/common}/accumlator/Buffer.java | 7 ++- .../common}/accumlator/BufferFactory.java | 2 +- .../common}/accumlator/InMemoryBuffer.java | 20 ++++++- .../accumlator/InMemoryBufferFactory.java | 2 +- .../lambda/common}/codec/LambdaJsonCodec.java | 3 +- .../config/AwsAuthenticationOptions.java | 2 +- .../lambda/common}/config/BatchOptions.java | 4 +- .../common}/config/ThresholdOptions.java | 7 ++- .../lambda/common/util}/ThresholdCheck.java | 7 +-- .../lambda/sink}/LambdaClientFactory.java | 4 +- .../plugins/lambda/sink}/LambdaSink.java | 8 +-- .../lambda/sink}/LambdaSinkConfig.java | 8 +-- .../lambda/sink}/LambdaSinkService.java | 33 ++++++----- .../lambda/sink}/dlq/DlqPushHandler.java | 2 +- .../sink}/dlq/LambdaSinkFailedDlqData.java | 2 +- .../lambda/common}/ThresholdCheckTest.java | 12 ++-- .../InMemoryBufferFactoryTest.java | 6 +- .../accumulator/InMemoryBufferTest.java | 31 +++++----- .../common}/codec/LambdaJsonCodecTest.java | 2 +- .../common}/config/ThresholdOptionsTest.java | 7 +-- .../lambda/sink}/LambdaClientFactoryTest.java | 21 ++++--- .../lambda/sink}/LambdaSinkConfigTest.java | 5 +- .../lambda/sink}/LambdaSinkServiceTest.java | 58 +++++++++---------- .../plugins/lambda/sink}/LambdaSinkTest.java | 13 ++--- .../lambda/sink}/dlq/DlqPushHandlerTest.java | 19 +++--- .../org.mockito.plugins.MockMaker | 0 settings.gradle | 2 +- 30 files changed, 163 insertions(+), 146 deletions(-) rename data-prepper-plugins/{lambda-sink => lambda}/README.md (100%) rename data-prepper-plugins/{lambda-sink => lambda}/build.gradle (89%) rename data-prepper-plugins/{lambda-sink => lambda}/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java (93%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common}/accumlator/Buffer.java (68%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common}/accumlator/BufferFactory.java (82%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common}/accumlator/InMemoryBuffer.java (84%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common}/accumlator/InMemoryBufferFactory.java (85%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common}/codec/LambdaJsonCodec.java (95%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common}/config/AwsAuthenticationOptions.java (95%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common}/config/BatchOptions.java (80%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common}/config/ThresholdOptions.java (95%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util}/ThresholdCheck.java (84%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaClientFactory.java (93%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaSink.java (93%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaSinkConfig.java (90%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaSinkService.java (92%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink}/dlq/DlqPushHandler.java (98%) rename data-prepper-plugins/{lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink}/dlq/LambdaSinkFailedDlqData.java (95%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common}/ThresholdCheckTest.java (95%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common}/accumulator/InMemoryBufferFactoryTest.java (78%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common}/accumulator/InMemoryBufferTest.java (95%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common}/codec/LambdaJsonCodecTest.java (98%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common}/config/ThresholdOptionsTest.java (93%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaClientFactoryTest.java (96%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaSinkConfigTest.java (94%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaSinkServiceTest.java (94%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaSinkTest.java (95%) rename data-prepper-plugins/{lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda => lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink}/dlq/DlqPushHandlerTest.java (95%) rename data-prepper-plugins/{lambda-sink => lambda}/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker (100%) diff --git a/data-prepper-plugins/lambda-sink/README.md b/data-prepper-plugins/lambda/README.md similarity index 100% rename from data-prepper-plugins/lambda-sink/README.md rename to data-prepper-plugins/lambda/README.md diff --git a/data-prepper-plugins/lambda-sink/build.gradle b/data-prepper-plugins/lambda/build.gradle similarity index 89% rename from data-prepper-plugins/lambda-sink/build.gradle rename to data-prepper-plugins/lambda/build.gradle index 429e190a6a..d0c09c9c8b 100644 --- a/data-prepper-plugins/lambda-sink/build.gradle +++ b/data-prepper-plugins/lambda/build.gradle @@ -19,6 +19,11 @@ dependencies { implementation'org.json:json' implementation libs.commons.lang3 implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' + implementation 'org.projectlombok:lombok:1.18.22' + compileOnly 'org.projectlombok:lombok:1.18.20' + annotationProcessor 'org.projectlombok:lombok:1.18.20' + testCompileOnly 'org.projectlombok:lombok:1.18.20' + testAnnotationProcessor 'org.projectlombok:lombok:1.18.20' testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-plugins:parse-json-processor') diff --git a/data-prepper-plugins/lambda-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java b/data-prepper-plugins/lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java similarity index 93% rename from data-prepper-plugins/lambda-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java rename to data-prepper-plugins/lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java index 89cf85ceac..76fb4831ce 100644 --- a/data-prepper-plugins/lambda-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java +++ b/data-prepper-plugins/lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java @@ -17,6 +17,7 @@ import org.mockito.Mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import org.mockito.MockitoAnnotations; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; @@ -29,12 +30,14 @@ 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.sink.lambda.accumlator.BufferFactory; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBufferFactory; -import org.opensearch.dataprepper.plugins.sink.lambda.config.BatchOptions; -import org.opensearch.dataprepper.plugins.sink.lambda.config.ThresholdOptions; -import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; -import org.opensearch.dataprepper.plugins.sink.lambda.dlq.DlqPushHandler; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.BufferFactory; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.BatchOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.ThresholdOptions; +import org.opensearch.dataprepper.plugins.lambda.sink.LambdaSinkConfig; +import org.opensearch.dataprepper.plugins.lambda.sink.LambdaSinkService; +import org.opensearch.dataprepper.plugins.lambda.sink.dlq.DlqPushHandler; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.lambda.LambdaClient; @@ -45,8 +48,6 @@ import java.util.HashMap; import java.util.List; -import static org.mockito.Mockito.when; - @ExtendWith(MockitoExtension.class) class LambdaSinkServiceIT { diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/Buffer.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java similarity index 68% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/Buffer.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java index 48afbe6a01..f52a8e5de0 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/Buffer.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java @@ -3,9 +3,10 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.accumlator; +package org.opensearch.dataprepper.plugins.lambda.common.accumlator; import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.lambda.model.InvokeResponse; import java.io.OutputStream; import java.time.Duration; @@ -21,7 +22,9 @@ public interface Buffer { Duration getDuration(); - void flushToLambda(); + void flushToLambdaAsync(); + + InvokeResponse flushToLambdaSync(); OutputStream getOutputStream(); diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/BufferFactory.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java similarity index 82% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/BufferFactory.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java index 80afd2f1ca..e44bbd6aee 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/BufferFactory.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.accumlator; +package org.opensearch.dataprepper.plugins.lambda.common.accumlator; import software.amazon.awssdk.services.lambda.LambdaClient; diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBuffer.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java similarity index 84% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBuffer.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java index bba70c6e62..5d9d5a5134 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBuffer.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.accumlator; +package org.opensearch.dataprepper.plugins.lambda.common.accumlator; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; @@ -62,7 +62,22 @@ public Duration getDuration() { @Override - public void flushToLambda() { + public void flushToLambdaAsync() { + InvokeResponse resp; + SdkBytes payload = getPayload(); + + // Setup an InvokeRequest. + InvokeRequest request = InvokeRequest.builder() + .functionName(functionName) + .payload(payload) + .invocationType(invocationType) + .build(); + + lambdaClient.invoke(request); + } + + @Override + public InvokeResponse flushToLambdaSync() { InvokeResponse resp; SdkBytes payload = getPayload(); @@ -74,6 +89,7 @@ public void flushToLambda() { .build(); resp = lambdaClient.invoke(request); + return resp; } private SdkBytes validatePayload(String payload_string) { diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBufferFactory.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java similarity index 85% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBufferFactory.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java index e58952c5cb..37ad4a4105 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/accumlator/InMemoryBufferFactory.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.accumlator; +package org.opensearch.dataprepper.plugins.lambda.common.accumlator; import software.amazon.awssdk.services.lambda.LambdaClient; diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodec.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java similarity index 95% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodec.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java index 5bf21f5e18..a1ccaa8561 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodec.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java @@ -2,7 +2,7 @@ * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.codec; +package org.opensearch.dataprepper.plugins.lambda.common.codec; import com.fasterxml.jackson.core.JsonEncoding; import com.fasterxml.jackson.core.JsonFactory; @@ -37,7 +37,6 @@ public String getExtension() { @Override public void start(final OutputStream outputStream, Event event, final OutputCodecContext codecContext) throws IOException { Objects.requireNonNull(outputStream); - Objects.requireNonNull(codecContext); this.codecContext = codecContext; generator = factory.createGenerator(outputStream, JsonEncoding.UTF8); if(Objects.nonNull(keyName)){ diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/AwsAuthenticationOptions.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java similarity index 95% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/AwsAuthenticationOptions.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java index 8d6c64829d..e40fa617ee 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/AwsAuthenticationOptions.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.config; +package org.opensearch.dataprepper.plugins.lambda.common.config; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.Size; diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/BatchOptions.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java similarity index 80% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/BatchOptions.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java index 3773d4e6ed..099bed2b54 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/BatchOptions.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.config; +package org.opensearch.dataprepper.plugins.lambda.common.config; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.NotNull; @@ -18,7 +18,7 @@ public class BatchOptions { @JsonProperty("threshold") @NotNull - ThresholdOptions thresholdOptions; + ThresholdOptions thresholdOptions = new ThresholdOptions(); public String getBatchKey(){return batchKey;} diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptions.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java similarity index 95% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptions.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java index 031157c4be..1f92b90b48 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptions.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java @@ -3,15 +3,16 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.config; +package org.opensearch.dataprepper.plugins.lambda.common.config; import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.Size; import org.hibernate.validator.constraints.time.DurationMax; import org.hibernate.validator.constraints.time.DurationMin; import org.opensearch.dataprepper.model.types.ByteCount; + import java.time.Duration; -import jakarta.validation.constraints.NotNull; -import jakarta.validation.constraints.Size; public class ThresholdOptions { diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheck.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java similarity index 84% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheck.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java index 74aa98e7f9..6bbf8a4ab8 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheck.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java @@ -3,10 +3,10 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.common.util; import org.opensearch.dataprepper.model.types.ByteCount; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.Buffer; import java.time.Duration; @@ -15,9 +15,6 @@ */ public class ThresholdCheck { - private ThresholdCheck() { - } - public static boolean checkThresholdExceed(final Buffer currentBuffer, final int maxEvents, final ByteCount maxBytes, final Duration maxCollectionDuration, final Boolean isBatchEnabled) { if (!isBatchEnabled) return true; diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactory.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java similarity index 93% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactory.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java index 3e33a4e835..03b94340f0 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactory.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java @@ -3,11 +3,11 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.core.retry.RetryPolicy; diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSink.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java similarity index 93% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSink.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java index b1ef905233..54e484fd13 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSink.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; @@ -17,9 +17,9 @@ import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.sink.SinkContext; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.BufferFactory; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBufferFactory; -import org.opensearch.dataprepper.plugins.sink.lambda.dlq.DlqPushHandler; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.BufferFactory; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.lambda.sink.dlq.DlqPushHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.lambda.LambdaClient; diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfig.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java similarity index 90% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfig.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java index a20fa41181..bb50e2510e 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfig.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java @@ -2,7 +2,7 @@ * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.Valid; @@ -10,11 +10,11 @@ import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.Size; import org.opensearch.dataprepper.model.configuration.PluginModel; -import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; -import org.opensearch.dataprepper.plugins.sink.lambda.config.BatchOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.BatchOptions; -import java.util.Objects; import java.util.Map; +import java.util.Objects; public class LambdaSinkConfig { diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkService.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java similarity index 92% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkService.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java index f10607e7d1..9a788e6816 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkService.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java @@ -3,29 +3,30 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.codec.OutputCodec; import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.plugin.PluginFactory; +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.sink.lambda.accumlator.Buffer; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.BufferFactory; -import org.opensearch.dataprepper.plugins.sink.lambda.codec.LambdaJsonCodec; -import org.opensearch.dataprepper.plugins.sink.lambda.config.BatchOptions; -import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.sink.lambda.dlq.DlqPushHandler; -import org.opensearch.dataprepper.plugins.sink.lambda.dlq.LambdaSinkFailedDlqData; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.BufferFactory; +import org.opensearch.dataprepper.plugins.lambda.common.codec.LambdaJsonCodec; +import org.opensearch.dataprepper.plugins.lambda.common.config.BatchOptions; +import org.opensearch.dataprepper.plugins.lambda.common.util.ThresholdCheck; +import org.opensearch.dataprepper.plugins.lambda.sink.dlq.DlqPushHandler; +import org.opensearch.dataprepper.plugins.lambda.sink.dlq.LambdaSinkFailedDlqData; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import software.amazon.awssdk.core.SdkBytes; import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.SdkBytes; import software.amazon.awssdk.core.exception.SdkClientException; import software.amazon.awssdk.services.lambda.LambdaClient; @@ -48,7 +49,7 @@ public class LambdaSinkService { private final PluginSetting pluginSetting; private final Lock reentrantLock; private final LambdaSinkConfig lambdaSinkConfig; - private LambdaClient lambdaClient; + private final LambdaClient lambdaClient; private final String functionName; private int maxEvents = 0; private ByteCount maxBytes = null; @@ -65,9 +66,9 @@ public class LambdaSinkService { private final List events; private OutputCodec codec = null; private final BatchOptions batchOptions; - private Boolean isBatchEnabled; + private final Boolean isBatchEnabled; private OutputCodecContext codecContext = null; - private String batchKey; + private final String batchKey; public LambdaSinkService(final LambdaClient lambdaClient, final LambdaSinkConfig lambdaSinkConfig, @@ -213,7 +214,7 @@ protected boolean retryFlushToLambda(Buffer currentBuffer, do { try { - currentBuffer.flushToLambda(); + currentBuffer.flushToLambdaAsync(); isUploadedToLambda = Boolean.TRUE; } catch (AwsServiceException | SdkClientException e) { errorMsgObj.set(e.getMessage()); diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandler.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java similarity index 98% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandler.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java index 1bdeb0a394..da8c52eb4e 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandler.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java @@ -2,7 +2,7 @@ * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.dlq; +package org.opensearch.dataprepper.plugins.lambda.sink.dlq; import com.fasterxml.jackson.databind.ObjectWriter; import io.micrometer.core.instrument.util.StringUtils; diff --git a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/LambdaSinkFailedDlqData.java b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java similarity index 95% rename from data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/LambdaSinkFailedDlqData.java rename to data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java index 0808010e37..8941966b77 100644 --- a/data-prepper-plugins/lambda-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/LambdaSinkFailedDlqData.java +++ b/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java @@ -2,7 +2,7 @@ * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.dlq; +package org.opensearch.dataprepper.plugins.lambda.sink.dlq; import com.fasterxml.jackson.core.JsonProcessingException; import software.amazon.awssdk.core.SdkBytes; diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheckTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java similarity index 95% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheckTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java index b63553911a..d56420d18f 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/ThresholdCheckTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java @@ -3,23 +3,23 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.common; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; +import static org.mockito.Mockito.when; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.model.types.ByteCount; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.lambda.common.util.ThresholdCheck; import java.io.IOException; import java.time.Duration; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.when; - @ExtendWith(MockitoExtension.class) class ThresholdCheckTest { diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferFactoryTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java similarity index 78% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferFactoryTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java index d161b28bb0..37276db819 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferFactoryTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java @@ -3,12 +3,12 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.accumulator; +package org.opensearch.dataprepper.plugins.lambda.common.accumulator; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.Buffer; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.InMemoryBufferFactory; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.MatcherAssert.assertThat; diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java similarity index 95% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java index 478650a300..fb164b1ac1 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/accumulator/InMemoryBufferTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java @@ -3,16 +3,26 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.accumulator; +package org.opensearch.dataprepper.plugins.lambda.common.accumulator; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; import org.hamcrest.Matchers; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import org.junit.jupiter.api.Assertions; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import static org.mockito.ArgumentMatchers.any; import org.mockito.Mock; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBuffer; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.InMemoryBuffer; import software.amazon.awssdk.core.SdkBytes; import software.amazon.awssdk.core.exception.SdkClientException; import software.amazon.awssdk.services.lambda.LambdaClient; @@ -25,17 +35,6 @@ import java.time.Instant; import java.time.temporal.ChronoUnit; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.notNullValue; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; -import static org.hamcrest.Matchers.lessThanOrEqualTo; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -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.when; - @ExtendWith(MockitoExtension.class) class InMemoryBufferTest { @@ -119,7 +118,7 @@ void test_with_write_event_into_buffer_and_flush_toLambda() throws IOException { inMemoryBuffer.setEventCount(eventCount); } assertDoesNotThrow(() -> { - inMemoryBuffer.flushToLambda(); + inMemoryBuffer.flushToLambdaAsync(); }); } @@ -136,7 +135,7 @@ void test_uploadedToLambda_success() throws IOException { OutputStream outputStream = inMemoryBuffer.getOutputStream(); outputStream.write(generateByteArray()); assertDoesNotThrow(() -> { - inMemoryBuffer.flushToLambda(); + inMemoryBuffer.flushToLambdaAsync(); }); } @@ -153,7 +152,7 @@ void test_uploadedToLambda_fails() { inMemoryBuffer = new InMemoryBuffer(lambdaClient, functionName, invocationType); Assertions.assertNotNull(inMemoryBuffer); - SdkClientException actualException = assertThrows(SdkClientException.class, () -> inMemoryBuffer.flushToLambda()); + SdkClientException actualException = assertThrows(SdkClientException.class, () -> inMemoryBuffer.flushToLambdaAsync()); assertThat(actualException, Matchers.equalTo(sdkClientException)); } diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodecTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java similarity index 98% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodecTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java index 6de6ce8a0e..4b6e4c5caf 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/codec/LambdaJsonCodecTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.codec; +package org.opensearch.dataprepper.plugins.lambda.common.codec; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptionsTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java similarity index 93% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptionsTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java index 53bd0a4edf..5d12aca3da 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/config/ThresholdOptionsTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java @@ -3,13 +3,12 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.config; - -import org.junit.jupiter.api.Test; -import org.opensearch.dataprepper.model.types.ByteCount; +package org.opensearch.dataprepper.plugins.lambda.common.config; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.types.ByteCount; class ThresholdOptionsTest { private static final String DEFAULT_BYTE_CAPACITY = "6mb"; diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactoryTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java similarity index 96% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactoryTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java index ab72ee44b8..9ed5c71fb2 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaClientFactoryTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java @@ -2,35 +2,34 @@ * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.ArgumentCaptor; +import static org.mockito.ArgumentMatchers.any; import org.mockito.Mock; import org.mockito.MockedStatic; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.lambda.LambdaClient; import software.amazon.awssdk.services.lambda.LambdaClientBuilder; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.CoreMatchers.notNullValue; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockStatic; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - import java.util.Map; import java.util.UUID; diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfigTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java similarity index 94% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfigTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java index eda9488a04..2a6dad3a69 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkConfigTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java @@ -2,12 +2,13 @@ * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import org.hamcrest.MatcherAssert; import org.junit.jupiter.api.Test; import software.amazon.awssdk.regions.Region; @@ -21,7 +22,7 @@ class LambdaSinkConfigTest { @Test void lambda_sink_default_max_connection_retries_test(){ - assertThat(new LambdaSinkConfig().getMaxConnectionRetries(),equalTo(DEFAULT_MAX_RETRIES)); + MatcherAssert.assertThat(new LambdaSinkConfig().getMaxConnectionRetries(),equalTo(DEFAULT_MAX_RETRIES)); } @Test diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java similarity index 94% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java index bbab8778c0..4e678c191d 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java @@ -2,34 +2,46 @@ * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; import io.micrometer.core.instrument.Counter; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertEquals; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.plugin.PluginFactory; +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.sink.lambda.accumlator.Buffer; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.BufferFactory; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBuffer; -import org.opensearch.dataprepper.plugins.sink.lambda.accumlator.InMemoryBufferFactory; -import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; -import org.opensearch.dataprepper.plugins.sink.lambda.config.BatchOptions; -import org.opensearch.dataprepper.plugins.sink.lambda.config.ThresholdOptions; -import org.opensearch.dataprepper.plugins.sink.lambda.dlq.DlqPushHandler; -import org.opensearch.dataprepper.plugins.sink.lambda.dlq.LambdaSinkFailedDlqData; -import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.BufferFactory; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.InMemoryBuffer; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.InMemoryBufferFactory; +import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.BatchOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.ThresholdOptions; +import org.opensearch.dataprepper.plugins.lambda.sink.dlq.DlqPushHandler; +import org.opensearch.dataprepper.plugins.lambda.sink.dlq.LambdaSinkFailedDlqData; import software.amazon.awssdk.awscore.exception.AwsServiceException; import software.amazon.awssdk.core.SdkBytes; import software.amazon.awssdk.http.SdkHttpResponse; @@ -37,7 +49,6 @@ import software.amazon.awssdk.services.lambda.model.InvokeRequest; import software.amazon.awssdk.services.lambda.model.InvokeResponse; - import java.io.ByteArrayOutputStream; import java.io.IOException; import java.time.Duration; @@ -47,19 +58,6 @@ import java.util.List; import java.util.Map; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; -import static org.mockito.ArgumentMatchers.any; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.mockito.BDDMockito.given; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.doThrow; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoInteractions; - public class LambdaSinkServiceTest { public static final int maxEvents = 10; @@ -193,7 +191,7 @@ public void lambda_sink_test_max_retires_works() throws IOException { ByteArrayOutputStream byteArrayOutputStream = new ByteArrayOutputStream(); when(buffer.getOutputStream()).thenReturn(byteArrayOutputStream); when(bufferFactory.getBuffer(any(LambdaClient.class),any(),any())).thenReturn(buffer); - doThrow(AwsServiceException.class).when(buffer).flushToLambda(); + doThrow(AwsServiceException.class).when(buffer).flushToLambdaAsync(); LambdaSinkService lambdaSinkService = new LambdaSinkService(lambdaClient, lambdaSinkConfig, @@ -209,7 +207,7 @@ public void lambda_sink_test_max_retires_works() throws IOException { Collection> records = List.of(eventRecord); lambdaSinkService.output(records); - verify(buffer, times(3)).flushToLambda(); + verify(buffer, times(3)).flushToLambdaAsync(); } @Test @@ -232,7 +230,7 @@ public void lambda_sink_test_dlq_works() throws IOException { when(buffer.getOutputStream()).thenReturn(byteArrayOutputStream); when(bufferFactory.getBuffer(any(LambdaClient.class),any(),any())).thenReturn(buffer); - doThrow(AwsServiceException.class).when(buffer).flushToLambda(); + doThrow(AwsServiceException.class).when(buffer).flushToLambdaAsync(); LambdaSinkService lambdaSinkService = new LambdaSinkService(lambdaClient, lambdaSinkConfig, @@ -249,7 +247,7 @@ public void lambda_sink_test_dlq_works() throws IOException { lambdaSinkService.output(records); - verify(buffer, times(3)).flushToLambda(); + verify(buffer, times(3)).flushToLambdaAsync(); verify(dlqPushHandler,times(1)).perform(any(PluginSetting.class),any(Object.class)); } diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java similarity index 95% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java index 1687cbd285..9a042014f0 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java @@ -3,28 +3,27 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; import org.junit.jupiter.api.Assertions; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.sink.SinkContext; -import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.lambda.LambdaClient; import java.util.HashMap; import java.util.Map; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - class LambdaSinkTest { public static final String S3_REGION = "us-east-1"; diff --git a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandlerTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java similarity index 95% rename from data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandlerTest.java rename to data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java index 17f39973b7..e1de3303a1 100644 --- a/data-prepper-plugins/lambda-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/lambda/dlq/DlqPushHandlerTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java @@ -2,17 +2,24 @@ * Copyright OpenSearch Contributors * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda.dlq; +package org.opensearch.dataprepper.plugins.lambda.sink.dlq; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyList; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.dlq.DlqProvider; import org.opensearch.dataprepper.plugins.dlq.DlqWriter; -import org.opensearch.dataprepper.plugins.sink.lambda.config.AwsAuthenticationOptions; import software.amazon.awssdk.core.SdkBytes; import java.io.IOException; @@ -20,14 +27,6 @@ import java.util.Map; import java.util.Optional; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyList; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - class DlqPushHandlerTest { private static final String BUCKET = "bucket"; diff --git a/data-prepper-plugins/lambda-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker similarity index 100% rename from data-prepper-plugins/lambda-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker rename to data-prepper-plugins/lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/settings.gradle b/settings.gradle index 8400ff98c2..64d86219ea 100644 --- a/settings.gradle +++ b/settings.gradle @@ -175,4 +175,4 @@ include 'data-prepper-plugins:mongodb' include 'data-prepper-plugins:rds-source' include 'data-prepper-plugins:http-source-common' include 'data-prepper-plugins:http-common' -include 'data-prepper-plugins:lambda-sink' \ No newline at end of file +include 'data-prepper-plugins:lambda' \ No newline at end of file From d29954c25363b827b7697e6073e724c2d427ec1d Mon Sep 17 00:00:00 2001 From: David Venable Date: Tue, 25 Jun 2024 17:09:27 -0500 Subject: [PATCH 037/159] Updates the chunking algorithm for http source's JsonCodec to account for actual byte size. Test using Unicode characters to prove this was incorrectly chunking and verify against future changes. (#4656) Signed-off-by: David Venable --- .../dataprepper/http/codec/JsonCodec.java | 3 +- .../dataprepper/http/codec/JsonCodecTest.java | 48 +++++++++++++++++++ 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java index fc25193a9d..4c0020a83e 100644 --- a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java @@ -10,6 +10,7 @@ import com.linecorp.armeria.common.HttpData; import java.io.IOException; +import java.nio.charset.Charset; import java.util.ArrayList; import java.util.List; import java.util.Map; @@ -56,7 +57,7 @@ public List> parse(HttpData httpData, int maxSize) throws IOExcepti size = OVERHEAD_CHARACTERS.length(); } innerJsonList.add(recordString); - size += recordString.length() + COMMA_OVERHEAD_LENGTH; + size += recordString.getBytes(Charset.defaultCharset()).length + COMMA_OVERHEAD_LENGTH; } if (size > OVERHEAD_CHARACTERS.length()) { jsonList.add(innerJsonList); diff --git a/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java index 4863667bc0..8843d8d6e7 100644 --- a/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java +++ b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java @@ -7,16 +7,31 @@ import com.linecorp.armeria.common.HttpData; import org.junit.jupiter.api.Test; +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 java.io.IOException; +import java.nio.charset.Charset; import java.util.List; +import java.util.stream.Collectors; +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.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.params.provider.Arguments.arguments; class JsonCodecTest { private final HttpData goodTestData = HttpData.ofUtf8("[{\"a\":\"b\"}, {\"c\":\"d\"}]"); private final HttpData goodLargeTestData = HttpData.ofUtf8("[{\"a1\":\"b1\"}, {\"a2\":\"b2\"}, {\"a3\":\"b3\"}, {\"a4\":\"b4\"}, {\"a5\":\"b5\"}]"); + private final HttpData goodLargeTestDataUnicode = HttpData.ofUtf8("[{\"ὊὊὊ1\":\"ὊὊὊ1\"}, {\"ὊὊὊ2\":\"ὊὊὊ2\"}, {\"a3\":\"b3\"}, {\"ὊὊὊ4\":\"ὊὊὊ4\"}]"); private final HttpData badTestDataJsonLine = HttpData.ofUtf8("{\"a\":\"b\"}"); private final HttpData badTestDataMultiJsonLines = HttpData.ofUtf8("{\"a\":\"b\"}{\"c\":\"d\"}"); private final HttpData badTestDataNonJson = HttpData.ofUtf8("non json content"); @@ -51,6 +66,25 @@ public void testParseSuccessWithMaxSize() throws IOException { assertEquals("{\"a5\":\"b5\"}", res.get(2).get(0)); } + @ParameterizedTest + @ArgumentsSource(JsonArrayWithKnownFirstArgumentsProvider.class) + public void parse_should_return_lists_smaller_than_provided_length( + final String inputJsonArray, final String knownFirstPart) throws IOException { + final int knownSingleBodySize = knownFirstPart.getBytes(Charset.defaultCharset()).length; + final int maxSize = (knownSingleBodySize * 2) + 3; + final List> chunkedBodies = objectUnderTest.parse(HttpData.ofUtf8(inputJsonArray), + maxSize); + + assertThat(chunkedBodies, notNullValue()); + assertThat(chunkedBodies.size(), greaterThanOrEqualTo(1)); + final String firstReconstructed = chunkedBodies.get(0).stream().collect(Collectors.joining(",", "[", "]")); + assertThat(firstReconstructed.getBytes(Charset.defaultCharset()).length, + lessThanOrEqualTo(maxSize)); + + assertThat(chunkedBodies.get(0).size(), greaterThanOrEqualTo(1)); + assertThat(chunkedBodies.get(0).get(0), equalTo(knownFirstPart)); + } + @Test public void testParseJsonLineFailure() { assertThrows(IOException.class, () -> objectUnderTest.parse(badTestDataJsonLine)); @@ -65,4 +99,18 @@ public void testParseMultiJsonLinesFailure() { public void testParseNonJsonFailure() { assertThrows(IOException.class, () -> objectUnderTest.parse(badTestDataNonJson)); } + + static class JsonArrayWithKnownFirstArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(ExtensionContext extensionContext) throws Exception { + return Stream.of( + arguments( + "[{\"ὊὊὊ1\":\"ὊὊὊ1\"}, {\"ὊὊὊ2\":\"ὊὊὊ2\"}, {\"a3\":\"b3\"}, {\"ὊὊὊ4\":\"ὊὊὊ4\"}]", + "{\"ὊὊὊ1\":\"ὊὊὊ1\"}"), + arguments( + "[{\"aaa1\":\"aaa1\"}, {\"aaa2\":\"aaa2\"}, {\"a3\":\"b3\"}, {\"bbb4\":\"bbb4\"}]", + "{\"aaa1\":\"aaa1\"}") + ); + } + } } \ No newline at end of file From c5fe1b48990e659b2e0e7b45d5f4ee916c24ce2a Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 26 Jun 2024 07:15:04 -0700 Subject: [PATCH 038/159] Support default route option for Events that match no other route (#4662) Support default route option for Events that match no other route Signed-off-by: Krishna Kondaka --- .../Router_ThreeRoutesDefaultIT.java | 130 ++++++++++++++++++ .../route/three-route-with-default-route.yaml | 41 ++++++ .../router/DataFlowComponentRouter.java | 5 +- .../router/DataFlowComponentRouterTest.java | 38 +++++ 4 files changed, 213 insertions(+), 1 deletion(-) create mode 100644 data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Router_ThreeRoutesDefaultIT.java create mode 100644 data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/route/three-route-with-default-route.yaml diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Router_ThreeRoutesDefaultIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Router_ThreeRoutesDefaultIT.java new file mode 100644 index 0000000000..fbc61053a5 --- /dev/null +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Router_ThreeRoutesDefaultIT.java @@ -0,0 +1,130 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.integration; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.InMemorySinkAccessor; +import org.opensearch.dataprepper.plugins.InMemorySourceAccessor; +import org.opensearch.dataprepper.test.framework.DataPrepperTestRunner; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.awaitility.Awaitility.await; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.empty; + +class Router_ThreeRoutesDefaultIT { + private static final String TESTING_KEY = "ConditionalRoutingIT"; + private static final String ALL_SOURCE_KEY = TESTING_KEY + "_all"; + private static final String ALPHA_SOURCE_KEY = TESTING_KEY + "_alpha"; + private static final String BETA_SOURCE_KEY = TESTING_KEY + "_beta"; + private static final String ALPHA_DEFAULT_SOURCE_KEY = TESTING_KEY + "_alpha_default"; + private static final String ALPHA_BETA_GAMMA_SOURCE_KEY = TESTING_KEY + "_alpha_beta_gamma"; + private static final String DEFAULT_SOURCE_KEY = TESTING_KEY + "_default"; + private static final String KNOWN_CONDITIONAL_KEY = "value"; + private static final String ALPHA_VALUE = "a"; + private static final String BETA_VALUE = "b"; + private static final String GAMMA_VALUE = "g"; + private static final String DEFAULT_VALUE = "z"; + private DataPrepperTestRunner dataPrepperTestRunner; + private InMemorySourceAccessor inMemorySourceAccessor; + private InMemorySinkAccessor inMemorySinkAccessor; + + @BeforeEach + void setUp() { + dataPrepperTestRunner = DataPrepperTestRunner.builder() + .withPipelinesDirectoryOrFile("route/three-route-with-default-route.yaml") + .build(); + + dataPrepperTestRunner.start(); + inMemorySourceAccessor = dataPrepperTestRunner.getInMemorySourceAccessor(); + inMemorySinkAccessor = dataPrepperTestRunner.getInMemorySinkAccessor(); + } + + @AfterEach + void tearDown() { + dataPrepperTestRunner.stop(); + } + + @Test + void test_default_route() { + final List> alphaEvents = createEvents(ALPHA_VALUE, 10); + final List> betaEvents = createEvents(BETA_VALUE, 20); + final List> gammaEvents = createEvents(GAMMA_VALUE, 20); + final List> defaultEvents = createEvents(DEFAULT_VALUE, 20); + + final List> allEvents = new ArrayList<>(alphaEvents); + allEvents.addAll(betaEvents); + allEvents.addAll(gammaEvents); + allEvents.addAll(defaultEvents); + Collections.shuffle(allEvents); + + inMemorySourceAccessor.submit(TESTING_KEY, allEvents); + + await().atMost(2, TimeUnit.SECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(ALPHA_SOURCE_KEY), not(empty())); + assertThat(inMemorySinkAccessor.get(BETA_SOURCE_KEY), not(empty())); + assertThat(inMemorySinkAccessor.get(ALL_SOURCE_KEY), not(empty())); + assertThat(inMemorySinkAccessor.get(ALPHA_DEFAULT_SOURCE_KEY), not(empty())); + assertThat(inMemorySinkAccessor.get(ALPHA_BETA_GAMMA_SOURCE_KEY), not(empty())); + assertThat(inMemorySinkAccessor.get(DEFAULT_SOURCE_KEY), not(empty())); + }); + + final List> actualAlphaRecords = inMemorySinkAccessor.get(ALPHA_SOURCE_KEY); + + assertThat(actualAlphaRecords.size(), equalTo(alphaEvents.size())); + + assertThat(actualAlphaRecords, containsInAnyOrder(allEvents.stream() + .filter(alphaEvents::contains).toArray())); + + final List> actualBetaRecords = inMemorySinkAccessor.get(BETA_SOURCE_KEY); + + assertThat(actualBetaRecords.size(), equalTo(betaEvents.size())); + + assertThat(actualBetaRecords, containsInAnyOrder(allEvents.stream() + .filter(betaEvents::contains).toArray())); + + final List> actualDefaultRecords = inMemorySinkAccessor.get(DEFAULT_SOURCE_KEY); + + assertThat(actualDefaultRecords.size(), equalTo(defaultEvents.size())); + assertThat(actualDefaultRecords, containsInAnyOrder(allEvents.stream() + .filter(defaultEvents::contains).toArray())); + + final List> actualAlphaDefaultRecords = new ArrayList<>(); + actualAlphaDefaultRecords.addAll(actualAlphaRecords); + actualAlphaDefaultRecords.addAll(actualDefaultRecords); + assertThat(actualAlphaDefaultRecords.size(), equalTo(defaultEvents.size()+alphaEvents.size())); + assertThat(actualAlphaDefaultRecords, containsInAnyOrder(allEvents.stream() + .filter(event -> defaultEvents.contains(event) || alphaEvents.contains(event)).toArray())); + + } + + private List> createEvents(final String value, final int numberToCreate) { + return IntStream.range(0, numberToCreate) + .mapToObj(i -> Map.of(KNOWN_CONDITIONAL_KEY, value, "arbitrary_field", UUID.randomUUID().toString())) + .map(map -> JacksonEvent.builder().withData(map).withEventType("TEST").build()) + .map(jacksonEvent -> (Event) jacksonEvent) + .map(Record::new) + .collect(Collectors.toList()); + } +} + diff --git a/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/route/three-route-with-default-route.yaml b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/route/three-route-with-default-route.yaml new file mode 100644 index 0000000000..6d608a0d0b --- /dev/null +++ b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/route/three-route-with-default-route.yaml @@ -0,0 +1,41 @@ +routing-pipeline: + workers: 4 + delay: 10 + source: + in_memory: + testing_key: ConditionalRoutingIT + buffer: + bounded_blocking: + # Use a small batch size to help ensure that multiple threads + # are picking up the different routes. + batch_size: 10 + route: + - alpha: '/value == "a"' + - beta: '/value == "b"' + - gamma: '/value == "g"' + sink: + - in_memory: + testing_key: ConditionalRoutingIT_alpha + routes: + - alpha + - in_memory: + testing_key: ConditionalRoutingIT_beta + routes: + - beta + - in_memory: + testing_key: ConditionalRoutingIT_alpha_default + routes: + - alpha + - _default + - in_memory: + testing_key: ConditionalRoutingIT_alpha_beta_gamma + routes: + - alpha + - beta + - gamma + - in_memory: + testing_key: ConditionalRoutingIT_default + routes: + - _default + - in_memory: + testing_key: ConditionalRoutingIT_all diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouter.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouter.java index 1e8850219d..4e5c89cc29 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouter.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouter.java @@ -20,6 +20,7 @@ * intended to help break apart {@link Router} for better testing. */ class DataFlowComponentRouter { + static final String DEFAULT_ROUTE = "_default"; void route(final Collection allRecords, final DataFlowComponent dataFlowComponent, final Map> recordsToRoutes, @@ -37,7 +38,9 @@ void route(final Collection allRecords, final Set routesForEvent = recordsToRoutes .getOrDefault(record, Collections.emptySet()); - if (routesForEvent.stream().anyMatch(dataFlowComponentRoutes::contains)) { + if (routesForEvent.size() == 0 && dataFlowComponentRoutes.contains(DEFAULT_ROUTE)) { + recordsForComponent.add(getRecordStrategy.getRecord(record)); + } else if (routesForEvent.stream().anyMatch(dataFlowComponentRoutes::contains)) { recordsForComponent.add(getRecordStrategy.getRecord(record)); } } diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouterTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouterTest.java index 3802356592..1ea74afe70 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouterTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/router/DataFlowComponentRouterTest.java @@ -158,6 +158,17 @@ void route_no_Events_when_none_have_matching_routes() { verify(componentRecordsConsumer).accept(testComponent, Collections.emptyList()); } + @Test + void route_no_Events_when_none_have_matching_routes_with_default_route() { + when(dataFlowComponent.getRoutes()).thenReturn(Set.of(DataFlowComponentRouter.DEFAULT_ROUTE)); + final Map> noMatchingRoutes = recordsIn.stream() + .collect(Collectors.toMap(Function.identity(), r -> Collections.emptySet())); + + createObjectUnderTest().route(recordsIn, dataFlowComponent, noMatchingRoutes, getRecordStrategy, componentRecordsConsumer); + + verify(componentRecordsConsumer).accept(testComponent, recordsIn); + } + @Test void route_all_Events_when_all_have_matched_route() { @@ -236,6 +247,33 @@ void route_no_Events_when_none_have_matching_routes() { verify(componentRecordsConsumer).accept(testComponent, Collections.emptyList()); } + @Test + void route_no_Events_when_none_have_matching_routes_with_default_route() { + when(dataFlowComponent.getRoutes()).thenReturn(Set.of(DataFlowComponentRouter.DEFAULT_ROUTE)); + final Map> noMatchingRoutes = recordsIn.stream() + .collect(Collectors.toMap(Function.identity(), r -> Collections.emptySet())); + + createObjectUnderTest().route(recordsIn, dataFlowComponent, noMatchingRoutes, getRecordStrategy, componentRecordsConsumer); + + verify(componentRecordsConsumer).accept(testComponent, recordsIn); + } + + @Test + void route_matched_events_with_none_to_default_route() { + DataFlowComponent dataFlowComponent2 = mock(DataFlowComponent.class); + when(dataFlowComponent2.getRoutes()).thenReturn(Set.of(DataFlowComponentRouter.DEFAULT_ROUTE)); + final Map> allMatchingRoutes = recordsIn.stream() + .collect(Collectors.toMap(Function.identity(), r -> Collections.singleton(knownRoute))); + + createObjectUnderTest().route(recordsIn, dataFlowComponent2, allMatchingRoutes, getRecordStrategy, componentRecordsConsumer); + verify(componentRecordsConsumer).accept(null, Collections.emptyList()); + createObjectUnderTest().route(recordsIn, dataFlowComponent, allMatchingRoutes, getRecordStrategy, componentRecordsConsumer); + + verify(componentRecordsConsumer).accept(testComponent, recordsIn); + + } + + @Test void route_all_Events_when_all_have_matched_route() { From 36d599f1183f3bdfcef32183c80f573bb9a20e6c Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 26 Jun 2024 14:25:57 -0500 Subject: [PATCH 039/159] Enhanced Kafka source logging through the use of MDC and better thread names for Kafka source threads. Resolves #4126. (#4663) Signed-off-by: David Venable --- .../plugins/kafka/common/KafkaMdc.java | 4 +- .../thread/KafkaPluginThreadFactory.java | 33 ++++- .../plugins/kafka/source/KafkaSource.java | 132 ++++++++++-------- .../thread/KafkaPluginThreadFactoryTest.java | 73 ++++++++++ .../plugins/kafka/source/KafkaSourceTest.java | 40 ++++++ 5 files changed, 225 insertions(+), 57 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/KafkaMdc.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/KafkaMdc.java index 9ae8985908..785d565e78 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/KafkaMdc.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/KafkaMdc.java @@ -3,6 +3,8 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.kafka.common;public class KafkaMdc { +package org.opensearch.dataprepper.plugins.kafka.common; + +public class KafkaMdc { public static final String MDC_KAFKA_PLUGIN_KEY = "kafkaPluginType"; } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactory.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactory.java index a05540c320..b5dede6cda 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactory.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactory.java @@ -25,7 +25,16 @@ public class KafkaPluginThreadFactory implements ThreadFactory { final ThreadFactory delegateThreadFactory, final String kafkaPluginType) { this.delegateThreadFactory = delegateThreadFactory; - this.threadPrefix = "kafka-" + kafkaPluginType + "-"; + this.threadPrefix = createPluginPart(kafkaPluginType); + this.kafkaPluginType = kafkaPluginType; + } + + KafkaPluginThreadFactory( + final ThreadFactory delegateThreadFactory, + final String kafkaPluginType, + final String kafkaTopic) { + this.delegateThreadFactory = delegateThreadFactory; + this.threadPrefix = normalizeName(kafkaTopic) + "-" + createPluginPart(kafkaPluginType); this.kafkaPluginType = kafkaPluginType; } @@ -39,6 +48,28 @@ public static KafkaPluginThreadFactory defaultExecutorThreadFactory(final String return new KafkaPluginThreadFactory(Executors.defaultThreadFactory(), kafkaPluginType); } + /** + * Creates an instance specifically for use with {@link Executors}. + * + * @param kafkaPluginType The name of the plugin type. e.g. sink, source, buffer + * @return An instance of the {@link KafkaPluginThreadFactory}. + */ + public static KafkaPluginThreadFactory defaultExecutorThreadFactory( + final String kafkaPluginType, + final String kafkaTopic) { + return new KafkaPluginThreadFactory(Executors.defaultThreadFactory(), kafkaPluginType, kafkaTopic); + } + + private static String createPluginPart(final String kafkaPluginType) { + return "kafka-" + kafkaPluginType + "-"; + } + + private static String normalizeName(final String kafkaTopic) { + final String limitedName = kafkaTopic.length() > 20 ? kafkaTopic.substring(0, 20) : kafkaTopic; + return limitedName + .toLowerCase().replaceAll("[^a-z0-9]", "-"); + } + @Override public Thread newThread(final Runnable runnable) { final Thread thread = delegateThreadFactory.newThread(() -> { 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 525c754929..e235594ce2 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 @@ -29,6 +29,8 @@ import org.opensearch.dataprepper.model.plugin.PluginConfigObservable; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.plugins.kafka.common.KafkaMdc; +import org.opensearch.dataprepper.plugins.kafka.common.thread.KafkaPluginThreadFactory; import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConsumerConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.OAuthConfig; @@ -46,6 +48,7 @@ import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.slf4j.MDC; import java.io.IOException; import java.util.ArrayList; @@ -73,10 +76,10 @@ public class KafkaSource implements Source> { private static final String NO_RESOLVABLE_URLS_ERROR_MESSAGE = "No resolvable bootstrap urls given in bootstrap.servers"; private static final long RETRY_SLEEP_INTERVAL = 30000; + private static final String MDC_KAFKA_PLUGIN_VALUE = "source"; private static final Logger LOG = LoggerFactory.getLogger(KafkaSource.class); private final KafkaSourceConfig sourceConfig; private final AtomicBoolean shutdownInProgress; - private ExecutorService executorService; private final PluginMetrics pluginMetrics; private KafkaCustomConsumer consumer; private KafkaConsumer kafkaConsumer; @@ -112,59 +115,65 @@ public KafkaSource(final KafkaSourceConfig sourceConfig, @Override public void start(Buffer> buffer) { - Properties authProperties = new Properties(); - KafkaSecurityConfigurer.setDynamicSaslClientCallbackHandler(authProperties, sourceConfig, pluginConfigObservable); - KafkaSecurityConfigurer.setAuthProperties(authProperties, sourceConfig, LOG); - sourceConfig.getTopics().forEach(topic -> { - consumerGroupID = topic.getGroupId(); - KafkaTopicConsumerMetrics topicMetrics = new KafkaTopicConsumerMetrics(topic.getName(), pluginMetrics, true); - Properties consumerProperties = getConsumerProperties(topic, authProperties); - MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType); - try { - int numWorkers = topic.getWorkers(); - executorService = Executors.newFixedThreadPool(numWorkers); - allTopicExecutorServices.add(executorService); - - IntStream.range(0, numWorkers).forEach(index -> { - while (true) { - try { - kafkaConsumer = createKafkaConsumer(schema, consumerProperties); - break; - } catch (ConfigException ce) { - if (ce.getMessage().contains(NO_RESOLVABLE_URLS_ERROR_MESSAGE)) { - LOG.warn("Exception while creating Kafka consumer: ", ce); - LOG.warn("Bootstrap URL could not be resolved. Retrying in {} ms...", RETRY_SLEEP_INTERVAL); - try { - sleep(RETRY_SLEEP_INTERVAL); - } catch (InterruptedException ie) { - Thread.currentThread().interrupt(); - throw new RuntimeException(ie); + try { + setMdc(); + Properties authProperties = new Properties(); + KafkaSecurityConfigurer.setDynamicSaslClientCallbackHandler(authProperties, sourceConfig, pluginConfigObservable); + KafkaSecurityConfigurer.setAuthProperties(authProperties, sourceConfig, LOG); + sourceConfig.getTopics().forEach(topic -> { + consumerGroupID = topic.getGroupId(); + KafkaTopicConsumerMetrics topicMetrics = new KafkaTopicConsumerMetrics(topic.getName(), pluginMetrics, true); + Properties consumerProperties = getConsumerProperties(topic, authProperties); + MessageFormat schema = MessageFormat.getByMessageFormatByName(schemaType); + try { + int numWorkers = topic.getWorkers(); + final ExecutorService executorService = Executors.newFixedThreadPool( + numWorkers, KafkaPluginThreadFactory.defaultExecutorThreadFactory(MDC_KAFKA_PLUGIN_VALUE, topic.getName())); + allTopicExecutorServices.add(executorService); + + IntStream.range(0, numWorkers).forEach(index -> { + while (true) { + try { + kafkaConsumer = createKafkaConsumer(schema, consumerProperties); + break; + } catch (ConfigException ce) { + if (ce.getMessage().contains(NO_RESOLVABLE_URLS_ERROR_MESSAGE)) { + LOG.warn("Exception while creating Kafka consumer: ", ce); + LOG.warn("Bootstrap URL could not be resolved. Retrying in {} ms...", RETRY_SLEEP_INTERVAL); + try { + sleep(RETRY_SLEEP_INTERVAL); + } catch (InterruptedException ie) { + Thread.currentThread().interrupt(); + throw new RuntimeException(ie); + } + } else { + throw ce; } - } else { - throw ce; } + } + consumer = new KafkaCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topic, schemaType, + acknowledgementSetManager, null, topicMetrics, PauseConsumePredicate.noPause()); + allTopicConsumers.add(consumer); + executorService.submit(consumer); + }); + } catch (Exception e) { + if (e instanceof BrokerNotAvailableException || e instanceof TimeoutException) { + LOG.error("The kafka broker is not available..."); + } else { + LOG.error("Failed to setup the Kafka Source Plugin.", e); } - consumer = new KafkaCustomConsumer(kafkaConsumer, shutdownInProgress, buffer, sourceConfig, topic, schemaType, - acknowledgementSetManager, null, topicMetrics, PauseConsumePredicate.noPause()); - allTopicConsumers.add(consumer); - - executorService.submit(consumer); - }); - } catch (Exception e) { - if (e instanceof BrokerNotAvailableException || e instanceof TimeoutException) { - LOG.error("The kafka broker is not available..."); - } else { - LOG.error("Failed to setup the Kafka Source Plugin.", e); + throw new RuntimeException(e); } - throw new RuntimeException(e); - } - LOG.info("Started Kafka source for topic " + topic.getName()); - }); + LOG.info("Started Kafka source for topic " + topic.getName()); + }); + } finally { + removeMdc(); + } } - public KafkaConsumer createKafkaConsumer(final MessageFormat schema, final Properties consumerProperties) { + KafkaConsumer createKafkaConsumer(final MessageFormat schema, final Properties consumerProperties) { switch (schema) { case JSON: return new KafkaConsumer(consumerProperties); @@ -183,19 +192,24 @@ public void start(Buffer> buffer) { @Override public void stop() { - shutdownInProgress.set(true); - final long shutdownWaitTime = calculateLongestThreadWaitingTime(); + try { + setMdc(); + shutdownInProgress.set(true); + final long shutdownWaitTime = calculateLongestThreadWaitingTime(); - LOG.info("Shutting down {} Executor services", allTopicExecutorServices.size()); - allTopicExecutorServices.forEach(executor -> stopExecutor(executor, shutdownWaitTime)); + LOG.info("Shutting down {} Executor services", allTopicExecutorServices.size()); + allTopicExecutorServices.forEach(executor -> stopExecutor(executor, shutdownWaitTime)); - LOG.info("Closing {} consumers", allTopicConsumers.size()); - allTopicConsumers.forEach(consumer -> consumer.closeConsumer()); + LOG.info("Closing {} consumers", allTopicConsumers.size()); + allTopicConsumers.forEach(consumer -> consumer.closeConsumer()); - LOG.info("Kafka source shutdown successfully..."); + LOG.info("Kafka source shutdown successfully..."); + } finally { + removeMdc(); + } } - public void stopExecutor(final ExecutorService executorService, final long shutdownWaitTime) { + private void stopExecutor(final ExecutorService executorService, final long shutdownWaitTime) { executorService.shutdown(); try { if (!executorService.awaitTermination(shutdownWaitTime, TimeUnit.SECONDS)) { @@ -346,7 +360,7 @@ private void setPropertiesForSchemaRegistryConnectivity(Properties properties) { } } - protected void sleep(final long millis) throws InterruptedException { + void sleep(final long millis) throws InterruptedException { Thread.sleep(millis); } @@ -366,4 +380,12 @@ private void updateConfig(final KafkaClusterConfigSupplier kafkaClusterConfigSup } } } + + private static void setMdc() { + MDC.put(KafkaMdc.MDC_KAFKA_PLUGIN_KEY, MDC_KAFKA_PLUGIN_VALUE); + } + + private static void removeMdc() { + MDC.remove(KafkaMdc.MDC_KAFKA_PLUGIN_KEY); + } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactoryTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactoryTest.java index 589f81a74c..1f1bc854dc 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactoryTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactoryTest.java @@ -8,6 +8,8 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; @@ -37,10 +39,12 @@ class KafkaPluginThreadFactoryTest { @Mock private Runnable runnable; private String pluginType; + private String topic; @BeforeEach void setUp() { pluginType = UUID.randomUUID().toString(); + topic = UUID.randomUUID().toString(); when(delegateThreadFactory.newThread(any(Runnable.class))).thenReturn(innerThread); } @@ -50,11 +54,20 @@ private KafkaPluginThreadFactory createObjectUnderTest() { return new KafkaPluginThreadFactory(delegateThreadFactory, pluginType); } + private KafkaPluginThreadFactory createObjectUnderTestWithTopic() { + return new KafkaPluginThreadFactory(delegateThreadFactory, pluginType, topic); + } + @Test void newThread_creates_thread_from_delegate() { assertThat(createObjectUnderTest().newThread(runnable), equalTo(innerThread)); } + @Test + void newThread_with_topic_creates_thread_from_delegate() { + assertThat(createObjectUnderTestWithTopic().newThread(runnable), equalTo(innerThread)); + } + @Test void newThread_creates_thread_with_name() { final KafkaPluginThreadFactory objectUnderTest = createObjectUnderTest(); @@ -69,6 +82,30 @@ void newThread_creates_thread_with_name() { verify(thread2).setName(String.format("kafka-%s-2", pluginType)); } + @ParameterizedTest + @CsvSource({ + "abcd12,abcd12", + "aBCd12,abcd12", + "abcd-12,abcd-12", + "has space,has-space", + "has!character,has-character", + "this-is-somewhat-too-long,this-is-somewhat-too" + }) + void newThread_with_topic_creates_thread_with_name( + final String topicName, + final String expectedPrefix) { + this.topic = topicName; + final KafkaPluginThreadFactory objectUnderTest = createObjectUnderTestWithTopic(); + + final Thread thread1 = objectUnderTest.newThread(runnable); + assertThat(thread1, notNullValue()); + verify(thread1).setName(String.format("%s-kafka-%s-1", expectedPrefix, pluginType)); + + final Thread thread2 = objectUnderTest.newThread(runnable); + assertThat(thread2, notNullValue()); + verify(thread2).setName(String.format("%s-kafka-%s-2", expectedPrefix, pluginType)); + } + @Test void newThread_creates_thread_with_wrapping_runnable() { createObjectUnderTest().newThread(runnable); @@ -85,6 +122,22 @@ void newThread_creates_thread_with_wrapping_runnable() { verify(runnable).run(); } + @Test + void newThread_with_topic_creates_thread_with_wrapping_runnable() { + createObjectUnderTestWithTopic().newThread(runnable); + + final ArgumentCaptor actualRunnableCaptor = ArgumentCaptor.forClass(Runnable.class); + verify(delegateThreadFactory).newThread(actualRunnableCaptor.capture()); + + final Runnable actualRunnable = actualRunnableCaptor.getValue(); + + assertThat(actualRunnable, not(equalTo(runnable))); + + verifyNoInteractions(runnable); + actualRunnable.run(); + verify(runnable).run(); + } + @Test void newThread_creates_thread_that_calls_MDC_on_run() { createObjectUnderTest().newThread(runnable); @@ -104,4 +157,24 @@ void newThread_creates_thread_that_calls_MDC_on_run() { assertThat(actualKafkaPluginType[0], equalTo(pluginType)); } + + @Test + void newThread_with_topic_creates_thread_that_calls_MDC_on_run() { + createObjectUnderTestWithTopic().newThread(runnable); + + final ArgumentCaptor actualRunnableCaptor = ArgumentCaptor.forClass(Runnable.class); + verify(delegateThreadFactory).newThread(actualRunnableCaptor.capture()); + + final Runnable actualRunnable = actualRunnableCaptor.getValue(); + + final String[] actualKafkaPluginType = new String[1]; + doAnswer(a -> { + actualKafkaPluginType[0] = MDC.get(KafkaMdc.MDC_KAFKA_PLUGIN_KEY); + return null; + }).when(runnable).run(); + + actualRunnable.run(); + + assertThat(actualKafkaPluginType[0], equalTo(pluginType)); + } } \ No newline at end of file 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 1503a7424d..3433a92b76 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 @@ -6,11 +6,14 @@ package org.opensearch.dataprepper.plugins.kafka.source; import org.apache.kafka.common.config.ConfigException; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; 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.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; @@ -21,6 +24,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.plugin.PluginConfigObservable; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.kafka.common.KafkaMdc; import org.opensearch.dataprepper.plugins.kafka.configuration.AuthConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.AwsConfig; import org.opensearch.dataprepper.plugins.kafka.configuration.TopicConsumerConfig; @@ -29,6 +33,7 @@ import org.opensearch.dataprepper.plugins.kafka.configuration.SchemaConfig; import org.opensearch.dataprepper.plugins.kafka.extension.KafkaClusterConfigSupplier; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; +import org.slf4j.MDC; import java.time.Duration; import java.util.Collections; @@ -41,6 +46,7 @@ import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.never; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; @@ -230,4 +236,38 @@ void test_updateConfig_not_using_kafkaClusterConfigExtension() { verify(sourceConfig, never()).setAwsConfig(any()); verify(sourceConfig, never()).setEncryptionConfig(any()); } + + @Nested + class MdcTests { + private MockedStatic mdcMockedStatic; + + @BeforeEach + void setUp() { + mdcMockedStatic = mockStatic(MDC.class); + } + + @AfterEach + void tearDown() { + mdcMockedStatic.close(); + } + + @Test + void start_sets_and_removes_MDC() { + when(topic1.getSessionTimeOut()).thenReturn(Duration.ofSeconds(15)); + when(topic2.getSessionTimeOut()).thenReturn(Duration.ofSeconds(15)); + + createObjectUnderTest().start(buffer); + + mdcMockedStatic.verify(() -> MDC.put(KafkaMdc.MDC_KAFKA_PLUGIN_KEY, "source")); + mdcMockedStatic.verify(() -> MDC.remove(KafkaMdc.MDC_KAFKA_PLUGIN_KEY)); + } + + @Test + void stop_sets_and_removes_MDC() { + createObjectUnderTest().stop(); + + mdcMockedStatic.verify(() -> MDC.put(KafkaMdc.MDC_KAFKA_PLUGIN_KEY, "source")); + mdcMockedStatic.verify(() -> MDC.remove(KafkaMdc.MDC_KAFKA_PLUGIN_KEY)); + } + } } From f27b8083ce618ef101e048bd3580d65f54ecff10 Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 27 Jun 2024 19:47:18 -0500 Subject: [PATCH 040/159] Adds the TRIAGING.md file to outline our triaging process (#4630) Adds the TRIAGING.md file, which outlines for the community the Data Prepper triaging process. Signed-off-by: David Venable --- TRIAGING.md | 73 +++++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 73 insertions(+) create mode 100644 TRIAGING.md diff --git a/TRIAGING.md b/TRIAGING.md new file mode 100644 index 0000000000..a4a25e1932 --- /dev/null +++ b/TRIAGING.md @@ -0,0 +1,73 @@ +Data Prepper + +The Data Prepper maintainers seek to promote an inclusive and engaged community of contributors. +In order to facilitate this, weekly triage meetings are open to all and attendance is encouraged for anyone who hopes to contribute, discuss an issue, or learn more about the project. +To learn more about contributing to the Data Prepper project visit the [Contributing](./CONTRIBUTING.md) documentation. + +### Do I need to attend for my issue to be addressed/triaged? + +Attendance is not required for your issue to be triaged or addressed. +All new issues are triaged weekly. + +### What happens if my issue does not get covered this time? + +Each meeting we seek to address all new issues. +However, should we run out of time before your issue is discussed, you are always welcome to attend the next meeting or to follow up on the issue post itself. + +### How do I join the triage meeting? + +Meetings are hosted regularly Tuesdays at 2:30 PM US Central Time (12:30 PM Pacific Time) and can be joined via the links posted on the [OpenSearch Meetup Group](https://www.meetup.com/opensearch/events/) list of events. +The event will be titled `Data Prepper Triage Meeting`. + +After joining the Zoom meeting, you can enable your video / voice to join the discussion. +If you do not have a webcam or microphone available, you can still join in via the text chat. + +If you have an issue you'd like to bring forth please consider getting a link to the issue so it can be presented to everyone in the meeting. + +### Is there an agenda for each week? + +Meetings are 30 minutes and structured as follows: + +1. Initial Gathering: As we gather, feel free to turn on video and engage in informal and open-to-all conversation. A volunteer Data Prepper maintainer will share the [Data Prepper Tracking Board](https://github.com/orgs/opensearch-project/projects/82/) and proceed. +2. Announcements: We will make any announcements at the beginning, if necessary. +3. Untriaged issues: We will review all untriaged [issues](https://github.com/orgs/opensearch-project/projects/82/views/6) for the Data Prepper repository. If you have an item here, you may spend a few minutes to explain your request. +4. Member Requests: Opportunity for any meeting member to ask for consideration of an issue or pull request. +5. Release review: If time permits, and we find it necessary, we will review [items for the current release](https://github.com/orgs/opensearch-project/projects/82/views/14). +6. Follow-up issues: If time permits, we will review the [follow up items](https://github.com/orgs/opensearch-project/projects/82/views/18). +7. Open Discussion: If time permits, allow for members of the meeting to surface any topics without issues filed or pull request created. + +### Do I need to have already contributed to the project to attend a triage meeting? + +No, all are welcome and encouraged to attend. +Attending the triage meetings is a great way for a new contributor to learn about the project as well as explore different avenues of contribution. + +### What if I have follow-up questions on an issue? + +If you have an existing issue you would like to discuss, you can always comment on the issue itself. +Alternatively, you are welcome to come to the triage meeting to discuss. + +### Is this meeting a good place to get help using Data Prepper? + +While we are always happy to help the community, the best resource for usage questions is the [the Data Prepper discussion forum](https://github.com/opensearch-project/data-prepper/discussions) on GitHub. + +There you can find answers to many common questions as well as speak with implementation experts and Data Prepper maintainers. + +### What are the issue labels associated with triaging? + +There are several labels that are particularly important for triaging in Data Prepper: + +| Label | When applied | Meaning | +| ----- | ------------ | ------- | +| [untriaged](https://github.com/opensearch-project/data-prepper/labels/untriaged) | When issues are created or re-opened. | Issues labeled as `untriaged` require the attention of the repository maintainers and may need to be prioritized for quicker resolution. It's crucial to keep the count of 'untriaged' labels low to ensure all potential security issues are addressed in a timely manner. | +| [follow up](https://github.com/opensearch-project/data-prepper/labels/follow%20up) | During triage meetings. | Issues labeled as `follow up` have been triaged. However, the maintainers may need to follow up further on it. This tag lets us triage an issue as not critical, but also be able to come back to it. +| [help wanted](https://github.com/opensearch-project/data-prepper/labels/help%20wanted) | Anytime. | Issues marked as `help wanted` signal that they are actionable and not the current focus of the project maintainers. Community contributions are especially encouraged for these issues. | +| [good first issue](https://github.com/opensearch-project/data-prepper/labels/good%20first%20issue) | Anytime. | Issues labeled as `good first issue` are small in scope and can be resolved with a single pull request. These are recommended starting points for newcomers looking to make their first contributions. | + + +### Is this where I should bring up potential security vulnerabilities? + +Due to the sensitive nature of security vulnerabilities, please report all potential vulnerabilities directly by following the steps outlined in the [Security Issue Response Process](https://github.com/opensearch-project/data-prepper/security/policy). + +### Who should I contact if I have further questions? + +You can always file an [issue](https://github.com/opensearch-project/data-prepper/issues/new/choose) for any question you have about the project. From 153828889359e90563387a7e2c262e12d2ccf6e5 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 1 Jul 2024 15:54:53 -0500 Subject: [PATCH 041/159] Updates our usage of the Apache Parquet project to use their new interfaces over the old Hadoop ones. This allows use to be ready to extract Hadoop as other changes are made to the Parquet project. Remove some Hadoop transitive dependencies and make Hadoop runtime only where possible. Added a test for INT96, clean up some test files. Contributes toward #4612. (#4623) Signed-off-by: David Venable --- .../parquet-codecs/build.gradle | 16 +++-- .../codec/parquet/ParquetInputCodec.java | 10 +-- .../codec/parquet/ParquetInputCodecTest.java | 64 +++++++++++++++++- .../resources/sample.snappy.parquet | Bin .../resources/test-parquet.parquet | Bin data-prepper-plugins/s3-sink/build.gradle | 5 +- .../sink/s3/ParquetOutputScenario.java | 3 +- .../plugins/sink/s3/S3SinkServiceIT.java | 7 +- .../codec/parquet/ParquetOutputCodecTest.java | 7 +- 9 files changed, 92 insertions(+), 20 deletions(-) rename data-prepper-plugins/parquet-codecs/src/{main => test}/resources/sample.snappy.parquet (100%) rename data-prepper-plugins/parquet-codecs/src/{main => test}/resources/test-parquet.parquet (100%) diff --git a/data-prepper-plugins/parquet-codecs/build.gradle b/data-prepper-plugins/parquet-codecs/build.gradle index ea783c53d4..dd59e28068 100644 --- a/data-prepper-plugins/parquet-codecs/build.gradle +++ b/data-prepper-plugins/parquet-codecs/build.gradle @@ -7,16 +7,24 @@ dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:common') implementation libs.avro.core - implementation libs.hadoop.common - implementation(libs.hadoop.mapreduce) { - exclude group: 'org.apache.hadoop', module: 'hadoop-hdfs-client' - } + implementation 'org.apache.commons:commons-text:1.11.0' implementation 'org.apache.parquet:parquet-avro:1.14.0' implementation 'org.apache.parquet:parquet-column:1.14.0' implementation 'org.apache.parquet:parquet-common:1.14.0' implementation 'org.apache.parquet:parquet-hadoop:1.14.0' + runtimeOnly(libs.hadoop.common) { + exclude group: 'org.eclipse.jetty' + exclude group: 'org.apache.hadoop', module: 'hadoop-auth' + } + runtimeOnly(libs.hadoop.mapreduce) { + exclude group: 'org.apache.hadoop', module: 'hadoop-hdfs-client' + } testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-test-event') + testImplementation(libs.hadoop.common) { + exclude group: 'org.eclipse.jetty' + exclude group: 'org.apache.hadoop', module: 'hadoop-auth' + } constraints { implementation('com.nimbusds:nimbus-jose-jwt') { diff --git a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetInputCodec.java b/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetInputCodec.java index fa9876f114..e85e0c9926 100644 --- a/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetInputCodec.java +++ b/data-prepper-plugins/parquet-codecs/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetInputCodec.java @@ -6,8 +6,9 @@ package org.opensearch.dataprepper.plugins.codec.parquet; import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.conf.Configuration; import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.conf.ParquetConfiguration; +import org.apache.parquet.conf.PlainParquetConfiguration; import org.apache.parquet.hadoop.ParquetReader; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; @@ -46,13 +47,13 @@ public class ParquetInputCodec implements InputCodec { private static final Logger LOG = LoggerFactory.getLogger(ParquetInputCodec.class); - private final Configuration configuration; + private final ParquetConfiguration configuration; private final EventFactory eventFactory; @DataPrepperPluginConstructor public ParquetInputCodec(final EventFactory eventFactory) { this.eventFactory = eventFactory; - configuration = new Configuration(); + configuration = new PlainParquetConfiguration(); configuration.setBoolean(READ_INT96_AS_FIXED, true); } @@ -80,8 +81,7 @@ public void parse(final InputFile inputFile, final DecompressionEngine decompres } private void parseParquetFile(final InputFile inputFile, final Consumer> eventConsumer) throws IOException { - try (ParquetReader reader = AvroParquetReader.builder(inputFile) - .withConf(this.configuration) + try (ParquetReader reader = AvroParquetReader.builder(inputFile, this.configuration) .build()) { GenericRecordJsonEncoder encoder = new GenericRecordJsonEncoder(); GenericRecord record = null; diff --git a/data-prepper-plugins/parquet-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetInputCodecTest.java b/data-prepper-plugins/parquet-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetInputCodecTest.java index 1510ad75cc..5ae5f82d0d 100644 --- a/data-prepper-plugins/parquet-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetInputCodecTest.java +++ b/data-prepper-plugins/parquet-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetInputCodecTest.java @@ -8,8 +8,17 @@ import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.conf.PlainParquetConfiguration; +import org.apache.parquet.example.data.Group; +import org.apache.parquet.example.data.simple.NanoTime; +import org.apache.parquet.example.data.simple.SimpleGroup; import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.hadoop.example.ExampleParquetWriter; +import org.apache.parquet.hadoop.example.GroupWriteSupport; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -33,12 +42,15 @@ import java.net.URISyntaxException; import java.net.URL; import java.nio.file.Paths; +import java.time.OffsetDateTime; +import java.time.temporal.JulianFields; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.function.Consumer; +import static org.apache.parquet.avro.AvroWriteSupport.WRITE_FIXED_AS_INT96; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.startsWith; import static org.hamcrest.MatcherAssert.assertThat; @@ -161,6 +173,22 @@ public void parseInputFile_parsesCorrectly() throws IOException { assertRecordsCorrect(actualRecords); } + @Test + public void parseInputStream_parsesCorrectly_with_int96() throws IOException { + final File testDataFile = File.createTempFile(FILE_PREFIX + "-int96-", FILE_SUFFIX); + testDataFile.deleteOnExit(); + generateTestDataInt96(testDataFile); + InputStream targetStream = new FileInputStream(testDataFile); + + parquetInputCodec.parse(targetStream, mockConsumer); + + final ArgumentCaptor> recordArgumentCaptor = ArgumentCaptor.forClass(Record.class); + verify(mockConsumer, times(10)).accept(recordArgumentCaptor.capture()); + + final List> actualRecords = recordArgumentCaptor.getAllValues(); + assertThat(actualRecords.size(), equalTo(10)); + } + @Test public void parseInputFile_snappyInputFile() throws IOException, URISyntaxException { URL resource = getClass().getClassLoader().getResource("sample.snappy.parquet"); @@ -203,8 +231,10 @@ public void parseInputFile_testParquetFile() throws IOException, URISyntaxExcept private static void generateTestData(final File file) throws IOException { Schema schema = new Schema.Parser().parse(SCHEMA_JSON); - ParquetWriter writer = AvroParquetWriter.builder(new LocalOutputFile(file)) + final ParquetWriter writer = AvroParquetWriter.builder(new LocalOutputFile(file)) .withSchema(schema) + .withConf(new PlainParquetConfiguration()) + .withEncryption(null) .build(); for (int i = 0; i < 10; i++) { @@ -220,6 +250,34 @@ private static void generateTestData(final File file) throws IOException { writer.close(); } + /** + * Generates a Parquet file with INT96 data. This must use the example + * schema rather than Avro, or it would not correctly reproduce possible INT96 + * error. + * + * @param file The file for Parquet + */ + private static void generateTestDataInt96(final File file) throws IOException { + final MessageType schema = new MessageType("test", List.of( + new PrimitiveType(Type.Repetition.OPTIONAL, PrimitiveType.PrimitiveTypeName.INT96, "my_timestamp_value") + )); + final PlainParquetConfiguration conf = new PlainParquetConfiguration(); + conf.setStrings(WRITE_FIXED_AS_INT96, "my_timestamp_value"); + conf.set(GroupWriteSupport.PARQUET_EXAMPLE_SCHEMA, schema.toString()); + final ParquetWriter writer = ExampleParquetWriter.builder(new LocalOutputFile(file)) + .withConf(conf) + .withEncryption(null) + .build(); + + for (int i = 0; i < 10; i++) { + final Group group = new SimpleGroup(schema); + group.add("my_timestamp_value", createInt96()); + + writer.write(group); + } + writer.close(); + } + private void assertRecordsCorrect(final List> records) { assertThat(records.size(), equalTo(10)); for (int i = 0; i < 10; i++) { @@ -240,5 +298,9 @@ private void assertRecordsCorrect(final List> records) { assertThat(record.getData().getMetadata().getEventType(), equalTo(EVENT_TYPE)); } } + + private static NanoTime createInt96() { + return new NanoTime((int) OffsetDateTime.now().getLong(JulianFields.JULIAN_DAY), System.nanoTime()); + } } diff --git a/data-prepper-plugins/parquet-codecs/src/main/resources/sample.snappy.parquet b/data-prepper-plugins/parquet-codecs/src/test/resources/sample.snappy.parquet similarity index 100% rename from data-prepper-plugins/parquet-codecs/src/main/resources/sample.snappy.parquet rename to data-prepper-plugins/parquet-codecs/src/test/resources/sample.snappy.parquet diff --git a/data-prepper-plugins/parquet-codecs/src/main/resources/test-parquet.parquet b/data-prepper-plugins/parquet-codecs/src/test/resources/test-parquet.parquet similarity index 100% rename from data-prepper-plugins/parquet-codecs/src/main/resources/test-parquet.parquet rename to data-prepper-plugins/parquet-codecs/src/test/resources/test-parquet.parquet diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index 5a6b174900..638b8246a5 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -19,7 +19,10 @@ dependencies { implementation 'org.jetbrains.kotlin:kotlin-stdlib:1.9.22' implementation project(':data-prepper-plugins:avro-codecs') implementation libs.avro.core - implementation libs.hadoop.common + implementation(libs.hadoop.common) { + exclude group: 'org.eclipse.jetty' + exclude group: 'org.apache.hadoop', module: 'hadoop-auth' + } implementation 'org.apache.parquet:parquet-avro:1.14.0' implementation 'software.amazon.awssdk:apache-client' implementation 'org.jetbrains.kotlin:kotlin-stdlib-common:1.9.22' diff --git a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/ParquetOutputScenario.java b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/ParquetOutputScenario.java index 6e3abc3250..e01c61fe09 100644 --- a/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/ParquetOutputScenario.java +++ b/data-prepper-plugins/s3-sink/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/s3/ParquetOutputScenario.java @@ -9,6 +9,7 @@ import org.apache.avro.util.Utf8; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.conf.PlainParquetConfiguration; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.ParquetReader; import org.apache.parquet.hadoop.metadata.BlockMetaData; @@ -65,7 +66,7 @@ public void validate(int expectedRecords, final List> sample int validatedRecords = 0; int count = 0; - try (final ParquetReader reader = AvroParquetReader.builder(inputFile) + try (final ParquetReader reader = AvroParquetReader.builder(inputFile, new PlainParquetConfiguration()) .build()) { GenericRecord record; 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 739ac876df..b7bbb1b97d 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 @@ -12,8 +12,6 @@ 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; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.example.data.Group; @@ -21,8 +19,8 @@ import org.apache.parquet.example.data.simple.convert.GroupRecordConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.io.ColumnIOFactory; +import org.apache.parquet.io.LocalInputFile; import org.apache.parquet.io.MessageColumnIO; import org.apache.parquet.io.RecordReader; import org.apache.parquet.schema.MessageType; @@ -79,6 +77,7 @@ import java.io.InputStream; import java.nio.charset.Charset; import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.StandardCopyOption; import java.time.Duration; import java.util.ArrayList; @@ -413,7 +412,7 @@ private List> createParquetRecordsList(final InputStream final File tempFile = File.createTempFile(FILE_NAME, FILE_SUFFIX); Files.copy(inputStream, tempFile.toPath(), StandardCopyOption.REPLACE_EXISTING); List> actualRecordList = new ArrayList<>(); - try (ParquetFileReader parquetFileReader = new ParquetFileReader(HadoopInputFile.fromPath(new Path(tempFile.toURI()), new Configuration()), ParquetReadOptions.builder().build())) { + try (final ParquetFileReader parquetFileReader = new ParquetFileReader(new LocalInputFile(Path.of(tempFile.toURI())), ParquetReadOptions.builder().build())) { final ParquetMetadata footer = parquetFileReader.getFooter(); final MessageType schema = createdParquetSchema(footer); PageReadStore pages; diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java index b441a7a6e3..d6b4160888 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java @@ -6,8 +6,6 @@ import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.example.data.Group; @@ -15,8 +13,8 @@ import org.apache.parquet.example.data.simple.convert.GroupRecordConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.ParquetMetadata; -import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.io.ColumnIOFactory; +import org.apache.parquet.io.LocalInputFile; import org.apache.parquet.io.MessageColumnIO; import org.apache.parquet.io.RecordReader; import org.apache.parquet.schema.MessageType; @@ -46,6 +44,7 @@ import java.io.IOException; import java.io.InputStream; import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.StandardCopyOption; import java.util.ArrayList; import java.util.Collections; @@ -556,7 +555,7 @@ private List> createParquetRecordsList(final InputStream inp final File tempFile = new File(tempDirectory, FILE_NAME); Files.copy(inputStream, tempFile.toPath(), StandardCopyOption.REPLACE_EXISTING); List> actualRecordList = new ArrayList<>(); - try (ParquetFileReader parquetFileReader = new ParquetFileReader(HadoopInputFile.fromPath(new Path(tempFile.toURI()), new Configuration()), ParquetReadOptions.builder().build())) { + try (final ParquetFileReader parquetFileReader = new ParquetFileReader(new LocalInputFile(Path.of(tempFile.toURI())), ParquetReadOptions.builder().build())) { final ParquetMetadata footer = parquetFileReader.getFooter(); final MessageType schema = createdParquetSchema(footer); PageReadStore pages; From 87292fcb19eefb9cd830c76379ad72b5554d162d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 1 Jul 2024 14:42:31 -0700 Subject: [PATCH 042/159] Bump com.apptasticsoftware:rssreader in /data-prepper-plugins/rss-source (#4672) Bumps [com.apptasticsoftware:rssreader](https://github.com/w3stling/rssreader) from 3.6.0 to 3.7.0. - [Release notes](https://github.com/w3stling/rssreader/releases) - [Commits](https://github.com/w3stling/rssreader/compare/v3.6.0...v3.7.0) --- updated-dependencies: - dependency-name: com.apptasticsoftware:rssreader dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- data-prepper-plugins/rss-source/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/rss-source/build.gradle b/data-prepper-plugins/rss-source/build.gradle index 68c0ff9e57..686e40367b 100644 --- a/data-prepper-plugins/rss-source/build.gradle +++ b/data-prepper-plugins/rss-source/build.gradle @@ -13,7 +13,7 @@ dependencies { implementation 'joda-time:joda-time:2.12.7' implementation 'com.fasterxml.jackson.core:jackson-core' implementation 'com.fasterxml.jackson.core:jackson-databind' - implementation 'com.apptasticsoftware:rssreader:3.6.0' + implementation 'com.apptasticsoftware:rssreader:3.7.0' testImplementation libs.commons.lang3 testImplementation project(':data-prepper-test-common') testImplementation 'org.mock-server:mockserver-junit-jupiter-no-dependencies:5.15.0' From 6db5b471e3c07dc08f921b69e41a585b72825691 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 1 Jul 2024 14:43:01 -0700 Subject: [PATCH 043/159] Bump org.wiremock:wiremock in /data-prepper-plugins/s3-source (#4683) Bumps [org.wiremock:wiremock](https://github.com/wiremock/wiremock) from 3.4.2 to 3.8.0. - [Release notes](https://github.com/wiremock/wiremock/releases) - [Commits](https://github.com/wiremock/wiremock/compare/3.4.2...3.8.0) --- updated-dependencies: - dependency-name: org.wiremock:wiremock dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- data-prepper-plugins/s3-source/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/s3-source/build.gradle b/data-prepper-plugins/s3-source/build.gradle index 1187fa7ec0..2a09ce3d90 100644 --- a/data-prepper-plugins/s3-source/build.gradle +++ b/data-prepper-plugins/s3-source/build.gradle @@ -31,7 +31,7 @@ dependencies { implementation 'dev.failsafe:failsafe:3.3.2' implementation 'org.apache.httpcomponents:httpcore:4.4.16' testImplementation libs.commons.lang3 - testImplementation 'org.wiremock:wiremock:3.4.2' + testImplementation 'org.wiremock:wiremock:3.8.0' testImplementation 'org.eclipse.jetty:jetty-bom:11.0.20' testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation testLibs.junit.vintage From d12b2175c98e3a4232f07661b2535a35282f3226 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 1 Jul 2024 14:43:47 -0700 Subject: [PATCH 044/159] Bump ws from 7.5.9 to 7.5.10 in /release/staging-resources-cdk (#4639) Bumps [ws](https://github.com/websockets/ws) from 7.5.9 to 7.5.10. - [Release notes](https://github.com/websockets/ws/releases) - [Commits](https://github.com/websockets/ws/compare/7.5.9...7.5.10) --- updated-dependencies: - dependency-name: ws dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- release/staging-resources-cdk/package-lock.json | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/release/staging-resources-cdk/package-lock.json b/release/staging-resources-cdk/package-lock.json index 7ac1eaed21..32da99d8c9 100644 --- a/release/staging-resources-cdk/package-lock.json +++ b/release/staging-resources-cdk/package-lock.json @@ -7720,9 +7720,9 @@ } }, "node_modules/ws": { - "version": "7.5.9", - "resolved": "https://registry.npmjs.org/ws/-/ws-7.5.9.tgz", - "integrity": "sha512-F+P9Jil7UiSKSkppIiD94dN07AwvFixvLIj1Og1Rl9GGMuNipJnV9JzjD6XuqmAeiswGvUmNLjr5cFuXwNS77Q==", + "version": "7.5.10", + "resolved": "https://registry.npmjs.org/ws/-/ws-7.5.10.tgz", + "integrity": "sha512-+dbF1tHwZpXcbOJdVOkzLDxZP1ailvSxM6ZweXTegylPny803bFhA+vqBYw4s31NSAk4S2Qz+AKXK9a4wkdjcQ==", "dev": true, "engines": { "node": ">=8.3.0" @@ -13755,9 +13755,9 @@ } }, "ws": { - "version": "7.5.9", - "resolved": "https://registry.npmjs.org/ws/-/ws-7.5.9.tgz", - "integrity": "sha512-F+P9Jil7UiSKSkppIiD94dN07AwvFixvLIj1Og1Rl9GGMuNipJnV9JzjD6XuqmAeiswGvUmNLjr5cFuXwNS77Q==", + "version": "7.5.10", + "resolved": "https://registry.npmjs.org/ws/-/ws-7.5.10.tgz", + "integrity": "sha512-+dbF1tHwZpXcbOJdVOkzLDxZP1ailvSxM6ZweXTegylPny803bFhA+vqBYw4s31NSAk4S2Qz+AKXK9a4wkdjcQ==", "dev": true, "requires": {} }, From a8df481dfba3880d0b3121757fa6bb64c79afb70 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 1 Jul 2024 14:44:18 -0700 Subject: [PATCH 045/159] Bump braces from 3.0.2 to 3.0.3 in /testing/aws-testing-cdk (#4638) Bumps [braces](https://github.com/micromatch/braces) from 3.0.2 to 3.0.3. - [Changelog](https://github.com/micromatch/braces/blob/master/CHANGELOG.md) - [Commits](https://github.com/micromatch/braces/compare/3.0.2...3.0.3) --- updated-dependencies: - dependency-name: braces dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- testing/aws-testing-cdk/package-lock.json | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/testing/aws-testing-cdk/package-lock.json b/testing/aws-testing-cdk/package-lock.json index fbb7310d4f..c7ae43fe77 100644 --- a/testing/aws-testing-cdk/package-lock.json +++ b/testing/aws-testing-cdk/package-lock.json @@ -2310,12 +2310,12 @@ } }, "node_modules/braces": { - "version": "3.0.2", - "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.2.tgz", - "integrity": "sha512-b8um+L1RzM3WDSzvhm6gIz1yfTbBt6YTlcEKAvsmqCZZFw46z626lVj9j1yEPW33H5H+lBQpZMP1k8l+78Ha0A==", + "version": "3.0.3", + "resolved": "https://registry.npmjs.org/braces/-/braces-3.0.3.tgz", + "integrity": "sha512-yQbXgO/OSZVD2IsiLlro+7Hf6Q18EJrKSEsdoMzKePKXct3gvD8oLcOQdIzGupr5Fj+EDe8gO/lxc1BzfMpxvA==", "dev": true, "dependencies": { - "fill-range": "^7.0.1" + "fill-range": "^7.1.1" }, "engines": { "node": ">=8" @@ -3102,9 +3102,9 @@ } }, "node_modules/fill-range": { - "version": "7.0.1", - "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.0.1.tgz", - "integrity": "sha512-qOo9F+dMUmC2Lcb4BbVvnKJxTPjCm+RRpe4gDuGrzkL7mEVl/djYSu2OdQ2Pa302N4oqkSg9ir6jaLWJ2USVpQ==", + "version": "7.1.1", + "resolved": "https://registry.npmjs.org/fill-range/-/fill-range-7.1.1.tgz", + "integrity": "sha512-YsGpe3WHLK8ZYi4tWDg2Jy3ebRz2rXowDxnld4bkQB00cc/1Zw9AWnC0i9ztDJitivtQvaI9KaLyKrc+hBW0yg==", "dev": true, "dependencies": { "to-regex-range": "^5.0.1" From 60230ca1400e49844015c86234de57d2155d2aee Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 1 Jul 2024 14:45:24 -0700 Subject: [PATCH 046/159] Bump software.amazon.awssdk:auth in /performance-test (#4685) Bumps software.amazon.awssdk:auth from 2.25.21 to 2.26.12. --- updated-dependencies: - dependency-name: software.amazon.awssdk:auth dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- performance-test/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/performance-test/build.gradle b/performance-test/build.gradle index 8c4a9693d2..6dd99cb08d 100644 --- a/performance-test/build.gradle +++ b/performance-test/build.gradle @@ -15,7 +15,7 @@ configurations.all { group 'org.opensearch.dataprepper.test.performance' dependencies { - gatlingImplementation 'software.amazon.awssdk:auth:2.25.21' + gatlingImplementation 'software.amazon.awssdk:auth:2.26.12' implementation 'com.fasterxml.jackson.core:jackson-core' testRuntimeOnly testLibs.junit.engine From af6bce4fc3fd993ce368a22e24da2a75f7df4d8a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 1 Jul 2024 14:46:28 -0700 Subject: [PATCH 047/159] Bump org.apache.maven:maven-artifact in /data-prepper-plugins/opensearch (#4692) Bumps [org.apache.maven:maven-artifact](https://github.com/apache/maven) from 3.9.6 to 3.9.8. - [Release notes](https://github.com/apache/maven/releases) - [Commits](https://github.com/apache/maven/compare/maven-3.9.6...maven-3.9.8) --- updated-dependencies: - dependency-name: org.apache.maven:maven-artifact dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- data-prepper-plugins/opensearch/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/opensearch/build.gradle b/data-prepper-plugins/opensearch/build.gradle index 1f96c1f6ea..b87e533afe 100644 --- a/data-prepper-plugins/opensearch/build.gradle +++ b/data-prepper-plugins/opensearch/build.gradle @@ -36,7 +36,7 @@ dependencies { implementation 'software.amazon.awssdk:apache-client' implementation 'software.amazon.awssdk:netty-nio-client' implementation 'co.elastic.clients:elasticsearch-java:7.17.0' - implementation('org.apache.maven:maven-artifact:3.9.6') { + implementation('org.apache.maven:maven-artifact:3.9.8') { exclude group: 'org.codehaus.plexus' } testImplementation testLibs.junit.vintage From c6ca7ad5a18c1b8af28716b9395c2467290c51bf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 2 Jul 2024 07:18:48 -0700 Subject: [PATCH 048/159] Bump org.skyscreamer:jsonassert from 1.5.1 to 1.5.3 in /data-prepper-api (#4678) Bumps [org.skyscreamer:jsonassert](https://github.com/skyscreamer/JSONassert) from 1.5.1 to 1.5.3. - [Release notes](https://github.com/skyscreamer/JSONassert/releases) - [Changelog](https://github.com/skyscreamer/JSONassert/blob/master/CHANGELOG.md) - [Commits](https://github.com/skyscreamer/JSONassert/compare/jsonassert-1.5.1...jsonassert-1.5.3) --- updated-dependencies: - dependency-name: org.skyscreamer:jsonassert dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- data-prepper-api/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-api/build.gradle b/data-prepper-api/build.gradle index 4ee8e7316e..0ad43ff470 100644 --- a/data-prepper-api/build.gradle +++ b/data-prepper-api/build.gradle @@ -16,7 +16,7 @@ dependencies { testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation libs.commons.lang3 testImplementation project(':data-prepper-test-common') - testImplementation 'org.skyscreamer:jsonassert:1.5.1' + testImplementation 'org.skyscreamer:jsonassert:1.5.3' testImplementation libs.commons.io } From a466013db00aafe995830c57f2d892d839bbf22c Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Tue, 2 Jul 2024 09:26:16 -0700 Subject: [PATCH 049/159] Parquet codec tests fix (#4698) Parquet codec tests fix Signed-off-by: Krishna Kondaka --- .../codec/parquet/ParquetOutputCodecTest.java | 97 ++++++++++++++----- 1 file changed, 74 insertions(+), 23 deletions(-) diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java index d6b4160888..059b908aa4 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/codec/parquet/ParquetOutputCodecTest.java @@ -6,15 +6,18 @@ import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; import org.apache.parquet.ParquetReadOptions; import org.apache.parquet.column.page.PageReadStore; import org.apache.parquet.example.data.Group; +import org.apache.parquet.schema.GroupType; import org.apache.parquet.example.data.simple.SimpleGroup; import org.apache.parquet.example.data.simple.convert.GroupRecordConverter; import org.apache.parquet.hadoop.ParquetFileReader; import org.apache.parquet.hadoop.metadata.ParquetMetadata; +import org.apache.parquet.hadoop.util.HadoopInputFile; import org.apache.parquet.io.ColumnIOFactory; -import org.apache.parquet.io.LocalInputFile; import org.apache.parquet.io.MessageColumnIO; import org.apache.parquet.io.RecordReader; import org.apache.parquet.schema.MessageType; @@ -39,12 +42,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.IOException; +import java.io.FileInputStream; import java.io.InputStream; +import java.io.IOException; +import java.io.File; import java.nio.file.Files; -import java.nio.file.Path; import java.nio.file.StandardCopyOption; import java.util.ArrayList; import java.util.Collections; @@ -59,6 +61,7 @@ import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -114,11 +117,12 @@ void test_happy_case(final int numberOfRecords) throws Exception { parquetOutputCodec.writeEvent(event, outputStream); } parquetOutputCodec.complete(outputStream); - List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(tempFile.toString().getBytes())); + List> actualRecords = createParquetRecordsList(new FileInputStream(tempFile)); int index = 0; + assertThat(inputMaps.size(), equalTo(actualRecords.size())); for (final Map actualMap : actualRecords) { assertThat(actualMap, notNullValue()); - Map expectedMap = generateRecords(numberOfRecords).get(index); + Map expectedMap = inputMaps.get(index); assertThat(expectedMap, Matchers.equalTo(actualMap)); index++; } @@ -141,14 +145,16 @@ void test_happy_case_nullable_records(final int numberOfRecords) throws Exceptio parquetOutputCodec.writeEvent(event, outputStream); } parquetOutputCodec.complete(outputStream); - List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(tempFile.toString().getBytes())); + List> actualRecords = createParquetRecordsList(new FileInputStream(tempFile)); int index = 0; + assertThat(inputMaps.size(), equalTo(actualRecords.size())); for (final Map actualMap : actualRecords) { assertThat(actualMap, notNullValue()); - Map expectedMap = generateRecords(numberOfRecords).get(index); + Map expectedMap = inputMaps.get(index); assertThat(expectedMap, Matchers.equalTo(actualMap)); index++; } + outputStream.close(); tempFile.delete(); } @@ -167,11 +173,12 @@ void test_happy_case_nullable_records_with_empty_maps(final int numberOfRecords) parquetOutputCodec.writeEvent(event, outputStream); } parquetOutputCodec.complete(outputStream); - List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(tempFile.toString().getBytes())); + List> actualRecords = createParquetRecordsList(new FileInputStream(tempFile)); int index = 0; + assertThat(inputMaps.size(), equalTo(actualRecords.size())); for (final Map actualMap : actualRecords) { assertThat(actualMap, notNullValue()); - Map expectedMap = generateRecords(numberOfRecords).get(index); + Map expectedMap = inputMaps.get(index); assertThat(expectedMap, Matchers.equalTo(actualMap)); index++; } @@ -193,6 +200,9 @@ void writeEvent_includes_record_when_field_does_not_exist_in_user_supplied_schem final Event eventWithInvalidField = mock(Event.class); final String invalidFieldName = UUID.randomUUID().toString(); Map mapWithInvalid = generateRecords(1).get(0); + Map mapWithoutInvalid = mapWithInvalid.entrySet() + .stream() + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); mapWithInvalid.put(invalidFieldName, UUID.randomUUID().toString()); when(eventWithInvalidField.toMap()).thenReturn(mapWithInvalid); final ParquetOutputCodec objectUnderTest = createObjectUnderTest(); @@ -204,12 +214,12 @@ void writeEvent_includes_record_when_field_does_not_exist_in_user_supplied_schem objectUnderTest.writeEvent(eventWithInvalidField, outputStream); objectUnderTest.complete(outputStream); - List> actualRecords = createParquetRecordsList(new ByteArrayInputStream(tempFile.toString().getBytes())); + List> actualRecords = createParquetRecordsList(new FileInputStream(tempFile)); int index = 0; for (final Map actualMap : actualRecords) { assertThat(actualMap, notNullValue()); - Map expectedMap = generateRecords(1).get(index); - assertThat(expectedMap, Matchers.equalTo(actualMap)); + assertThat(mapWithInvalid, not(Matchers.equalTo(actualMap))); + assertThat(mapWithoutInvalid, Matchers.equalTo(actualMap)); index++; } } @@ -550,12 +560,34 @@ private static Schema createStandardInnerSchemaForNestedRecord( return assembler.endRecord(); } - private List> createParquetRecordsList(final InputStream inputStream) throws IOException { + private List extractStringList(SimpleGroup group, String fieldName) { + int fieldIndex = group.getType().getFieldIndex(fieldName); + int repetitionCount = group.getGroup(fieldIndex, 0).getFieldRepetitionCount(0); + List resultList = new ArrayList<>(); + for (int i = 0; i < repetitionCount; i++) { + resultList.add(group.getGroup(fieldIndex, 0).getString(0, i)); + } + return resultList; + } + + private Map extractNestedGroup(SimpleGroup group, String fieldName) { + + Map resultMap = new HashMap<>(); + int fieldIndex = group.getType().getFieldIndex(fieldName); + int f1 = group.getGroup(fieldIndex, 0).getType().getFieldIndex("firstFieldInNestedRecord"); + resultMap.put("firstFieldInNestedRecord", group.getGroup(fieldIndex, 0).getString(f1,0)); + int f2 = group.getGroup(fieldIndex, 0).getType().getFieldIndex("secondFieldInNestedRecord"); + resultMap.put("secondFieldInNestedRecord", group.getGroup(fieldIndex, 0).getInteger(f2,0)); + + return resultMap; + } + + private List> createParquetRecordsList(final InputStream inputStream) throws IOException, RuntimeException { final File tempFile = new File(tempDirectory, FILE_NAME); Files.copy(inputStream, tempFile.toPath(), StandardCopyOption.REPLACE_EXISTING); List> actualRecordList = new ArrayList<>(); - try (final ParquetFileReader parquetFileReader = new ParquetFileReader(new LocalInputFile(Path.of(tempFile.toURI())), ParquetReadOptions.builder().build())) { + try (ParquetFileReader parquetFileReader = new ParquetFileReader(HadoopInputFile.fromPath(new Path(tempFile.toURI()), new Configuration()), ParquetReadOptions.builder().build())) { final ParquetMetadata footer = parquetFileReader.getFooter(); final MessageType schema = createdParquetSchema(footer); PageReadStore pages; @@ -566,15 +598,34 @@ private List> createParquetRecordsList(final InputStream inp final RecordReader recordReader = columnIO.getRecordReader(pages, new GroupRecordConverter(schema)); for (int row = 0; row < rows; row++) { final Map eventData = new HashMap<>(); - int fieldIndex = 0; final SimpleGroup simpleGroup = (SimpleGroup) recordReader.read(); + final GroupType groupType = simpleGroup.getType(); + + for (Type field : schema.getFields()) { - try { - eventData.put(field.getName(), simpleGroup.getValueToString(fieldIndex, 0)); - } catch (Exception parquetException) { - LOG.error("Failed to parse Parquet", parquetException); + Object value; + int fieldIndex = groupType.getFieldIndex(field.getName()); + if (simpleGroup.getFieldRepetitionCount(fieldIndex) == 0) { + continue; + } + switch (field.getName()) { + case "name": value = simpleGroup.getString(fieldIndex, 0); + break; + case "age": value = simpleGroup.getInteger(fieldIndex, 0); + break; + case "myLong": value = simpleGroup.getLong(fieldIndex, 0); + break; + case "myFloat": value = simpleGroup.getFloat(fieldIndex, 0); + break; + case "myDouble": value = simpleGroup.getDouble(fieldIndex, 0); + break; + case "myArray": value = extractStringList(simpleGroup, "myArray"); + break; + case "nestedRecord": value = extractNestedGroup(simpleGroup, "nestedRecord"); + break; + default: throw new IllegalArgumentException("Unknown field"); } - fieldIndex++; + eventData.put(field.getName(), value); } actualRecordList.add((HashMap) eventData); } @@ -590,4 +641,4 @@ private List> createParquetRecordsList(final InputStream inp private MessageType createdParquetSchema(ParquetMetadata parquetMetadata) { return parquetMetadata.getFileMetaData().getSchema(); } -} \ No newline at end of file +} From 9183e44846aefcccd64fbb38f418e17d2c313a70 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Wed, 3 Jul 2024 09:51:34 -0500 Subject: [PATCH 050/159] Export to S3 in RDS source (#4664) * Trigger RDS export to S3 Signed-off-by: Hai Yan * Add unit tests Signed-off-by: Hai Yan * Remove unused imports Signed-off-by: Hai Yan * Address review comments Signed-off-by: Hai Yan * Address further comments Signed-off-by: Hai Yan * Make checkSnapshotStatus a runnable Signed-off-by: Hai Yan * Revert "Make checkSnapshotStatus a runnable" This reverts commit 5caed6ffb218d64180b10285c5c9115f21d6f3a2. Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- .../plugins/source/rds/RdsService.java | 10 +- .../plugins/source/rds/RdsSource.java | 5 +- .../rds/coordination/PartitionFactory.java | 35 ++++ .../partition/ExportPartition.java | 68 +++++++ .../coordination/partition/GlobalState.java | 52 +++++ .../partition/LeaderPartition.java | 55 +++++ .../state/ExportProgressState.java | 115 +++++++++++ .../state/LeaderProgressState.java | 25 +++ .../source/rds/export/ExportScheduler.java | 190 +++++++++++++++++- .../source/rds/export/ExportTaskManager.java | 79 ++++++++ .../source/rds/export/SnapshotManager.java | 66 ++++++ .../source/rds/leader/LeaderScheduler.java | 89 ++++++++ .../source/rds/model/ExportStatus.java | 36 ++++ .../source/rds/model/SnapshotInfo.java | 43 ++++ .../source/rds/model/SnapshotStatus.java | 22 ++ .../plugins/source/rds/RdsServiceTest.java | 1 - .../coordination/PartitionFactoryTest.java | 61 ++++++ .../rds/export/ExportSchedulerTest.java | 171 ++++++++++++++++ .../rds/export/ExportTaskManagerTest.java | 104 ++++++++++ .../rds/export/SnapshotManagerTest.java | 115 +++++++++++ .../rds/leader/LeaderSchedulerTest.java | 135 +++++++++++++ .../source/rds/model/ExportStatusTest.java | 49 +++++ 22 files changed, 1519 insertions(+), 7 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/ExportPartition.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/GlobalState.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/LeaderPartition.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/LeaderProgressState.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportTaskManager.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManager.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportStatus.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/SnapshotInfo.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/SnapshotStatus.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactoryTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportTaskManagerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportStatusTest.java diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index 0e8a92e31d..f059dd52bf 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -29,6 +29,8 @@ public class RdsService { private final PluginMetrics pluginMetrics; private final RdsSourceConfig sourceConfig; private ExecutorService executor; + private LeaderScheduler leaderScheduler; + private ExportScheduler exportScheduler; public RdsService(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig, @@ -51,8 +53,10 @@ public RdsService(final EnhancedSourceCoordinator sourceCoordinator, public void start(Buffer> buffer) { LOG.info("Start running RDS service"); final List runnableList = new ArrayList<>(); - runnableList.add(new LeaderScheduler(sourceCoordinator, sourceConfig)); - runnableList.add(new ExportScheduler(sourceCoordinator, rdsClient, pluginMetrics)); + leaderScheduler = new LeaderScheduler(sourceCoordinator, sourceConfig); + exportScheduler = new ExportScheduler(sourceCoordinator, rdsClient, pluginMetrics); + runnableList.add(leaderScheduler); + runnableList.add(exportScheduler); executor = Executors.newFixedThreadPool(runnableList.size()); runnableList.forEach(executor::submit); @@ -65,6 +69,8 @@ public void start(Buffer> buffer) { public void shutdown() { if (executor != null) { LOG.info("shutdown RDS schedulers"); + exportScheduler.shutdown(); + leaderScheduler.shutdown(); executor.shutdownNow(); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java index cc4bd23ca0..a9fe983572 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java @@ -17,6 +17,8 @@ import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; import org.opensearch.dataprepper.model.source.coordinator.enhanced.UsesEnhancedSourceCoordination; +import org.opensearch.dataprepper.plugins.source.rds.coordination.PartitionFactory; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,6 +49,7 @@ public RdsSource(final PluginMetrics pluginMetrics, @Override public void start(Buffer> buffer) { Objects.requireNonNull(sourceCoordinator); + sourceCoordinator.createPartition(new LeaderPartition()); rdsService = new RdsService(sourceCoordinator, sourceConfig, clientFactory, pluginMetrics); @@ -70,6 +73,6 @@ public void setEnhancedSourceCoordinator(EnhancedSourceCoordinator sourceCoordin @Override public Function getPartitionFactory() { - return null; + return new PartitionFactory(); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java new file mode 100644 index 0000000000..db35f5076b --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java @@ -0,0 +1,35 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; + +import java.util.function.Function; + +/** + * Partition factory to map a {@link SourcePartitionStoreItem} to a {@link EnhancedSourcePartition}. + */ +public class PartitionFactory implements Function { + + @Override + public EnhancedSourcePartition apply(SourcePartitionStoreItem partitionStoreItem) { + String sourceIdentifier = partitionStoreItem.getSourceIdentifier(); + String partitionType = sourceIdentifier.substring(sourceIdentifier.lastIndexOf('|') + 1); + + if (LeaderPartition.PARTITION_TYPE.equals(partitionType)) { + return new LeaderPartition(partitionStoreItem); + } if (ExportPartition.PARTITION_TYPE.equals(partitionType)) { + return new ExportPartition(partitionStoreItem); + } else { + // Unable to acquire other partitions. + return new GlobalState(partitionStoreItem); + } + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/ExportPartition.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/ExportPartition.java new file mode 100644 index 0000000000..5d79378dec --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/ExportPartition.java @@ -0,0 +1,68 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; + +import java.util.Optional; + +/** + * An ExportPartition represents an export job needs to be run for tables. + * Each export job has an export time associate with it. + * Each job maintains the state such as total files/records etc. independently. + * The source identifier contains keyword 'EXPORT' + */ +public class ExportPartition extends EnhancedSourcePartition { + public static final String PARTITION_TYPE = "EXPORT"; + + private static final String DB_CLUSTER = "cluster"; + private static final String DB_INSTANCE = "instance"; + + private final String dbIdentifier; + + private final boolean isCluster; + + private final ExportProgressState progressState; + + public ExportPartition(String dbIdentifier, boolean isCluster, ExportProgressState progressState) { + this.dbIdentifier = dbIdentifier; + this.isCluster = isCluster; + this.progressState = progressState; + } + + public ExportPartition(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + String [] keySplits = sourcePartitionStoreItem.getSourcePartitionKey().split("\\|"); + dbIdentifier = keySplits[0]; + isCluster = DB_CLUSTER.equals(keySplits[1]); + progressState = convertStringToPartitionProgressState(ExportProgressState.class, sourcePartitionStoreItem.getPartitionProgressState()); + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + final String dbType = isCluster ? DB_CLUSTER : DB_INSTANCE; + return dbIdentifier + "|" + dbType; + } + + @Override + public Optional getProgressState() { + if (progressState != null) { + return Optional.of(progressState); + } + return Optional.empty(); + } + + public String getDbIdentifier() { + return dbIdentifier; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/GlobalState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/GlobalState.java new file mode 100644 index 0000000000..c6f1d394a2 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/GlobalState.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; + +import java.util.Map; +import java.util.Optional; + +public class GlobalState extends EnhancedSourcePartition> { + + private final String stateName; + + private Map state; + + public GlobalState(String stateName, Map state) { + this.stateName = stateName; + this.state = state; + } + + public GlobalState(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + stateName = sourcePartitionStoreItem.getSourcePartitionKey(); + state = convertStringToPartitionProgressState(null, sourcePartitionStoreItem.getPartitionProgressState()); + } + + @Override + public String getPartitionType() { + return null; + } + + @Override + public String getPartitionKey() { + return stateName; + } + + @Override + public Optional> getProgressState() { + if (state != null) { + return Optional.of(state); + } + return Optional.empty(); + } + + public void setProgressState(Map state) { + this.state = state; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/LeaderPartition.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/LeaderPartition.java new file mode 100644 index 0000000000..806b199998 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/LeaderPartition.java @@ -0,0 +1,55 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.LeaderProgressState; + +import java.util.Optional; + +/** + *

A LeaderPartition is for some tasks that should be done in a single node only.

+ *

Hence whatever node owns the lease of this partition will be acted as a 'leader'.

+ *

In this DynamoDB source design, a leader node will be responsible for:

+ *
    + *
  • Initialization process (create EXPORT and STREAM partitions)
  • + *
  • Triggering RDS export task
  • + *
  • Reading stream data
  • + *
+ */ +public class LeaderPartition extends EnhancedSourcePartition { + public static final String PARTITION_TYPE = "LEADER"; + + // identifier for the partition + private static final String DEFAULT_PARTITION_KEY = "GLOBAL"; + + private final LeaderProgressState state; + + public LeaderPartition() { + this.state = new LeaderProgressState(); + } + + public LeaderPartition(SourcePartitionStoreItem partitionStoreItem) { + setSourcePartitionStoreItem(partitionStoreItem); + this.state = convertStringToPartitionProgressState(LeaderProgressState.class, partitionStoreItem.getPartitionProgressState()); + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return DEFAULT_PARTITION_KEY; + } + + @Override + public Optional getProgressState() { + return Optional.of(state); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java new file mode 100644 index 0000000000..cde2be6dd8 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java @@ -0,0 +1,115 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; + +/** + * Progress state for an EXPORT partition + */ +public class ExportProgressState { + + @JsonProperty("snapshotId") + private String snapshotId; + + @JsonProperty("exportTaskId") + private String exportTaskId; + + @JsonProperty("iamRoleArn") + private String iamRoleArn; + + @JsonProperty("bucket") + private String bucket; + + @JsonProperty("prefix") + private String prefix; + + @JsonProperty("tables") + private List tables; + + @JsonProperty("kmsKeyId") + private String kmsKeyId; + + @JsonProperty("exportTime") + private String exportTime; + + @JsonProperty("status") + private String status; + + public String getSnapshotId() { + return snapshotId; + } + + public void setSnapshotId(String snapshotId) { + this.snapshotId = snapshotId; + } + + public String getExportTaskId() { + return exportTaskId; + } + + public void setExportTaskId(String exportTaskId) { + this.exportTaskId = exportTaskId; + } + + public String getIamRoleArn() { + return iamRoleArn; + } + + public void setIamRoleArn(String iamRoleArn) { + this.iamRoleArn = iamRoleArn; + } + + public String getBucket() { + return bucket; + } + + public void setBucket(String bucket) { + this.bucket = bucket; + } + + public String getPrefix() { + return prefix; + } + + public void setPrefix(String prefix) { + this.prefix = prefix; + } + + public List getTables() { + return tables; + } + + public void setTables(List tables) { + this.tables = tables; + } + + public String getKmsKeyId() { + return kmsKeyId; + } + + public void setKmsKeyId(String kmsKeyId) { + this.kmsKeyId = kmsKeyId; + } + + public String getExportTime() { + return exportTime; + } + + public void setExportTime(String exportTime) { + this.exportTime = exportTime; + } + + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/LeaderProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/LeaderProgressState.java new file mode 100644 index 0000000000..216fb64fae --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/LeaderProgressState.java @@ -0,0 +1,25 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Progress state for a LEADER partition + */ +public class LeaderProgressState { + + @JsonProperty("initialized") + private boolean initialized = false; + + public boolean isInitialized() { + return initialized; + } + + public void setInitialized(boolean initialized) { + this.initialized = initialized; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index 9c83643c68..51db82248b 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -7,13 +7,23 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; +import org.opensearch.dataprepper.plugins.source.rds.model.ExportStatus; +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.rds.RdsClient; import java.time.Duration; +import java.time.Instant; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.function.BiConsumer; public class ExportScheduler implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(ExportScheduler.class); @@ -23,14 +33,16 @@ public class ExportScheduler implements Runnable { private static final int DEFAULT_MAX_CLOSE_COUNT = 36; private static final int DEFAULT_CHECKPOINT_INTERVAL_MILLS = 5 * 60_000; private static final int DEFAULT_CHECK_STATUS_INTERVAL_MILLS = 30 * 1000; + private static final Duration DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT = Duration.ofMinutes(60); private final RdsClient rdsClient; - private final PluginMetrics pluginMetrics; - private final EnhancedSourceCoordinator sourceCoordinator; - private final ExecutorService executor; + private final ExportTaskManager exportTaskManager; + private final SnapshotManager snapshotManager; + + private volatile boolean shutdownRequested = false; public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, final RdsClient rdsClient, @@ -39,10 +51,182 @@ public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, this.sourceCoordinator = sourceCoordinator; this.rdsClient = rdsClient; this.executor = Executors.newCachedThreadPool(); + this.exportTaskManager = new ExportTaskManager(rdsClient); + this.snapshotManager = new SnapshotManager(rdsClient); } @Override public void run() { + LOG.debug("Start running Export Scheduler"); + while (!shutdownRequested && !Thread.currentThread().isInterrupted()) { + try { + final Optional sourcePartition = sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE); + + if (sourcePartition.isPresent()) { + ExportPartition exportPartition = (ExportPartition) sourcePartition.get(); + LOG.debug("Acquired an export partition: {}", exportPartition.getPartitionKey()); + + String exportTaskId = getOrCreateExportTaskId(exportPartition); + + if (exportTaskId == null) { + LOG.error("The export to S3 failed, it will be retried"); + closeExportPartitionWithError(exportPartition); + } else { + CompletableFuture checkStatus = CompletableFuture.supplyAsync(() -> checkExportStatus(exportPartition), executor); + checkStatus.whenComplete(completeExport(exportPartition)); + } + } + + try { + Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException e) { + LOG.info("The ExportScheduler was interrupted while waiting to retry, stopping processing"); + break; + } + } catch (final Exception e) { + LOG.error("Received an exception during export, backing off and retrying", e); + try { + Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException ex) { + LOG.info("The ExportScheduler was interrupted while waiting to retry, stopping processing"); + break; + } + } + } + LOG.warn("Export scheduler interrupted, looks like shutdown has triggered"); + executor.shutdownNow(); + } + + public void shutdown() { + shutdownRequested = true; + } + + private String getOrCreateExportTaskId(ExportPartition exportPartition) { + ExportProgressState progressState = exportPartition.getProgressState().get(); + + if (progressState.getExportTaskId() != null) { + LOG.info("Export task has already created for db {}", exportPartition.getDbIdentifier()); + return progressState.getExportTaskId(); + } + + LOG.info("Creating a new snapshot for db {}", exportPartition.getDbIdentifier()); + SnapshotInfo snapshotInfo = snapshotManager.createSnapshot(exportPartition.getDbIdentifier()); + if (snapshotInfo != null) { + LOG.info("Snapshot id is {}", snapshotInfo.getSnapshotId()); + progressState.setSnapshotId(snapshotInfo.getSnapshotId()); + sourceCoordinator.saveProgressStateForPartition(exportPartition, null); + } else { + LOG.error("The snapshot failed to create, it will be retried"); + closeExportPartitionWithError(exportPartition); + return null; + } + + final String snapshotId = snapshotInfo.getSnapshotId(); + try { + checkSnapshotStatus(snapshotId, DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT); + } catch (Exception e) { + LOG.warn("Check snapshot status for {} failed", snapshotId, e); + sourceCoordinator.giveUpPartition(exportPartition); + return null; + } + + LOG.info("Creating an export task for db {} from snapshot {}", exportPartition.getDbIdentifier(), snapshotId); + String exportTaskId = exportTaskManager.startExportTask( + snapshotInfo.getSnapshotArn(), progressState.getIamRoleArn(), progressState.getBucket(), + progressState.getPrefix(), progressState.getKmsKeyId(), progressState.getTables()); + + if (exportTaskId != null) { + LOG.info("Export task id is {}", exportTaskId); + progressState.setExportTaskId(exportTaskId); + sourceCoordinator.saveProgressStateForPartition(exportPartition, null); + } else { + LOG.error("The export task failed to create, it will be retried"); + closeExportPartitionWithError(exportPartition); + return null; + } + + return exportTaskId; + } + + private void closeExportPartitionWithError(ExportPartition exportPartition) { + ExportProgressState exportProgressState = exportPartition.getProgressState().get(); + // Clear current task id, so that a new export can be submitted. + exportProgressState.setExportTaskId(null); + sourceCoordinator.closePartition(exportPartition, DEFAULT_CLOSE_DURATION, DEFAULT_MAX_CLOSE_COUNT); + } + + private String checkSnapshotStatus(String snapshotId, Duration timeout) { + final Instant endTime = Instant.now().plus(timeout); + + LOG.debug("Start checking status of snapshot {}", snapshotId); + while (Instant.now().isBefore(endTime)) { + SnapshotInfo snapshotInfo = snapshotManager.checkSnapshotStatus(snapshotId); + String status = snapshotInfo.getStatus(); + // Valid snapshot statuses are: available, copying, creating + // The status should never be "copying" here + if (SnapshotStatus.AVAILABLE.getStatusName().equals(status)) { + LOG.info("Snapshot {} is available.", snapshotId); + return status; + } + + LOG.debug("Snapshot {} is still creating. Wait and check later", snapshotId); + try { + Thread.sleep(DEFAULT_CHECK_STATUS_INTERVAL_MILLS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + throw new RuntimeException("Snapshot status check timed out."); + } + + private String checkExportStatus(ExportPartition exportPartition) { + long lastCheckpointTime = System.currentTimeMillis(); + String exportTaskId = exportPartition.getProgressState().get().getExportTaskId(); + + LOG.debug("Start checking the status of export {}", exportTaskId); + while (true) { + if (System.currentTimeMillis() - lastCheckpointTime > DEFAULT_CHECKPOINT_INTERVAL_MILLS) { + sourceCoordinator.saveProgressStateForPartition(exportPartition, null); + lastCheckpointTime = System.currentTimeMillis(); + } + + // Valid statuses are: CANCELED, CANCELING, COMPLETE, FAILED, IN_PROGRESS, STARTING + String status = exportTaskManager.checkExportStatus(exportTaskId); + LOG.debug("Current export status is {}.", status); + if (ExportStatus.isTerminal(status)) { + LOG.info("Export {} is completed with final status {}", exportTaskId, status); + return status; + } + LOG.debug("Export {} is still running in progress. Wait and check later", exportTaskId); + try { + Thread.sleep(DEFAULT_CHECK_STATUS_INTERVAL_MILLS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + } + + private BiConsumer completeExport(ExportPartition exportPartition) { + return (status, ex) -> { + if (ex != null) { + LOG.warn("Check export status for {} failed", exportPartition.getPartitionKey(), ex); + sourceCoordinator.giveUpPartition(exportPartition); + } else { + if (!ExportStatus.COMPLETE.name().equals(status)) { + LOG.error("Export failed with status {}", status); + closeExportPartitionWithError(exportPartition); + return; + } + LOG.info("Export for {} completed successfully", exportPartition.getPartitionKey()); + + completeExportPartition(exportPartition); + } + }; + } + private void completeExportPartition(ExportPartition exportPartition) { + ExportProgressState progressState = exportPartition.getProgressState().get(); + progressState.setStatus("Completed"); + sourceCoordinator.completePartition(exportPartition); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportTaskManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportTaskManager.java new file mode 100644 index 0000000000..dc447c2f42 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportTaskManager.java @@ -0,0 +1,79 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.DescribeExportTasksRequest; +import software.amazon.awssdk.services.rds.model.DescribeExportTasksResponse; +import software.amazon.awssdk.services.rds.model.StartExportTaskRequest; +import software.amazon.awssdk.services.rds.model.StartExportTaskResponse; + +import java.util.Collection; +import java.util.UUID; + +public class ExportTaskManager { + + private static final Logger LOG = LoggerFactory.getLogger(ExportTaskManager.class); + + // Export identifier cannot be longer than 60 characters + private static final int EXPORT_TASK_ID_MAX_LENGTH = 60; + + private final RdsClient rdsClient; + + public ExportTaskManager(final RdsClient rdsClient) { + this.rdsClient = rdsClient; + } + + public String startExportTask(String snapshotArn, String iamRoleArn, String bucket, String prefix, String kmsKeyId, Collection includeTables) { + final String exportTaskId = generateExportTaskId(snapshotArn); + StartExportTaskRequest.Builder requestBuilder = StartExportTaskRequest.builder() + .exportTaskIdentifier(exportTaskId) + .sourceArn(snapshotArn) + .iamRoleArn(iamRoleArn) + .s3BucketName(bucket) + .s3Prefix(prefix) + .kmsKeyId(kmsKeyId); + + if (includeTables != null && !includeTables.isEmpty()) { + requestBuilder.exportOnly(includeTables); + } + + try { + StartExportTaskResponse response = rdsClient.startExportTask(requestBuilder.build()); + LOG.info("Export task submitted with id {} and status {}", exportTaskId, response.status()); + return exportTaskId; + + } catch (Exception e) { + LOG.error("Failed to start an export task", e); + return null; + } + } + + public String checkExportStatus(String exportTaskId) { + DescribeExportTasksRequest request = DescribeExportTasksRequest.builder() + .exportTaskIdentifier(exportTaskId) + .build(); + + DescribeExportTasksResponse response = rdsClient.describeExportTasks(request); + + return response.exportTasks().get(0).status(); + } + + private String generateExportTaskId(String snapshotArn) { + String snapshotId = Arn.fromString(snapshotArn).resource().resource(); + return truncateString(snapshotId, EXPORT_TASK_ID_MAX_LENGTH - 16) + "-export-" + UUID.randomUUID().toString().substring(0, 8); + } + + private String truncateString(String originalString, int maxLength) { + if (originalString.length() <= maxLength) { + return originalString; + } + return originalString.substring(0, maxLength); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManager.java new file mode 100644 index 0000000000..7b8da8717c --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManager.java @@ -0,0 +1,66 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; + +import java.time.Instant; +import java.util.UUID; + +public class SnapshotManager { + private static final Logger LOG = LoggerFactory.getLogger(SnapshotManager.class); + + private final RdsClient rdsClient; + + public SnapshotManager(final RdsClient rdsClient) { + this.rdsClient = rdsClient; + } + + public SnapshotInfo createSnapshot(String dbInstanceId) { + final String snapshotId = generateSnapshotId(dbInstanceId); + CreateDbSnapshotRequest request = CreateDbSnapshotRequest.builder() + .dbInstanceIdentifier(dbInstanceId) + .dbSnapshotIdentifier(snapshotId) + .build(); + + try { + CreateDbSnapshotResponse response = rdsClient.createDBSnapshot(request); + String snapshotArn = response.dbSnapshot().dbSnapshotArn(); + String status = response.dbSnapshot().status(); + Instant createTime = response.dbSnapshot().snapshotCreateTime(); + LOG.info("Creating snapshot with id {} and status {}", snapshotId, status); + + return new SnapshotInfo(snapshotId, snapshotArn, createTime, status); + } catch (Exception e) { + LOG.error("Failed to create snapshot for {}", dbInstanceId, e); + return null; + } + } + + public SnapshotInfo checkSnapshotStatus(String snapshotId) { + DescribeDbSnapshotsRequest request = DescribeDbSnapshotsRequest.builder() + .dbSnapshotIdentifier(snapshotId) + .build(); + + DescribeDbSnapshotsResponse response = rdsClient.describeDBSnapshots(request); + String snapshotArn = response.dbSnapshots().get(0).dbSnapshotArn(); + String status = response.dbSnapshots().get(0).status(); + Instant createTime = response.dbSnapshots().get(0).snapshotCreateTime(); + + return new SnapshotInfo(snapshotId, snapshotArn, createTime, status); + } + + private String generateSnapshotId(String dbClusterId) { + return dbClusterId + "-snapshot-" + UUID.randomUUID().toString().substring(0, 8); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index ca99a7c8f1..4831f1e91a 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -6,11 +6,19 @@ package org.opensearch.dataprepper.plugins.source.rds.leader; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.LeaderProgressState; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.time.Duration; +import java.time.Instant; +import java.util.Optional; public class LeaderScheduler implements Runnable { @@ -20,6 +28,9 @@ public class LeaderScheduler implements Runnable { private final EnhancedSourceCoordinator sourceCoordinator; private final RdsSourceConfig sourceConfig; + private LeaderPartition leaderPartition; + private volatile boolean shutdownRequested = false; + public LeaderScheduler(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig) { this.sourceCoordinator = sourceCoordinator; this.sourceConfig = sourceConfig; @@ -27,6 +38,84 @@ public LeaderScheduler(final EnhancedSourceCoordinator sourceCoordinator, final @Override public void run() { + LOG.info("Starting Leader Scheduler for initialization."); + + while (!shutdownRequested && !Thread.currentThread().isInterrupted()) { + try { + // Try to acquire the lease if not owned + if (leaderPartition == null) { + final Optional sourcePartition = sourceCoordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE); + if (sourcePartition.isPresent()) { + LOG.info("Running as a LEADER node."); + leaderPartition = (LeaderPartition) sourcePartition.get(); + } + } + + // Once owned, run Normal LEADER node process + if (leaderPartition != null) { + LeaderProgressState leaderProgressState = leaderPartition.getProgressState().get(); + if (!leaderProgressState.isInitialized()) { + init(); + } + } + } catch (final Exception e) { + LOG.error("Exception occurred in primary leader scheduling loop", e); + } finally { + if (leaderPartition != null) { + // Extend the timeout + // will always be a leader until shutdown + sourceCoordinator.saveProgressStateForPartition(leaderPartition, Duration.ofMinutes(DEFAULT_EXTEND_LEASE_MINUTES)); + } + + try { + Thread.sleep(DEFAULT_LEASE_INTERVAL.toMillis()); + } catch (final InterruptedException e) { + LOG.info("InterruptedException occurred while waiting in leader scheduling loop."); + break; + } + } + } + + // Should stop + LOG.warn("Quitting Leader Scheduler"); + if (leaderPartition != null) { + sourceCoordinator.giveUpPartition(leaderPartition); + } + } + + public void shutdown() { + shutdownRequested = true; + } + + private void init() { + LOG.info("Initializing RDS source service..."); + + // Create a Global state in the coordination table for the configuration. + // Global State here is designed to be able to read whenever needed + // So that the jobs can refer to the configuration. + sourceCoordinator.createPartition(new GlobalState(sourceConfig.getDbIdentifier(), null)); + + if (sourceConfig.isExportEnabled()) { + Instant startTime = Instant.now(); + LOG.debug("Export is enabled. Creating export partition in the source coordination store."); + createExportPartition(sourceConfig, startTime); + } + LOG.debug("Update initialization state"); + LeaderProgressState leaderProgressState = leaderPartition.getProgressState().get(); + leaderProgressState.setInitialized(true); } + + private void createExportPartition(RdsSourceConfig sourceConfig, Instant exportTime) { + ExportProgressState progressState = new ExportProgressState(); + progressState.setIamRoleArn(sourceConfig.getAwsAuthenticationConfig().getAwsStsRoleArn()); + progressState.setBucket(sourceConfig.getS3Bucket()); + progressState.setPrefix(sourceConfig.getS3Prefix()); + progressState.setTables(sourceConfig.getTableNames()); + progressState.setKmsKeyId(sourceConfig.getExport().getKmsKeyId()); + progressState.setExportTime(exportTime.toString()); + ExportPartition exportPartition = new ExportPartition(sourceConfig.getDbIdentifier(), sourceConfig.isCluster(), progressState); + sourceCoordinator.createPartition(exportPartition); + } + } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportStatus.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportStatus.java new file mode 100644 index 0000000000..16fb91b7f4 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportStatus.java @@ -0,0 +1,36 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +import java.util.Arrays; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public enum ExportStatus { + CANCELED, + CANCELING, + COMPLETE, + FAILED, + IN_PROGRESS, + STARTING; + + private static final Map TYPES_MAP = Arrays.stream(ExportStatus.values()) + .collect(Collectors.toMap( + Enum::name, + value -> value + )); + private static final Set TERMINAL_STATUSES = Set.of(CANCELED, COMPLETE, FAILED); + + public static ExportStatus fromString(final String name) { + return TYPES_MAP.get(name); + } + + public static boolean isTerminal(final String name) { + ExportStatus status = fromString(name); + return status != null && TERMINAL_STATUSES.contains(status); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/SnapshotInfo.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/SnapshotInfo.java new file mode 100644 index 0000000000..11bd452497 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/SnapshotInfo.java @@ -0,0 +1,43 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +import java.time.Instant; + +public class SnapshotInfo { + + private final String snapshotId; + private final String snapshotArn; + private final Instant createTime; + private String status; + + public SnapshotInfo(String snapshotId, String snapshotArn, Instant createTime, String status) { + this.snapshotId = snapshotId; + this.snapshotArn = snapshotArn; + this.createTime = createTime; + this.status = status; + } + + public String getSnapshotId() { + return snapshotId; + } + + public String getSnapshotArn() { + return snapshotArn; + } + + public Instant getCreateTime() { + return createTime; + } + + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/SnapshotStatus.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/SnapshotStatus.java new file mode 100644 index 0000000000..a2d18f70f9 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/SnapshotStatus.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +public enum SnapshotStatus { + AVAILABLE("available"), + COPYING("copying"), + CREATING("creating"); + + private final String statusName; + + SnapshotStatus(final String statusName) { + this.statusName = statusName; + } + + public String getStatusName() { + return statusName; + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java index 218c23d121..6aaa0b0bd5 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java @@ -56,7 +56,6 @@ class RdsServiceTest { @BeforeEach void setUp() { when(clientFactory.buildRdsClient()).thenReturn(rdsClient); - } @Test diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactoryTest.java new file mode 100644 index 0000000000..c092a8b48c --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactoryTest.java @@ -0,0 +1,61 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; + +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class PartitionFactoryTest { + + @Mock + private SourcePartitionStoreItem partitionStoreItem; + + @Test + void given_leader_partition_item_then_create_leader_partition() { + PartitionFactory objectUnderTest = createObjectUnderTest(); + when(partitionStoreItem.getSourceIdentifier()).thenReturn(UUID.randomUUID() + "|" + LeaderPartition.PARTITION_TYPE); + when(partitionStoreItem.getPartitionProgressState()).thenReturn(null); + + assertThat(objectUnderTest.apply(partitionStoreItem), instanceOf(LeaderPartition.class)); + } + + @Test + void given_export_partition_item_then_create_export_partition() { + PartitionFactory objectUnderTest = createObjectUnderTest(); + when(partitionStoreItem.getSourceIdentifier()).thenReturn(UUID.randomUUID() + "|" + ExportPartition.PARTITION_TYPE); + when(partitionStoreItem.getSourcePartitionKey()).thenReturn(UUID.randomUUID() + "|" + UUID.randomUUID()); + when(partitionStoreItem.getPartitionProgressState()).thenReturn(null); + + assertThat(objectUnderTest.apply(partitionStoreItem), instanceOf(ExportPartition.class)); + } + + @Test + void given_store_item_of_undefined_type_then_create_global_state() { + PartitionFactory objectUnderTest = createObjectUnderTest(); + when(partitionStoreItem.getSourceIdentifier()).thenReturn(UUID.randomUUID() + "|" + UUID.randomUUID()); + when(partitionStoreItem.getSourcePartitionKey()).thenReturn(UUID.randomUUID().toString()); + when(partitionStoreItem.getPartitionProgressState()).thenReturn(null); + + assertThat(objectUnderTest.apply(partitionStoreItem), instanceOf(GlobalState.class)); + } + + private PartitionFactory createObjectUnderTest() { + return new PartitionFactory(); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java new file mode 100644 index 0000000000..d0560ab30d --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java @@ -0,0 +1,171 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DBSnapshot; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; +import software.amazon.awssdk.services.rds.model.DescribeExportTasksRequest; +import software.amazon.awssdk.services.rds.model.DescribeExportTasksResponse; +import software.amazon.awssdk.services.rds.model.StartExportTaskRequest; +import software.amazon.awssdk.services.rds.model.StartExportTaskResponse; + +import java.time.Duration; +import java.time.Instant; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + + +@ExtendWith(MockitoExtension.class) +class ExportSchedulerTest { + + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + + @Mock + private RdsClient rdsClient; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private ExportPartition exportPartition; + + @Mock(answer = Answers.RETURNS_DEFAULTS) + private ExportProgressState exportProgressState; + + private ExportScheduler exportScheduler; + + @BeforeEach + void setUp() { + exportScheduler = createObjectUnderTest(); + } + + @Test + void test_given_no_export_partition_then_not_export() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.empty()); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(exportScheduler); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(ExportPartition.PARTITION_TYPE)); + Thread.sleep(100); + executorService.shutdownNow(); + + verifyNoInteractions(rdsClient); + } + + @Test + void test_given_export_partition_and_task_id_then_complete_export() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.of(exportPartition)); + when(exportPartition.getPartitionKey()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getExportTaskId()).thenReturn(UUID.randomUUID().toString()); + when(exportPartition.getProgressState()).thenReturn(Optional.of(exportProgressState)); + + DescribeExportTasksResponse describeExportTasksResponse = mock(DescribeExportTasksResponse.class, Mockito.RETURNS_DEEP_STUBS); + when(describeExportTasksResponse.exportTasks().get(0).status()).thenReturn("COMPLETE"); + when(rdsClient.describeExportTasks(any(DescribeExportTasksRequest.class))).thenReturn(describeExportTasksResponse); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(exportScheduler); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(ExportPartition.PARTITION_TYPE)); + Thread.sleep(100); + executorService.shutdownNow(); + + verify(sourceCoordinator).completePartition(exportPartition); + verify(rdsClient, never()).startExportTask(any(StartExportTaskRequest.class)); + verify(rdsClient, never()).createDBSnapshot(any(CreateDbSnapshotRequest.class)); + } + + + @Test + void test_given_export_partition_and_no_task_id_then_start_and_complete_export() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.of(exportPartition)); + when(exportPartition.getPartitionKey()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getExportTaskId()).thenReturn(null).thenReturn(UUID.randomUUID().toString()); + when(exportPartition.getProgressState()).thenReturn(Optional.of(exportProgressState)); + final String dbIdentifier = UUID.randomUUID().toString(); + when(exportPartition.getDbIdentifier()).thenReturn(dbIdentifier); + + // Mock snapshot response + CreateDbSnapshotResponse createDbSnapshotResponse = mock(CreateDbSnapshotResponse.class); + DBSnapshot dbSnapshot = mock(DBSnapshot.class); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:snapshot-0b5ae174"; + when(dbSnapshot.dbSnapshotArn()).thenReturn(snapshotArn); + when(dbSnapshot.status()).thenReturn("creating").thenReturn("available"); + when(dbSnapshot.snapshotCreateTime()).thenReturn(Instant.now()); + when(createDbSnapshotResponse.dbSnapshot()).thenReturn(dbSnapshot); + when(rdsClient.createDBSnapshot(any(CreateDbSnapshotRequest.class))).thenReturn(createDbSnapshotResponse); + + DescribeDbSnapshotsResponse describeDbSnapshotsResponse = DescribeDbSnapshotsResponse.builder() + .dbSnapshots(dbSnapshot) + .build(); + when(rdsClient.describeDBSnapshots(any(DescribeDbSnapshotsRequest.class))).thenReturn(describeDbSnapshotsResponse); + + // Mock export response + StartExportTaskResponse startExportTaskResponse = mock(StartExportTaskResponse.class); + when(startExportTaskResponse.status()).thenReturn("STARTING"); + when(rdsClient.startExportTask(any(StartExportTaskRequest.class))).thenReturn(startExportTaskResponse); + + DescribeExportTasksResponse describeExportTasksResponse = mock(DescribeExportTasksResponse.class, Mockito.RETURNS_DEEP_STUBS); + when(describeExportTasksResponse.exportTasks().get(0).status()).thenReturn("COMPLETE"); + when(rdsClient.describeExportTasks(any(DescribeExportTasksRequest.class))).thenReturn(describeExportTasksResponse); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(exportScheduler); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(ExportPartition.PARTITION_TYPE)); + Thread.sleep(200); + executorService.shutdownNow(); + + verify(rdsClient).createDBSnapshot(any(CreateDbSnapshotRequest.class)); + verify(rdsClient).startExportTask(any(StartExportTaskRequest.class)); + verify(sourceCoordinator).completePartition(exportPartition); + } + + @Test + void test_shutDown() { + lenient().when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.empty()); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(exportScheduler); + exportScheduler.shutdown(); + verifyNoMoreInteractions(sourceCoordinator, rdsClient); + executorService.shutdownNow(); + } + + private ExportScheduler createObjectUnderTest() { + return new ExportScheduler(sourceCoordinator, rdsClient, pluginMetrics); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportTaskManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportTaskManagerTest.java new file mode 100644 index 0000000000..15a23277c7 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportTaskManagerTest.java @@ -0,0 +1,104 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.DescribeExportTasksRequest; +import software.amazon.awssdk.services.rds.model.DescribeExportTasksResponse; +import software.amazon.awssdk.services.rds.model.ExportTask; +import software.amazon.awssdk.services.rds.model.StartExportTaskRequest; + +import java.util.List; +import java.util.UUID; +import java.util.stream.Stream; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + + +@ExtendWith(MockitoExtension.class) +class ExportTaskManagerTest { + + @Mock + private RdsClient rdsClient; + + private ExportTaskManager exportTaskManager; + + @BeforeEach + void setUp() { + exportTaskManager = createObjectUnderTest(); + } + + @ParameterizedTest + @MethodSource("provideStartExportTaskTestParameters") + void test_start_export_task(List exportOnly) { + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:" + UUID.randomUUID(); + final String iamRoleArn = "arn:aws:iam:us-east-1:123456789012:role:" + UUID.randomUUID(); + final String bucket = UUID.randomUUID().toString(); + final String prefix = UUID.randomUUID().toString(); + final String kmsKey = UUID.randomUUID().toString(); + + exportTaskManager.startExportTask(snapshotArn, iamRoleArn, bucket, prefix, kmsKey, exportOnly); + + final ArgumentCaptor exportTaskRequestArgumentCaptor = + ArgumentCaptor.forClass(StartExportTaskRequest.class); + + verify(rdsClient).startExportTask(exportTaskRequestArgumentCaptor.capture()); + + final StartExportTaskRequest actualRequest = exportTaskRequestArgumentCaptor.getValue(); + assertThat(actualRequest.sourceArn(), equalTo(snapshotArn)); + assertThat(actualRequest.iamRoleArn(), equalTo(iamRoleArn)); + assertThat(actualRequest.s3BucketName(), equalTo(bucket)); + assertThat(actualRequest.s3Prefix(), equalTo(prefix)); + assertThat(actualRequest.kmsKeyId(), equalTo(kmsKey)); + assertThat(actualRequest.exportOnly(), equalTo(exportOnly)); + } + + @Test + void test_check_export_status() { + final String exportTaskId = UUID.randomUUID().toString(); + DescribeExportTasksResponse describeExportTasksResponse = mock(DescribeExportTasksResponse.class); + when(describeExportTasksResponse.exportTasks()).thenReturn(List.of(ExportTask.builder().status("COMPLETE").build())); + when(rdsClient.describeExportTasks(any(DescribeExportTasksRequest.class))).thenReturn(describeExportTasksResponse); + + exportTaskManager.checkExportStatus(exportTaskId); + + final ArgumentCaptor exportTaskRequestArgumentCaptor = + ArgumentCaptor.forClass(DescribeExportTasksRequest.class); + + verify(rdsClient).describeExportTasks(exportTaskRequestArgumentCaptor.capture()); + + final DescribeExportTasksRequest actualRequest = exportTaskRequestArgumentCaptor.getValue(); + assertThat(actualRequest.exportTaskIdentifier(), equalTo(exportTaskId)); + } + + private static Stream provideStartExportTaskTestParameters() { + final String tableName1 = UUID.randomUUID().toString(); + final String tableName2 = UUID.randomUUID().toString(); + return Stream.of( + Arguments.of(List.of()), + Arguments.of(List.of(tableName1)), + Arguments.of(List.of(tableName1, tableName2)) + ); + } + + private ExportTaskManager createObjectUnderTest() { + return new ExportTaskManager(rdsClient); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java new file mode 100644 index 0000000000..bca52a5fdd --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java @@ -0,0 +1,115 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DBSnapshot; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; + +import java.time.Instant; +import java.util.List; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class SnapshotManagerTest { + + @Mock + private RdsClient rdsClient; + + private SnapshotManager snapshotManager; + + @BeforeEach + void setUp() { + snapshotManager = createObjectUnderTest(); + } + + @Test + void test_create_snapshot_with_success() { + String dbInstanceId = UUID.randomUUID().toString(); + CreateDbSnapshotResponse createDbSnapshotResponse = mock(CreateDbSnapshotResponse.class); + DBSnapshot dbSnapshot = mock(DBSnapshot.class); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:snapshot-0b5ae174"; + final String status = "creating"; + final Instant createTime = Instant.now(); + when(dbSnapshot.dbSnapshotArn()).thenReturn(snapshotArn); + when(dbSnapshot.status()).thenReturn(status); + when(dbSnapshot.snapshotCreateTime()).thenReturn(createTime); + when(createDbSnapshotResponse.dbSnapshot()).thenReturn(dbSnapshot); + when(rdsClient.createDBSnapshot(any(CreateDbSnapshotRequest.class))).thenReturn(createDbSnapshotResponse); + + SnapshotInfo snapshotInfo = snapshotManager.createSnapshot(dbInstanceId); + + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(CreateDbSnapshotRequest.class); + verify(rdsClient).createDBSnapshot(argumentCaptor.capture()); + + CreateDbSnapshotRequest request = argumentCaptor.getValue(); + assertThat(request.dbInstanceIdentifier(), equalTo(dbInstanceId)); + + assertThat(snapshotInfo, notNullValue()); + assertThat(snapshotInfo.getSnapshotArn(), equalTo(snapshotArn)); + assertThat(snapshotInfo.getStatus(), equalTo(status)); + assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); + } + + @Test + void test_create_snapshot_throws_exception_then_returns_null() { + String dbInstanceId = UUID.randomUUID().toString(); + when(rdsClient.createDBSnapshot(any(CreateDbSnapshotRequest.class))).thenThrow(new RuntimeException("Error")); + + SnapshotInfo snapshotInfo = snapshotManager.createSnapshot(dbInstanceId); + + assertThat(snapshotInfo, equalTo(null)); + } + + @Test + void test_check_snapshot_status_returns_correct_result() { + DBSnapshot dbSnapshot = mock(DBSnapshot.class); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:snapshot-0b5ae174"; + final String status = "creating"; + final Instant createTime = Instant.now(); + when(dbSnapshot.dbSnapshotArn()).thenReturn(snapshotArn); + when(dbSnapshot.status()).thenReturn(status); + when(dbSnapshot.snapshotCreateTime()).thenReturn(createTime); + DescribeDbSnapshotsResponse describeDbSnapshotsResponse = mock(DescribeDbSnapshotsResponse.class); + when(describeDbSnapshotsResponse.dbSnapshots()).thenReturn(List.of(dbSnapshot)); + + final String snapshotId = UUID.randomUUID().toString(); + DescribeDbSnapshotsRequest describeDbSnapshotsRequest = DescribeDbSnapshotsRequest.builder() + .dbSnapshotIdentifier(snapshotId) + .build(); + when(rdsClient.describeDBSnapshots(describeDbSnapshotsRequest)).thenReturn(describeDbSnapshotsResponse); + + SnapshotInfo snapshotInfo = snapshotManager.checkSnapshotStatus(snapshotId); + + assertThat(snapshotInfo, notNullValue()); + assertThat(snapshotInfo.getSnapshotId(), equalTo(snapshotId)); + assertThat(snapshotInfo.getSnapshotArn(), equalTo(snapshotArn)); + assertThat(snapshotInfo.getStatus(), equalTo(status)); + assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); + } + + private SnapshotManager createObjectUnderTest() { + return new SnapshotManager(rdsClient); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java new file mode 100644 index 0000000000..e844cc0ff4 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java @@ -0,0 +1,135 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.leader; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.ExportConfig; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.LeaderProgressState; + +import java.time.Duration; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + + +@ExtendWith(MockitoExtension.class) +class LeaderSchedulerTest { + + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + + @Mock(answer = Answers.RETURNS_DEFAULTS) + private RdsSourceConfig sourceConfig; + + @Mock + private LeaderPartition leaderPartition; + + @Mock + private LeaderProgressState leaderProgressState; + + private LeaderScheduler leaderScheduler; + + @BeforeEach + void setUp() { + leaderScheduler = createObjectUnderTest(); + + AwsAuthenticationConfig awsAuthenticationConfig = mock(AwsAuthenticationConfig.class); + lenient().when(awsAuthenticationConfig.getAwsStsRoleArn()).thenReturn(UUID.randomUUID().toString()); + lenient().when(sourceConfig.getAwsAuthenticationConfig()).thenReturn(awsAuthenticationConfig); + ExportConfig exportConfig = mock(ExportConfig.class); + lenient().when(exportConfig.getKmsKeyId()).thenReturn(UUID.randomUUID().toString()); + lenient().when(sourceConfig.getExport()).thenReturn(exportConfig); + } + + @Test + void non_leader_node_should_not_perform_init() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)).thenReturn(Optional.empty()); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(leaderScheduler); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)); + Thread.sleep(100); + executorService.shutdownNow(); + + verify(sourceCoordinator, never()).createPartition(any(GlobalState.class)); + verify(sourceCoordinator, never()).createPartition(any(ExportPartition.class)); + } + + @Test + void leader_node_should_perform_init_if_not_initialized() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)).thenReturn(Optional.of(leaderPartition)); + when(leaderPartition.getProgressState()).thenReturn(Optional.of(leaderProgressState)); + when(leaderProgressState.isInitialized()).thenReturn(false); + when(sourceConfig.isExportEnabled()).thenReturn(true); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(leaderScheduler); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)); + Thread.sleep(100); + executorService.shutdownNow(); + + verify(sourceCoordinator).createPartition(any(GlobalState.class)); + verify(sourceCoordinator).createPartition(any(ExportPartition.class)); + verify(sourceCoordinator).saveProgressStateForPartition(eq(leaderPartition), any(Duration.class)); + } + + @Test + void leader_node_should_skip_init_if_initialized() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)).thenReturn(Optional.of(leaderPartition)); + when(leaderPartition.getProgressState()).thenReturn(Optional.of(leaderProgressState)); + when(leaderProgressState.isInitialized()).thenReturn(true); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(leaderScheduler); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)); + Thread.sleep(100); + executorService.shutdownNow(); + + verify(sourceCoordinator, never()).createPartition(any(GlobalState.class)); + verify(sourceCoordinator, never()).createPartition(any(ExportPartition.class)); + verify(sourceCoordinator).saveProgressStateForPartition(eq(leaderPartition), any(Duration.class)); + } + + @Test + void test_shutDown() { + lenient().when(sourceCoordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)).thenReturn(Optional.empty()); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(leaderScheduler); + leaderScheduler.shutdown(); + verifyNoMoreInteractions(sourceCoordinator); + executorService.shutdownNow(); + } + + private LeaderScheduler createObjectUnderTest() { + return new LeaderScheduler(sourceCoordinator, sourceConfig); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportStatusTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportStatusTest.java new file mode 100644 index 0000000000..16ef0c0a1b --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportStatusTest.java @@ -0,0 +1,49 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +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.MatcherAssert.assertThat; + +class ExportStatusTest { + + @ParameterizedTest + @EnumSource(ExportStatus.class) + void fromString_returns_expected_value(final ExportStatus status) { + assertThat(ExportStatus.fromString(status.name()), equalTo(status)); + } + + @ParameterizedTest + @ArgumentsSource(ProvideTerminalStatusTestData.class) + void test_is_terminal_returns_expected_result(final String status, final boolean expected_result) { + assertThat(ExportStatus.isTerminal(status), equalTo(expected_result)); + } + + static class ProvideTerminalStatusTestData implements ArgumentsProvider { + @Override + public Stream provideArguments(ExtensionContext context) { + return Stream.of( + Arguments.of("COMPLETE", true), + Arguments.of("CANCELED", true), + Arguments.of("FAILED", true), + Arguments.of("CANCELING", false), + Arguments.of("IN_PROGRESS", false), + Arguments.of("STARTING", false), + Arguments.of("INVALID_STATUS", false), + Arguments.of(null, false) + ); + } + } +} From 31add2564a2f2fd658fc0d277415402f847e2d88 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 3 Jul 2024 11:41:10 -0500 Subject: [PATCH 051/159] Updates Parquet to 1.14.1 and Hadoop to 3.4.0. Make use of Gradle's version catalogue for Parquet. (#4705) Signed-off-by: David Venable --- data-prepper-api/build.gradle | 2 +- data-prepper-plugins/avro-codecs/build.gradle | 2 +- data-prepper-plugins/common/build.gradle | 2 +- data-prepper-plugins/csv-processor/build.gradle | 2 +- data-prepper-plugins/event-json-codecs/build.gradle | 2 +- data-prepper-plugins/newline-codecs/build.gradle | 2 +- data-prepper-plugins/parquet-codecs/build.gradle | 8 ++++---- .../parse-json-processor/build.gradle | 2 +- data-prepper-plugins/s3-sink/build.gradle | 2 +- data-prepper-plugins/s3-source/build.gradle | 11 +++++------ settings.gradle | 9 ++++++--- 11 files changed, 23 insertions(+), 21 deletions(-) diff --git a/data-prepper-api/build.gradle b/data-prepper-api/build.gradle index 0ad43ff470..045d331704 100644 --- a/data-prepper-api/build.gradle +++ b/data-prepper-api/build.gradle @@ -12,7 +12,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jdk8' - implementation 'org.apache.parquet:parquet-common:1.14.0' + implementation libs.parquet.common testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation libs.commons.lang3 testImplementation project(':data-prepper-test-common') diff --git a/data-prepper-plugins/avro-codecs/build.gradle b/data-prepper-plugins/avro-codecs/build.gradle index e6c5ea5e54..2bce28bbe0 100644 --- a/data-prepper-plugins/avro-codecs/build.gradle +++ b/data-prepper-plugins/avro-codecs/build.gradle @@ -6,7 +6,7 @@ dependencies { implementation project(path: ':data-prepper-api') implementation libs.avro.core - implementation 'org.apache.parquet:parquet-common:1.14.0' + implementation libs.parquet.common implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:apache-client' testImplementation 'org.json:json:20240205' diff --git a/data-prepper-plugins/common/build.gradle b/data-prepper-plugins/common/build.gradle index 947d1234d4..aec7d7bddc 100644 --- a/data-prepper-plugins/common/build.gradle +++ b/data-prepper-plugins/common/build.gradle @@ -19,7 +19,7 @@ dependencies { implementation libs.bouncycastle.bcpkix implementation libs.reflections.core implementation 'io.micrometer:micrometer-core' - implementation 'org.apache.parquet:parquet-common:1.14.0' + implementation libs.parquet.common implementation 'org.xerial.snappy:snappy-java:1.1.10.5' testImplementation project(':data-prepper-plugins:blocking-buffer') testImplementation project(':data-prepper-test-event') diff --git a/data-prepper-plugins/csv-processor/build.gradle b/data-prepper-plugins/csv-processor/build.gradle index 56c02daf83..cda0694a66 100644 --- a/data-prepper-plugins/csv-processor/build.gradle +++ b/data-prepper-plugins/csv-processor/build.gradle @@ -12,7 +12,7 @@ dependencies { implementation project(':data-prepper-api') implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-csv' implementation 'io.micrometer:micrometer-core' - implementation 'org.apache.parquet:parquet-common:1.14.0' + implementation libs.parquet.common implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:apache-client' testImplementation project(':data-prepper-plugins:log-generator-source') diff --git a/data-prepper-plugins/event-json-codecs/build.gradle b/data-prepper-plugins/event-json-codecs/build.gradle index aad563d19d..2278bf6033 100644 --- a/data-prepper-plugins/event-json-codecs/build.gradle +++ b/data-prepper-plugins/event-json-codecs/build.gradle @@ -15,7 +15,7 @@ dependencies { implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-xml' implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.17.0' testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310:2.17.0' - implementation 'org.apache.parquet:parquet-common:1.14.0' + implementation libs.parquet.common testImplementation project(':data-prepper-test-common') } diff --git a/data-prepper-plugins/newline-codecs/build.gradle b/data-prepper-plugins/newline-codecs/build.gradle index b504ed30ee..c71e8755ef 100644 --- a/data-prepper-plugins/newline-codecs/build.gradle +++ b/data-prepper-plugins/newline-codecs/build.gradle @@ -5,7 +5,7 @@ plugins { dependencies { implementation project(':data-prepper-api') implementation 'com.fasterxml.jackson.core:jackson-annotations' - implementation 'org.apache.parquet:parquet-common:1.14.0' + implementation libs.parquet.common testImplementation project(':data-prepper-plugins:common') testImplementation project(':data-prepper-test-event') } diff --git a/data-prepper-plugins/parquet-codecs/build.gradle b/data-prepper-plugins/parquet-codecs/build.gradle index dd59e28068..fbc8f4a209 100644 --- a/data-prepper-plugins/parquet-codecs/build.gradle +++ b/data-prepper-plugins/parquet-codecs/build.gradle @@ -8,10 +8,10 @@ dependencies { implementation project(':data-prepper-plugins:common') implementation libs.avro.core implementation 'org.apache.commons:commons-text:1.11.0' - implementation 'org.apache.parquet:parquet-avro:1.14.0' - implementation 'org.apache.parquet:parquet-column:1.14.0' - implementation 'org.apache.parquet:parquet-common:1.14.0' - implementation 'org.apache.parquet:parquet-hadoop:1.14.0' + implementation libs.parquet.avro + implementation libs.parquet.column + implementation libs.parquet.common + implementation libs.parquet.hadoop runtimeOnly(libs.hadoop.common) { exclude group: 'org.eclipse.jetty' exclude group: 'org.apache.hadoop', module: 'hadoop-auth' diff --git a/data-prepper-plugins/parse-json-processor/build.gradle b/data-prepper-plugins/parse-json-processor/build.gradle index 44959173ba..488dbf7d86 100644 --- a/data-prepper-plugins/parse-json-processor/build.gradle +++ b/data-prepper-plugins/parse-json-processor/build.gradle @@ -13,7 +13,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-ion' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-xml' - implementation 'org.apache.parquet:parquet-common:1.14.0' + implementation libs.parquet.common testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-test-event') } diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index 638b8246a5..d8ca855b13 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -23,7 +23,7 @@ dependencies { exclude group: 'org.eclipse.jetty' exclude group: 'org.apache.hadoop', module: 'hadoop-auth' } - implementation 'org.apache.parquet:parquet-avro:1.14.0' + implementation libs.parquet.avro implementation 'software.amazon.awssdk:apache-client' implementation 'org.jetbrains.kotlin:kotlin-stdlib-common:1.9.22' implementation libs.commons.lang3 diff --git a/data-prepper-plugins/s3-source/build.gradle b/data-prepper-plugins/s3-source/build.gradle index 2a09ce3d90..b0209a5d08 100644 --- a/data-prepper-plugins/s3-source/build.gradle +++ b/data-prepper-plugins/s3-source/build.gradle @@ -27,7 +27,7 @@ dependencies { implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-csv' implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' implementation 'org.xerial.snappy:snappy-java:1.1.10.5' - implementation 'org.apache.parquet:parquet-common:1.14.0' + implementation libs.parquet.common implementation 'dev.failsafe:failsafe:3.3.2' implementation 'org.apache.httpcomponents:httpcore:4.4.16' testImplementation libs.commons.lang3 @@ -45,11 +45,10 @@ dependencies { testImplementation project(':data-prepper-plugins:parquet-codecs') testImplementation project(':data-prepper-test-event') testImplementation libs.avro.core - testImplementation testLibs.hadoop.common - testImplementation 'org.apache.parquet:parquet-avro:1.14.0' - testImplementation 'org.apache.parquet:parquet-column:1.14.0' - testImplementation 'org.apache.parquet:parquet-common:1.14.0' - testImplementation 'org.apache.parquet:parquet-hadoop:1.14.0' + testImplementation libs.hadoop.common + testImplementation libs.parquet.avro + testImplementation libs.parquet.column + testImplementation libs.parquet.hadoop } test { diff --git a/settings.gradle b/settings.gradle index 64d86219ea..ca9fcfbdfb 100644 --- a/settings.gradle +++ b/settings.gradle @@ -60,7 +60,12 @@ dependencyResolutionManagement { library('commons-io', 'commons-io', 'commons-io').version('2.15.1') library('commons-codec', 'commons-codec', 'commons-codec').version('1.16.0') library('commons-compress', 'org.apache.commons', 'commons-compress').version('1.24.0') - version('hadoop', '3.3.6') + version('parquet', '1.14.1') + library('parquet-common', 'org.apache.parquet', 'parquet-common').versionRef('parquet') + library('parquet-avro', 'org.apache.parquet', 'parquet-avro').versionRef('parquet') + library('parquet-column', 'org.apache.parquet', 'parquet-column').versionRef('parquet') + library('parquet-hadoop', 'org.apache.parquet', 'parquet-hadoop').versionRef('parquet') + version('hadoop', '3.4.0') library('hadoop-common', 'org.apache.hadoop', 'hadoop-common').versionRef('hadoop') library('hadoop-mapreduce', 'org.apache.hadoop', 'hadoop-mapreduce-client-core').versionRef('hadoop') version('avro', '1.11.3') @@ -74,7 +79,6 @@ dependencyResolutionManagement { version('awaitility', '4.2.0') version('spring', '5.3.28') version('slf4j', '2.0.6') - version('hadoop', '3.3.6') library('junit-core', 'org.junit.jupiter', 'junit-jupiter').versionRef('junit') library('junit-params', 'org.junit.jupiter', 'junit-jupiter-params').versionRef('junit') library('junit-engine', 'org.junit.jupiter', 'junit-jupiter-engine').versionRef('junit') @@ -88,7 +92,6 @@ dependencyResolutionManagement { library('awaitility', 'org.awaitility', 'awaitility').versionRef('awaitility') library('spring-test', 'org.springframework', 'spring-test').versionRef('spring') library('slf4j-simple', 'org.slf4j', 'slf4j-simple').versionRef('slf4j') - library('hadoop-common', 'org.apache.hadoop', 'hadoop-common').versionRef('hadoop') } } } From 39c74dba4398406b814f2f20df5d97fe45aabd08 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 3 Jul 2024 12:51:37 -0500 Subject: [PATCH 052/159] Updates to the CODE_OF_CONDUCT.md from the opensearch-project's official CODE_OF_CONDUCT.md. (#4665) Signed-off-by: David Venable --- CODE_OF_CONDUCT.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CODE_OF_CONDUCT.md b/CODE_OF_CONDUCT.md index 2e863762fd..cb23547755 100644 --- a/CODE_OF_CONDUCT.md +++ b/CODE_OF_CONDUCT.md @@ -1,5 +1,5 @@ -This code of conduct applies to all spaces provided by the OpenSource project including in code, documentation, issue trackers, mailing lists, chat channels, wikis, blogs, social media and any other communication channels used by the project. +This code of conduct applies to all spaces provided by the OpenSource project including in code, documentation, issue trackers, mailing lists, chat channels, wikis, blogs, social media, events, conferences, meetings, and any other communication channels used by the project. **Our open source communities endeavor to:** @@ -8,7 +8,6 @@ This code of conduct applies to all spaces provided by the OpenSource project in * Be Respectful: We are committed to encouraging differing viewpoints, accepting constructive criticism and work collaboratively towards decisions that help the project grow. Disrespectful and unacceptable behavior will not be tolerated. * Be Collaborative: We are committed to supporting what is best for our community and users. When we build anything for the benefit of the project, we should document the work we do and communicate to others on how this affects their work. - **Our Responsibility. As contributors, members, or bystanders we each individually have the responsibility to behave professionally and respectfully at all times. Disrespectful and unacceptable behaviors include, but are not limited to:** * The use of violent threats, abusive, discriminatory, or derogatory language; @@ -19,6 +18,7 @@ This code of conduct applies to all spaces provided by the OpenSource project in * Publishing private information, such as physical or electronic address, without permission; * Other conduct which could reasonably be considered inappropriate in a professional setting; * Advocating for or encouraging any of the above behaviors. -* Enforcement and Reporting Code of Conduct Issues: + +**Enforcement and Reporting Code of Conduct Issues:** Instances of abusive, harassing, or otherwise unacceptable behavior may be reported. [Contact us](mailto:opensource-codeofconduct@amazon.com). All complaints will be reviewed and investigated and will result in a response that is deemed necessary and appropriate to the circumstances. From 3d2265fb713ea8e989d8fe6529b015cd4ab16955 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 3 Jul 2024 12:43:46 -0700 Subject: [PATCH 053/159] Key Value processor value grouping optimization (#4704) * dplive1.yaml Signed-off-by: Kondaka * Optimize findInStartGroup in KV processor Signed-off-by: Kondaka * Rebased to latest Signed-off-by: Kondaka * Cleanup Signed-off-by: Kondaka * Minor improvements to code Signed-off-by: Kondaka * Addressed review comments Signed-off-by: Kondaka --------- Signed-off-by: Kondaka --- .../processor/keyvalue/KeyValueProcessor.java | 25 +++++++++++-------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java index ea3a7accdb..c42e015829 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessor.java @@ -281,19 +281,22 @@ private void addPart(List parts, final String str, final int start, fina } } - public int findInStartGroup(final String str, int idx) { + private int findInStartGroup(final String str, final int idx) { + if (idx < 0 || idx >= str.length()) { + return -1; // Invalid starting index + } + for (int j = 0; j < startGroupStrings.length; j++) { - try { - if (startGroupStrings[j].equals(str.substring(idx, idx+startGroupStrings[j].length()))) { - // For " and ', make sure, it's not escaped - if (j <= 1 && (idx == 0 || str.charAt(idx-1) != '\\')) { - return j; - } else if (j > 1) { - return j; - } + String startGroup = startGroupStrings[j]; + int startGroupLen = startGroup.length(); + + if (idx + startGroupLen <= str.length() && str.startsWith(startGroup, idx)) { + // For the first two elements, check for escape characters + if (j <= 1 && (idx == 0 || str.charAt(idx - 1) != '\\')) { + return j; + } else if (j > 1) { + return j; } - } catch (Exception e) { - return -1; } } return -1; From 27796a153c53c1b93f0e9c149ed6b01786fd776c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 3 Jul 2024 12:56:23 -0700 Subject: [PATCH 054/159] Bump net.bytebuddy:byte-buddy-agent in /data-prepper-plugins/opensearch (#4592) Bumps [net.bytebuddy:byte-buddy-agent](https://github.com/raphw/byte-buddy) from 1.14.12 to 1.14.17. - [Release notes](https://github.com/raphw/byte-buddy/releases) - [Changelog](https://github.com/raphw/byte-buddy/blob/master/release-notes.md) - [Commits](https://github.com/raphw/byte-buddy/compare/byte-buddy-1.14.12...byte-buddy-1.14.17) --- updated-dependencies: - dependency-name: net.bytebuddy:byte-buddy-agent dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- data-prepper-plugins/opensearch/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/opensearch/build.gradle b/data-prepper-plugins/opensearch/build.gradle index b87e533afe..4da9ea823c 100644 --- a/data-prepper-plugins/opensearch/build.gradle +++ b/data-prepper-plugins/opensearch/build.gradle @@ -42,7 +42,7 @@ dependencies { testImplementation testLibs.junit.vintage testImplementation libs.commons.io testImplementation 'net.bytebuddy:byte-buddy:1.14.12' - testImplementation 'net.bytebuddy:byte-buddy-agent:1.14.12' + testImplementation 'net.bytebuddy:byte-buddy-agent:1.14.17' testImplementation testLibs.slf4j.simple testImplementation testLibs.mockito.inline } From 3e8513b24e57af04fc6a394f7817c8d81f64a561 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 3 Jul 2024 13:49:02 -0700 Subject: [PATCH 055/159] Bump net.bytebuddy:byte-buddy in /data-prepper-plugins/opensearch (#4593) Bumps [net.bytebuddy:byte-buddy](https://github.com/raphw/byte-buddy) from 1.14.12 to 1.14.17. - [Release notes](https://github.com/raphw/byte-buddy/releases) - [Changelog](https://github.com/raphw/byte-buddy/blob/master/release-notes.md) - [Commits](https://github.com/raphw/byte-buddy/compare/byte-buddy-1.14.12...byte-buddy-1.14.17) --- updated-dependencies: - dependency-name: net.bytebuddy:byte-buddy dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- data-prepper-plugins/opensearch/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/opensearch/build.gradle b/data-prepper-plugins/opensearch/build.gradle index 4da9ea823c..bece32eaae 100644 --- a/data-prepper-plugins/opensearch/build.gradle +++ b/data-prepper-plugins/opensearch/build.gradle @@ -41,7 +41,7 @@ dependencies { } testImplementation testLibs.junit.vintage testImplementation libs.commons.io - testImplementation 'net.bytebuddy:byte-buddy:1.14.12' + testImplementation 'net.bytebuddy:byte-buddy:1.14.17' testImplementation 'net.bytebuddy:byte-buddy-agent:1.14.17' testImplementation testLibs.slf4j.simple testImplementation testLibs.mockito.inline From 253e59245fd9c39c959c1c8caaeff1b226a5a0ab Mon Sep 17 00:00:00 2001 From: Santhosh Gandhe <1909520+san81@users.noreply.github.com> Date: Thu, 4 Jul 2024 11:11:27 -0700 Subject: [PATCH 056/159] Introducing delete input configuration option for some parsers (#4702) * Introduced delete_source configuration option to give flexibility for the user to drop the raw source record if they don't want to propagate it downstream Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * addressing review comments Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * added delete_source option to other similar parser classes Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> --------- Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> --- .../parse/AbstractParseProcessor.java | 6 ++++ .../processor/parse/CommonParseConfig.java | 7 ++++- .../parse/ion/ParseIonProcessorConfig.java | 10 ++++++- .../parse/json/ParseJsonProcessorConfig.java | 10 ++++++- .../parse/xml/ParseXmlProcessorConfig.java | 10 ++++++- .../ion/ParseIonProcessorConfigTest.java | 3 ++ .../parse/ion/ParseIonProcessorTest.java | 17 +++++++++++ .../json/ParseJsonProcessorConfigTest.java | 4 +++ .../parse/json/ParseJsonProcessorTest.java | 30 ++++++++++++++----- .../xml/ParseXmlProcessorConfigTest.java | 3 ++ .../parse/xml/ParseXmlProcessorTest.java | 16 ++++++++++ 11 files changed, 104 insertions(+), 12 deletions(-) diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java index a2b984d070..878316c183 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java @@ -36,6 +36,7 @@ public abstract class AbstractParseProcessor extends AbstractProcessor tagsOnFailure; private final boolean overwriteIfDestinationExists; + private final boolean deleteSourceRequested; private final ExpressionEvaluator expressionEvaluator; @@ -50,6 +51,7 @@ protected AbstractParseProcessor(PluginMetrics pluginMetrics, parseWhen = commonParseConfig.getParseWhen(); tagsOnFailure = commonParseConfig.getTagsOnFailure(); overwriteIfDestinationExists = commonParseConfig.getOverwriteIfDestinationExists(); + deleteSourceRequested = commonParseConfig.isDeleteSourceRequested(); this.expressionEvaluator = expressionEvaluator; } @@ -93,6 +95,10 @@ public Collection> doExecute(final Collection> recor } else if (overwriteIfDestinationExists || !event.containsKey(destination)) { event.put(destination, parsedValue); } + + if(deleteSourceRequested) { + event.delete(this.source); + } } catch (Exception e) { LOG.error(EVENT, "An exception occurred while using the {} processor on Event [{}]", getProcessorName(), record.getData(), e); } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java index 193631bea9..5fd5050b3d 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java @@ -27,7 +27,6 @@ public interface CommonParseConfig { * An optional setting used to specify a JSON Pointer. Pointer points to the JSON key that will be parsed into the destination. * There is no pointer by default, meaning that the entirety of source will be parsed. If the target key would overwrite an existing * key in the Event then the absolute path of the target key will be placed into destination - * * Note: (should this be configurable/what about double conflicts?) * @return String representing JSON Pointer */ @@ -54,4 +53,10 @@ public interface CommonParseConfig { * Defaults to true. */ boolean getOverwriteIfDestinationExists(); + + /** + * An optional setting used to request dropping the original raw message after successfully parsing the input event. + * Defaults to false. + */ + boolean isDeleteSourceRequested(); } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java index 67a2f464ad..fcc2950477 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java @@ -35,6 +35,9 @@ public class ParseIonProcessorConfig implements CommonParseConfig { @JsonProperty("overwrite_if_destination_exists") private boolean overwriteIfDestinationExists = true; + @JsonProperty + private boolean deleteSource = false; + @Override public String getSource() { return source; @@ -68,6 +71,11 @@ boolean isValidDestination() { if (Objects.isNull(destination)) return true; final String trimmedDestination = destination.trim(); - return trimmedDestination.length() != 0 && !(trimmedDestination.equals("/")); + return !trimmedDestination.isEmpty() && !(trimmedDestination.equals("/")); + } + + @Override + public boolean isDeleteSourceRequested() { + return deleteSource; } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java index e0a2e91c1d..49ff2a5969 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java @@ -35,6 +35,9 @@ public class ParseJsonProcessorConfig implements CommonParseConfig { @JsonProperty("overwrite_if_destination_exists") private boolean overwriteIfDestinationExists = true; + @JsonProperty + private boolean deleteSource = false; + @Override public String getSource() { return source; @@ -63,11 +66,16 @@ public boolean getOverwriteIfDestinationExists() { return overwriteIfDestinationExists; } + @Override + public boolean isDeleteSourceRequested() { + return deleteSource; + } + @AssertTrue(message = "destination cannot be empty, whitespace, or a front slash (/)") boolean isValidDestination() { if (Objects.isNull(destination)) return true; final String trimmedDestination = destination.trim(); - return trimmedDestination.length() != 0 && !(trimmedDestination.equals("/")); + return !trimmedDestination.isEmpty() && !(trimmedDestination.equals("/")); } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java index df4fabc397..c90173dc43 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java @@ -30,6 +30,9 @@ public class ParseXmlProcessorConfig implements CommonParseConfig { @JsonProperty("overwrite_if_destination_exists") private boolean overwriteIfDestinationExists = true; + @JsonProperty + private boolean deleteSource = false; + @Override public String getSource() { return source; @@ -65,6 +68,11 @@ boolean isValidDestination() { if (Objects.isNull(destination)) return true; final String trimmedDestination = destination.trim(); - return trimmedDestination.length() != 0 && !(trimmedDestination.equals("/")); + return !trimmedDestination.isEmpty() && !(trimmedDestination.equals("/")); + } + + @Override + public boolean isDeleteSourceRequested() { + return deleteSource; } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfigTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfigTest.java index 0fb274ba13..8c47650c05 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfigTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfigTest.java @@ -57,6 +57,9 @@ void test_when_destinationIsWhiteSpaceOrFrontSlash_then_isValidDestinationFalse( setField(ParseIonProcessorConfig.class, config, "tagsOnFailure", tagsList); assertThat(config.getTagsOnFailure(), equalTo(tagsList)); + + setField(ParseIonProcessorConfig.class, config, "deleteSource", true); + assertThat(config.isDeleteSourceRequested(), equalTo(true)); } } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java index 62873866d7..c9a8fdf4e5 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java @@ -47,6 +47,23 @@ void test_when_using_ion_features_then_processorParsesCorrectly() { final String serializedMessage = "{bareKey: 1, symbol: SYMBOL, timestamp: 2023-11-30T21:05:23.383Z, attribute: dollars::100.0 }"; final Event parsedEvent = createAndParseMessageEvent(serializedMessage); + assertThat(parsedEvent.containsKey(processorConfig.getSource()), equalTo(true)); + assertThat(parsedEvent.get(processorConfig.getSource(), Object.class), equalTo(serializedMessage)); + assertThat(parsedEvent.get("bareKey", Integer.class), equalTo(1)); + assertThat(parsedEvent.get("symbol", String.class), equalTo("SYMBOL")); + assertThat(parsedEvent.get("timestamp", String.class), equalTo("2023-11-30T21:05:23.383Z")); + assertThat(parsedEvent.get("attribute", Double.class), equalTo(100.0)); + } + + @Test + void test_when_deleteSourceFlagEnabled() { + when(processorConfig.isDeleteSourceRequested()).thenReturn(true); + parseJsonProcessor = new ParseIonProcessor(pluginMetrics, ionProcessorConfig, expressionEvaluator); + + final String serializedMessage = "{bareKey: 1, symbol: SYMBOL, timestamp: 2023-11-30T21:05:23.383Z, attribute: dollars::100.0 }"; + final Event parsedEvent = createAndParseMessageEvent(serializedMessage); + + assertThat(parsedEvent.containsKey(processorConfig.getSource()), equalTo(false)); assertThat(parsedEvent.get("bareKey", Integer.class), equalTo(1)); assertThat(parsedEvent.get("symbol", String.class), equalTo("SYMBOL")); assertThat(parsedEvent.get("timestamp", String.class), equalTo("2023-11-30T21:05:23.383Z")); diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfigTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfigTest.java index 459fab6ea5..aa138a0e7e 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfigTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfigTest.java @@ -29,6 +29,7 @@ public void test_when_defaultParseJsonProcessorConfig_then_returns_default_value assertThat(objectUnderTest.getPointer(), equalTo(null)); assertThat(objectUnderTest.getTagsOnFailure(), equalTo(null)); assertThat(objectUnderTest.getOverwriteIfDestinationExists(), equalTo(true)); + assertThat(objectUnderTest.isDeleteSourceRequested(), equalTo(false)); } @Nested @@ -57,6 +58,9 @@ void test_when_destinationIsWhiteSpaceOrFrontSlash_then_isValidDestinationFalse( setField(ParseJsonProcessorConfig.class, config, "tagsOnFailure", tagsList); assertThat(config.getTagsOnFailure(), equalTo(tagsList)); + + setField(ParseJsonProcessorConfig.class, config, "deleteSource", true); + assertThat(config.isDeleteSourceRequested(), equalTo(true)); } } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java index 4594cbe2f5..1416d6cf35 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java @@ -194,6 +194,22 @@ void test_when_nestedJSONArray_then_parsedIntoArrayAndIndicesAccessible() { assertThat(parsedEvent.get(pointerToFirstElement, String.class), equalTo(value.get(0))); } + @Test + void test_when_deleteSourceFlagEnabled() { + when(processorConfig.isDeleteSourceRequested()).thenReturn(true); + parseJsonProcessor = new ParseJsonProcessor(pluginMetrics, jsonProcessorConfig, expressionEvaluator); + + final String key = "key"; + final ArrayList value = new ArrayList<>(List.of("Element0","Element1","Element2")); + final String jsonArray = "{\"key\":[\"Element0\",\"Element1\",\"Element2\"]}"; + final Event parsedEvent = createAndParseMessageEvent(jsonArray); + + assertThat(parsedEvent.containsKey(processorConfig.getSource()), equalTo(false)); + assertThat(parsedEvent.get(key, ArrayList.class), equalTo(value)); + final String pointerToFirstElement = key + "/0"; + assertThat(parsedEvent.get(pointerToFirstElement, String.class), equalTo(value.get(0))); + } + @Test void test_when_nestedJSONArrayOfJSON_then_parsedIntoArrayAndIndicesAccessible() { parseJsonProcessor = createObjectUnderTest(); @@ -373,23 +389,21 @@ private String constructDeeplyNestedJsonPointer(final int numberOfLayers) { /** * Naive serialization that converts every = to : and wraps every word with double quotes (no error handling or input validation). - * @param messageMap - * @return + * @param messageMap source key value map + * @return serialized string representation of the map */ private String convertMapToJSONString(final Map messageMap) { final String replaceEquals = messageMap.toString().replace("=",":"); - final String addQuotes = replaceEquals.replaceAll("(\\w+)", "\"$1\""); // wrap every word in quotes - return addQuotes; + return replaceEquals.replaceAll("(\\w+)", "\"$1\""); } /** * Creates a Map that maps a single key to a value nested numberOfLayers layers deep. - * @param numberOfLayers - * @return + * @param numberOfLayers indicates the depth of layers count + * @return a Map representing the nested structure */ private Map constructArbitrarilyDeepJsonMap(final int numberOfLayers) { - final Map result = Collections.singletonMap(DEEPLY_NESTED_KEY_NAME,deepJsonMapHelper(0,numberOfLayers)); - return result; + return Collections.singletonMap(DEEPLY_NESTED_KEY_NAME,deepJsonMapHelper(0,numberOfLayers)); } private Object deepJsonMapHelper(final int currentLayer, final int numberOfLayers) { diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfigTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfigTest.java index d5e7e1ec43..bab6d6e919 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfigTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfigTest.java @@ -52,6 +52,9 @@ void test_when_destinationIsWhiteSpaceOrFrontSlash_then_isValidDestinationFalse( setField(ParseXmlProcessorConfig.class, config, "tagsOnFailure", tagsList); assertThat(config.getTagsOnFailure(), equalTo(tagsList)); + + setField(ParseXmlProcessorConfig.class, config, "deleteSource", true); + assertThat(config.isDeleteSourceRequested(), equalTo(true)); } } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java index 51de35ca70..8d9bc4cde3 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java @@ -60,6 +60,22 @@ void test_when_using_xml_features_then_processorParsesCorrectly() { assertThat(parsedEvent.get("age", String.class), equalTo("30")); } + @Test + void test_when_deleteSourceFlagEnabled() { + + final String tagOnFailure = UUID.randomUUID().toString(); + when(processorConfig.getTagsOnFailure()).thenReturn(List.of(tagOnFailure)); + when(processorConfig.isDeleteSourceRequested()).thenReturn(true); + + parseXmlProcessor = createObjectUnderTest(); + + final String serializedMessage = "John Doe30"; + final Event parsedEvent = createAndParseMessageEvent(serializedMessage); + assertThat(parsedEvent.containsKey(processorConfig.getSource()), equalTo(false)); + assertThat(parsedEvent.get("name", String.class), equalTo("John Doe")); + assertThat(parsedEvent.get("age", String.class), equalTo("30")); + } + @Test void test_when_using_invalid_xml_tags_correctly() { From 1d259cff3a8d8a529c40142676c9be06e931b38d Mon Sep 17 00:00:00 2001 From: David Venable Date: Tue, 9 Jul 2024 15:57:19 -0500 Subject: [PATCH 057/159] Updates the user_agent processor to use the EventKey. (#4628) Updates the user_agent processor to use the EventKey. Signed-off-by: David Venable Co-authored-by: Karsten Schnitter --- .../user-agent-processor/build.gradle | 1 + .../useragent/UserAgentProcessor.java | 17 +++++++++++++---- .../useragent/UserAgentProcessorConfig.java | 8 ++++++-- .../useragent/UserAgentProcessorTest.java | 18 ++++++++++++------ 4 files changed, 32 insertions(+), 12 deletions(-) diff --git a/data-prepper-plugins/user-agent-processor/build.gradle b/data-prepper-plugins/user-agent-processor/build.gradle index 746ee40397..5e92b158f5 100644 --- a/data-prepper-plugins/user-agent-processor/build.gradle +++ b/data-prepper-plugins/user-agent-processor/build.gradle @@ -13,6 +13,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.github.ua-parser:uap-java:1.6.1' implementation libs.caffeine + testImplementation project(':data-prepper-test-event') } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessor.java b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessor.java index 32779655dc..c84b308645 100644 --- a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessor.java +++ b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessor.java @@ -9,6 +9,8 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -30,12 +32,19 @@ public class UserAgentProcessor extends AbstractProcessor, Record< private static final Logger LOG = LoggerFactory.getLogger(UserAgentProcessor.class); private final UserAgentProcessorConfig config; private final Parser userAgentParser; + private final EventKey sourceKey; + private final EventKey targetKey; @DataPrepperPluginConstructor - public UserAgentProcessor(final PluginMetrics pluginMetrics, final UserAgentProcessorConfig config) { + public UserAgentProcessor( + final UserAgentProcessorConfig config, + final EventKeyFactory eventKeyFactory, + final PluginMetrics pluginMetrics) { super(pluginMetrics); this.config = config; this.userAgentParser = new CaffeineCachingParser(config.getCacheSize()); + this.sourceKey = config.getSource(); + this.targetKey = eventKeyFactory.createEventKey(config.getTarget(), EventKeyFactory.EventAction.PUT); } @Override @@ -44,7 +53,7 @@ public Collection> doExecute(final Collection> recor final Event event = record.getData(); try { - final String userAgentStr = event.get(config.getSource(), String.class); + final String userAgentStr = event.get(sourceKey, String.class); Objects.requireNonNull(userAgentStr); final Client clientInfo = this.userAgentParser.parse(userAgentStr); @@ -53,10 +62,10 @@ public Collection> doExecute(final Collection> recor if (!config.getExcludeOriginal()) { parsedUserAgent.put("original", userAgentStr); } - event.put(config.getTarget(), parsedUserAgent); + event.put(targetKey, parsedUserAgent); } catch (Exception e) { LOG.error(EVENT, "An exception occurred when parsing user agent data from event [{}] with source key [{}]", - event, config.getSource(), e); + event, sourceKey, e); final List tagsOnParseFailure = config.getTagsOnParseFailure(); if (Objects.nonNull(tagsOnParseFailure) && tagsOnParseFailure.size() > 0) { diff --git a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java index e62fc5a2da..0dcf46e2a1 100644 --- a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java +++ b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java @@ -8,6 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import java.util.List; @@ -18,7 +21,8 @@ public class UserAgentProcessorConfig { @NotEmpty @NotNull @JsonProperty("source") - private String source; + @EventKeyConfiguration(EventKeyFactory.EventAction.GET) + private EventKey source; @NotNull @JsonProperty("target") @@ -34,7 +38,7 @@ public class UserAgentProcessorConfig { @JsonProperty("tags_on_parse_failure") private List tagsOnParseFailure; - public String getSource() { + public EventKey getSource() { return source; } diff --git a/data-prepper-plugins/user-agent-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorTest.java b/data-prepper-plugins/user-agent-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorTest.java index da0923f509..a346218d0a 100644 --- a/data-prepper-plugins/user-agent-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorTest.java +++ b/data-prepper-plugins/user-agent-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorTest.java @@ -12,8 +12,10 @@ import org.junit.jupiter.params.provider.MethodSource; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.event.TestEventKeyFactory; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; @@ -38,11 +40,13 @@ class UserAgentProcessorTest { @Mock private UserAgentProcessorConfig mockConfig; + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); + @ParameterizedTest @MethodSource("userAgentStringArguments") public void testParsingUserAgentStrings( String uaString, String uaName, String uaVersion, String osName, String osVersion, String osFull, String deviceName) { - when(mockConfig.getSource()).thenReturn("source"); + when(mockConfig.getSource()).thenReturn(eventKeyFactory.createEventKey("source")); when(mockConfig.getTarget()).thenReturn("user_agent"); when(mockConfig.getCacheSize()).thenReturn(TEST_CACHE_SIZE); @@ -64,7 +68,7 @@ public void testParsingUserAgentStrings( @MethodSource("userAgentStringArguments") public void testParsingUserAgentStringsWithCustomTarget( String uaString, String uaName, String uaVersion, String osName, String osVersion, String osFull, String deviceName) { - when(mockConfig.getSource()).thenReturn("source"); + when(mockConfig.getSource()).thenReturn(eventKeyFactory.createEventKey("source")); when(mockConfig.getTarget()).thenReturn("my_target"); when(mockConfig.getCacheSize()).thenReturn(TEST_CACHE_SIZE); @@ -86,7 +90,7 @@ public void testParsingUserAgentStringsWithCustomTarget( @MethodSource("userAgentStringArguments") public void testParsingUserAgentStringsExcludeOriginal( String uaString, String uaName, String uaVersion, String osName, String osVersion, String osFull, String deviceName) { - when(mockConfig.getSource()).thenReturn("source"); + when(mockConfig.getSource()).thenReturn(eventKeyFactory.createEventKey("source")); when(mockConfig.getTarget()).thenReturn("user_agent"); when(mockConfig.getExcludeOriginal()).thenReturn(true); when(mockConfig.getCacheSize()).thenReturn(TEST_CACHE_SIZE); @@ -107,8 +111,9 @@ public void testParsingUserAgentStringsExcludeOriginal( @Test public void testParsingWhenUserAgentStringNotExist() { - when(mockConfig.getSource()).thenReturn("bad_source"); + when(mockConfig.getSource()).thenReturn(eventKeyFactory.createEventKey("bad_source")); when(mockConfig.getCacheSize()).thenReturn(TEST_CACHE_SIZE); + when(mockConfig.getTarget()).thenReturn("user_agent"); final UserAgentProcessor processor = createObjectUnderTest(); final Record testRecord = createTestRecord(UUID.randomUUID().toString()); @@ -120,8 +125,9 @@ public void testParsingWhenUserAgentStringNotExist() { @Test public void testTagsAddedOnParseFailure() { - when(mockConfig.getSource()).thenReturn("bad_source"); + when(mockConfig.getSource()).thenReturn(eventKeyFactory.createEventKey("bad_source")); when(mockConfig.getCacheSize()).thenReturn(TEST_CACHE_SIZE); + when(mockConfig.getTarget()).thenReturn("user_agent"); final String tagOnFailure1 = UUID.randomUUID().toString(); final String tagOnFailure2 = UUID.randomUUID().toString(); @@ -138,7 +144,7 @@ public void testTagsAddedOnParseFailure() { } private UserAgentProcessor createObjectUnderTest() { - return new UserAgentProcessor(pluginMetrics, mockConfig); + return new UserAgentProcessor(mockConfig, eventKeyFactory, pluginMetrics); } private Record createTestRecord(String uaString) { From ce887653629cd2c12c6e0d45401de30be188965d Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 10 Jul 2024 10:49:20 -0500 Subject: [PATCH 058/159] Removes Zookeeper from Data Prepper. This was a transitive dependency from Hadoop. (#4707) Signed-off-by: David Venable --- data-prepper-plugins/parquet-codecs/build.gradle | 4 ++++ data-prepper-plugins/s3-sink/build.gradle | 1 + data-prepper-plugins/s3-source/build.gradle | 6 +++++- 3 files changed, 10 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/parquet-codecs/build.gradle b/data-prepper-plugins/parquet-codecs/build.gradle index fbc8f4a209..c402fb6741 100644 --- a/data-prepper-plugins/parquet-codecs/build.gradle +++ b/data-prepper-plugins/parquet-codecs/build.gradle @@ -15,15 +15,19 @@ dependencies { runtimeOnly(libs.hadoop.common) { exclude group: 'org.eclipse.jetty' exclude group: 'org.apache.hadoop', module: 'hadoop-auth' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' } runtimeOnly(libs.hadoop.mapreduce) { + exclude group: 'org.eclipse.jetty' exclude group: 'org.apache.hadoop', module: 'hadoop-hdfs-client' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' } testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-test-event') testImplementation(libs.hadoop.common) { exclude group: 'org.eclipse.jetty' exclude group: 'org.apache.hadoop', module: 'hadoop-auth' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' } constraints { diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index d8ca855b13..4ea0a364fd 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -22,6 +22,7 @@ dependencies { implementation(libs.hadoop.common) { exclude group: 'org.eclipse.jetty' exclude group: 'org.apache.hadoop', module: 'hadoop-auth' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' } implementation libs.parquet.avro implementation 'software.amazon.awssdk:apache-client' diff --git a/data-prepper-plugins/s3-source/build.gradle b/data-prepper-plugins/s3-source/build.gradle index b0209a5d08..06818d8eaa 100644 --- a/data-prepper-plugins/s3-source/build.gradle +++ b/data-prepper-plugins/s3-source/build.gradle @@ -45,7 +45,11 @@ dependencies { testImplementation project(':data-prepper-plugins:parquet-codecs') testImplementation project(':data-prepper-test-event') testImplementation libs.avro.core - testImplementation libs.hadoop.common + testImplementation(libs.hadoop.common) { + exclude group: 'org.eclipse.jetty' + exclude group: 'org.apache.hadoop', module: 'hadoop-auth' + exclude group: 'org.apache.zookeeper', module: 'zookeeper' + } testImplementation libs.parquet.avro testImplementation libs.parquet.column testImplementation libs.parquet.hadoop From 0f5e10f7d70fe3658c79de86725935b8ea853e73 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 10 Jul 2024 10:50:21 -0500 Subject: [PATCH 059/159] Update the mutate string processors to use the EventKey. #4646 (#4649) Change the source and keys properties for mutate string processors to use EventKey such that they are parsed by Data Prepper core. Also, use the TestEventFactory in the tests to avoid use of JacksonEvent directly. Removes an unused class. Signed-off-by: David Venable --- .../mutate-string-processors/build.gradle | 1 + .../mutatestring/AbstractStringProcessor.java | 7 +++-- .../LowercaseStringProcessor.java | 7 +++-- .../mutatestring/SplitStringProcessor.java | 3 +- .../SplitStringProcessorConfig.java | 7 +++-- .../SubstituteStringProcessor.java | 3 +- .../SubstituteStringProcessorConfig.java | 7 +++-- .../mutatestring/TrimStringProcessor.java | 7 +++-- .../UppercaseStringProcessor.java | 7 +++-- .../mutatestring/WithKeysConfig.java | 9 +++--- .../mutatestring/WithKeysProcessorConfig.java | 28 ------------------- .../LowercaseStringProcessorTests.java | 26 +++++++++++------ .../SplitStringProcessorTests.java | 14 ++++++++-- .../SubstituteStringProcessorTests.java | 15 ++++++++-- .../TrimStringProcessorTests.java | 25 +++++++++++------ .../UppercaseStringProcessorTests.java | 26 +++++++++++------ 16 files changed, 107 insertions(+), 85 deletions(-) delete mode 100644 data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysProcessorConfig.java diff --git a/data-prepper-plugins/mutate-string-processors/build.gradle b/data-prepper-plugins/mutate-string-processors/build.gradle index 3fbbc37254..0723e63c10 100644 --- a/data-prepper-plugins/mutate-string-processors/build.gradle +++ b/data-prepper-plugins/mutate-string-processors/build.gradle @@ -22,4 +22,5 @@ dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:common') implementation 'com.fasterxml.jackson.core:jackson-databind' + testImplementation project(':data-prepper-test-event') } \ No newline at end of file diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/AbstractStringProcessor.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/AbstractStringProcessor.java index 19d11daf62..ae7a242da3 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/AbstractStringProcessor.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/AbstractStringProcessor.java @@ -8,6 +8,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.record.Record; @@ -46,8 +47,8 @@ public Collection> doExecute(final Collection> recor private void performStringAction(final Event recordEvent) { try { - for(T entry : entries) { - final String key = getKey(entry); + for(final T entry : entries) { + final EventKey key = getKey(entry); if(recordEvent.containsKey(key)) { final Object value = recordEvent.get(key, Object.class); @@ -64,7 +65,7 @@ private void performStringAction(final Event recordEvent) protected abstract void performKeyAction(final Event recordEvent, final T entry, final String value); - protected abstract String getKey(final T entry); + protected abstract EventKey getKey(final T entry); @Override public void prepareForShutdown() { diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/LowercaseStringProcessor.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/LowercaseStringProcessor.java index b76e922c61..c2c2071e95 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/LowercaseStringProcessor.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/LowercaseStringProcessor.java @@ -9,6 +9,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.processor.Processor; import java.util.Locale; @@ -18,20 +19,20 @@ * no action is performed. */ @DataPrepperPlugin(name = "lowercase_string", pluginType = Processor.class, pluginConfigurationType = WithKeysConfig.class) -public class LowercaseStringProcessor extends AbstractStringProcessor { +public class LowercaseStringProcessor extends AbstractStringProcessor { @DataPrepperPluginConstructor public LowercaseStringProcessor(final PluginMetrics pluginMetrics, final WithKeysConfig config) { super(pluginMetrics, config); } @Override - protected void performKeyAction(final Event recordEvent, final String key, final String value) + protected void performKeyAction(final Event recordEvent, final EventKey key, final String value) { recordEvent.put(key, value.toLowerCase(Locale.ROOT)); } @Override - protected String getKey(final String entry) { + protected EventKey getKey(final EventKey entry) { return entry; } } diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessor.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessor.java index acac832095..6bc89178d8 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessor.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.processor.Processor; import java.util.HashMap; @@ -64,7 +65,7 @@ protected void performKeyAction(final Event recordEvent, final SplitStringProces } @Override - protected String getKey(final SplitStringProcessorConfig.Entry entry) { + protected EventKey getKey(final SplitStringProcessorConfig.Entry entry) { return entry.getSource(); } diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java index 84e4228798..25809819f8 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java @@ -11,6 +11,7 @@ import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.Size; +import org.opensearch.dataprepper.model.event.EventKey; import java.util.List; @@ -19,7 +20,7 @@ public static class Entry { @NotEmpty @NotNull - private String source; + private EventKey source; @JsonProperty("delimiter_regex") private String delimiterRegex; @@ -30,7 +31,7 @@ public static class Entry { @JsonProperty("split_when") private String splitWhen; - public String getSource() { + public EventKey getSource() { return source; } @@ -44,7 +45,7 @@ public String getDelimiter() { public String getSplitWhen() { return splitWhen; } - public Entry(final String source, final String delimiterRegex, final String delimiter, final String splitWhen) { + public Entry(final EventKey source, final String delimiterRegex, final String delimiter, final String splitWhen) { this.source = source; this.delimiterRegex = delimiterRegex; this.delimiter = delimiter; diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessor.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessor.java index 7332ce836f..e6dceb62fc 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessor.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.processor.Processor; import java.util.HashMap; @@ -51,7 +52,7 @@ protected void performKeyAction(final Event recordEvent, final SubstituteStringP } @Override - protected String getKey(final SubstituteStringProcessorConfig.Entry entry) { + protected EventKey getKey(final SubstituteStringProcessorConfig.Entry entry) { return entry.getSource(); } } diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java index 07789b083a..5813b7cf0b 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java @@ -6,19 +6,20 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; import com.fasterxml.jackson.annotation.JsonProperty; +import org.opensearch.dataprepper.model.event.EventKey; import java.util.List; public class SubstituteStringProcessorConfig implements StringProcessorConfig { public static class Entry { - private String source; + private EventKey source; private String from; private String to; @JsonProperty("substitute_when") private String substituteWhen; - public String getSource() { + public EventKey getSource() { return source; } @@ -32,7 +33,7 @@ public String getTo() { public String getSubstituteWhen() { return substituteWhen; } - public Entry(final String source, final String from, final String to, final String substituteWhen) { + public Entry(final EventKey source, final String from, final String to, final String substituteWhen) { this.source = source; this.from = from; this.to = to; diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/TrimStringProcessor.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/TrimStringProcessor.java index 2f0e5f0dc2..2a1213f30f 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/TrimStringProcessor.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/TrimStringProcessor.java @@ -9,6 +9,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.processor.Processor; /** @@ -16,20 +17,20 @@ * If the value is not a string, no action is performed. */ @DataPrepperPlugin(name = "trim_string", pluginType = Processor.class, pluginConfigurationType = WithKeysConfig.class) -public class TrimStringProcessor extends AbstractStringProcessor { +public class TrimStringProcessor extends AbstractStringProcessor { @DataPrepperPluginConstructor public TrimStringProcessor(final PluginMetrics pluginMetrics, final WithKeysConfig config) { super(pluginMetrics, config); } @Override - protected void performKeyAction(final Event recordEvent, final String key, final String value) + protected void performKeyAction(final Event recordEvent, final EventKey key, final String value) { recordEvent.put(key, value.trim()); } @Override - protected String getKey(final String entry) { + protected EventKey getKey(final EventKey entry) { return entry; } } diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/UppercaseStringProcessor.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/UppercaseStringProcessor.java index 9d3665fdd2..28e7aa9847 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/UppercaseStringProcessor.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/UppercaseStringProcessor.java @@ -9,6 +9,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.processor.Processor; import java.util.Locale; @@ -18,19 +19,19 @@ * no action is performed. */ @DataPrepperPlugin(name = "uppercase_string", pluginType = Processor.class, pluginConfigurationType = WithKeysConfig.class) -public class UppercaseStringProcessor extends AbstractStringProcessor { +public class UppercaseStringProcessor extends AbstractStringProcessor { @DataPrepperPluginConstructor public UppercaseStringProcessor(final PluginMetrics pluginMetrics, final WithKeysConfig config) { super(pluginMetrics, config); } @Override - protected String getKey(final String entry) { + protected EventKey getKey(final EventKey entry) { return entry; } @Override - protected void performKeyAction(final Event recordEvent, final String entry, final String value) + protected void performKeyAction(final Event recordEvent, final EventKey entry, final String value) { recordEvent.put(entry, value.toUpperCase(Locale.ROOT)); } diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java index bfe10d02ca..05a9c198a6 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java @@ -8,22 +8,23 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.event.EventKey; import java.util.List; -public class WithKeysConfig implements StringProcessorConfig { +public class WithKeysConfig implements StringProcessorConfig { @NotNull @NotEmpty @JsonProperty("with_keys") - private List withKeys; + private List withKeys; @Override - public List getIterativeConfig() { + public List getIterativeConfig() { return withKeys; } - public List getWithKeys() { + public List getWithKeys() { return withKeys; } } diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysProcessorConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysProcessorConfig.java deleted file mode 100644 index 814518c83d..0000000000 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysProcessorConfig.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.processor.mutatestring; - -import com.fasterxml.jackson.annotation.JsonProperty; -import jakarta.validation.constraints.NotEmpty; -import jakarta.validation.constraints.NotNull; - -import java.util.List; - -public abstract class WithKeysProcessorConfig implements StringProcessorConfig { - @NotEmpty - @NotNull - @JsonProperty("with_keys") - private List withKeys; - - @Override - public List getIterativeConfig() { - return withKeys; - } - - public List getWithKeys() { - return withKeys; - } -} diff --git a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/LowercaseStringProcessorTests.java b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/LowercaseStringProcessorTests.java index 18bddf31a9..8185d8ef8c 100644 --- a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/LowercaseStringProcessorTests.java +++ b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/LowercaseStringProcessorTests.java @@ -5,21 +5,26 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.record.Record; 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.event.TestEventFactory; +import org.opensearch.dataprepper.event.TestEventKeyFactory; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.record.Record; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; @@ -29,6 +34,9 @@ @ExtendWith(MockitoExtension.class) public class LowercaseStringProcessorTests { + private static final EventFactory TEST_EVENT_FACTORY = TestEventFactory.getTestEventFactory(); + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); + @Mock private PluginMetrics pluginMetrics; @@ -37,7 +45,7 @@ public class LowercaseStringProcessorTests { @BeforeEach public void setup() { - lenient().when(config.getIterativeConfig()).thenReturn(Collections.singletonList("message")); + lenient().when(config.getIterativeConfig()).thenReturn(Stream.of("message").map(eventKeyFactory::createEventKey).collect(Collectors.toList())); } @Test @@ -52,7 +60,7 @@ public void testHappyPathLowercaseStringProcessor() { @Test public void testHappyPathMultiLowercaseStringProcessor() { - when(config.getIterativeConfig()).thenReturn(Arrays.asList("message", "message2")); + when(config.getIterativeConfig()).thenReturn(Stream.of("message", "message2").map(eventKeyFactory::createEventKey).collect(Collectors.toList())); final LowercaseStringProcessor processor = createObjectUnderTest(); final Record record = getEvent("THISISAMESSAGE"); @@ -67,7 +75,7 @@ public void testHappyPathMultiLowercaseStringProcessor() { @Test public void testHappyPathMultiMixedLowercaseStringProcessor() { - lenient().when(config.getIterativeConfig()).thenReturn(Arrays.asList("message", "message2")); + lenient().when(config.getIterativeConfig()).thenReturn(Stream.of("message", "message2").map(eventKeyFactory::createEventKey).collect(Collectors.toList())); final LowercaseStringProcessor processor = createObjectUnderTest(); final Record record = getEvent("THISISAMESSAGE"); @@ -137,7 +145,7 @@ private Record getEvent(Object message) { } private static Record buildRecordWithEvent(final Map data) { - return new Record<>(JacksonEvent.builder() + return new Record<>(TEST_EVENT_FACTORY.eventBuilder(EventBuilder.class) .withData(data) .withEventType("event") .build()); diff --git a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorTests.java b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorTests.java index 1f2db4a672..7883dcfd05 100644 --- a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorTests.java +++ b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorTests.java @@ -5,10 +5,15 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; +import org.opensearch.dataprepper.event.TestEventFactory; +import org.opensearch.dataprepper.event.TestEventKeyFactory; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -36,6 +41,8 @@ @ExtendWith(MockitoExtension.class) class SplitStringProcessorTests { + private final EventFactory testEventFactory = TestEventFactory.getTestEventFactory(); + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); @Mock private PluginMetrics pluginMetrics; @@ -115,13 +122,14 @@ void test_event_is_the_same_when_splitWhen_condition_returns_false() { private SplitStringProcessorConfig.Entry createEntry(final String source, final String delimiterRegex, final String delimiter, final String splitWhen) { - return new SplitStringProcessorConfig.Entry(source, delimiterRegex, delimiter, splitWhen); + final EventKey sourceKey = eventKeyFactory.createEventKey(source); + return new SplitStringProcessorConfig.Entry(sourceKey, delimiterRegex, delimiter, splitWhen); } private Record createEvent(final String message) { final Map eventData = new HashMap<>(); eventData.put("message", message); - return new Record<>(JacksonEvent.builder() + return new Record<>(testEventFactory.eventBuilder(EventBuilder.class) .withEventType("event") .withData(eventData) .build()); diff --git a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorTests.java b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorTests.java index 04175ee229..dd8d9b1dd8 100644 --- a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorTests.java +++ b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorTests.java @@ -5,10 +5,15 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; +import org.opensearch.dataprepper.event.TestEventFactory; +import org.opensearch.dataprepper.event.TestEventKeyFactory; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -33,6 +38,8 @@ @ExtendWith(MockitoExtension.class) public class SubstituteStringProcessorTests { + private static final EventFactory TEST_EVENT_FACTORY = TestEventFactory.getTestEventFactory(); + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); @Mock private PluginMetrics pluginMetrics; @@ -42,6 +49,7 @@ public class SubstituteStringProcessorTests { @Mock private ExpressionEvaluator expressionEvaluator; + @BeforeEach public void setup() { lenient().when(config.getIterativeConfig()).thenReturn(Collections.singletonList(createEntry("message", "a", "b", null))); @@ -181,7 +189,8 @@ public boolean equals(Object other) { } private SubstituteStringProcessorConfig.Entry createEntry(final String source, final String from, final String to, final String substituteWhen) { - final SubstituteStringProcessorConfig.Entry entry = new SubstituteStringProcessorConfig.Entry(source, from, to, substituteWhen); + final EventKey sourceKey = eventKeyFactory.createEventKey(source); + final SubstituteStringProcessorConfig.Entry entry = new SubstituteStringProcessorConfig.Entry(sourceKey, from, to, substituteWhen); return entry; } @@ -197,7 +206,7 @@ private Record getEvent(Object message) { } private static Record buildRecordWithEvent(final Map data) { - return new Record<>(JacksonEvent.builder() + return new Record<>(TEST_EVENT_FACTORY.eventBuilder(EventBuilder.class) .withData(data) .withEventType("event") .build()); diff --git a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/TrimStringProcessorTests.java b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/TrimStringProcessorTests.java index 06efbbad96..921f6a6094 100644 --- a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/TrimStringProcessorTests.java +++ b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/TrimStringProcessorTests.java @@ -5,21 +5,26 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.record.Record; 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.event.TestEventFactory; +import org.opensearch.dataprepper.event.TestEventKeyFactory; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.record.Record; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; @@ -29,6 +34,8 @@ @ExtendWith(MockitoExtension.class) public class TrimStringProcessorTests { + private static final EventFactory TEST_EVENT_FACTORY = TestEventFactory.getTestEventFactory(); + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); @Mock private PluginMetrics pluginMetrics; @@ -37,7 +44,7 @@ public class TrimStringProcessorTests { @BeforeEach public void setup() { - lenient().when(config.getIterativeConfig()).thenReturn(Collections.singletonList("message")); + lenient().when(config.getIterativeConfig()).thenReturn(Stream.of("message").map(eventKeyFactory::createEventKey).collect(Collectors.toList())); } @Test @@ -62,7 +69,7 @@ public void testSpaceInMiddleTrimStringProcessor() { @Test public void testHappyPathMultiTrimStringProcessor() { - when(config.getIterativeConfig()).thenReturn(Arrays.asList("message", "message2")); + when(config.getIterativeConfig()).thenReturn(Stream.of("message", "message2").map(eventKeyFactory::createEventKey).collect(Collectors.toList())); final TrimStringProcessor processor = createObjectUnderTest(); final Record record = getEvent("thisisamessage "); @@ -77,7 +84,7 @@ public void testHappyPathMultiTrimStringProcessor() { @Test public void testHappyPathMultiMixedTrimStringProcessor() { - lenient().when(config.getIterativeConfig()).thenReturn(Arrays.asList("message", "message2")); + lenient().when(config.getIterativeConfig()).thenReturn(Stream.of("message", "message2").map(eventKeyFactory::createEventKey).collect(Collectors.toList())); final TrimStringProcessor processor = createObjectUnderTest(); final Record record = getEvent("thisisamessage "); @@ -147,7 +154,7 @@ private Record getEvent(Object message) { } private static Record buildRecordWithEvent(final Map data) { - return new Record<>(JacksonEvent.builder() + return new Record<>(TEST_EVENT_FACTORY.eventBuilder(EventBuilder.class) .withData(data) .withEventType("event") .build()); diff --git a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/UppercaseStringProcessorTests.java b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/UppercaseStringProcessorTests.java index 14af79d202..c4db6a55e5 100644 --- a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/UppercaseStringProcessorTests.java +++ b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/UppercaseStringProcessorTests.java @@ -5,21 +5,26 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.record.Record; 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.event.TestEventFactory; +import org.opensearch.dataprepper.event.TestEventKeyFactory; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.record.Record; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; @@ -29,6 +34,9 @@ @ExtendWith(MockitoExtension.class) public class UppercaseStringProcessorTests { + private static final EventFactory TEST_EVENT_FACTORY = TestEventFactory.getTestEventFactory(); + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); + @Mock private PluginMetrics pluginMetrics; @@ -37,7 +45,7 @@ public class UppercaseStringProcessorTests { @BeforeEach public void setup() { - lenient().when(config.getIterativeConfig()).thenReturn(Collections.singletonList("message")); + lenient().when(config.getIterativeConfig()).thenReturn(Stream.of("message").map(eventKeyFactory::createEventKey).collect(Collectors.toList())); } @Test @@ -52,7 +60,7 @@ public void testHappyPathUppercaseStringProcessor() { @Test public void testHappyPathMultiUppercaseStringProcessor() { - when(config.getIterativeConfig()).thenReturn(Arrays.asList("message", "message2")); + when(config.getIterativeConfig()).thenReturn(Stream.of("message", "message2").map(eventKeyFactory::createEventKey).collect(Collectors.toList())); final UppercaseStringProcessor processor = createObjectUnderTest(); final Record record = getEvent("thisisamessage"); @@ -67,7 +75,7 @@ public void testHappyPathMultiUppercaseStringProcessor() { @Test public void testHappyPathMultiMixedUppercaseStringProcessor() { - lenient().when(config.getIterativeConfig()).thenReturn(Arrays.asList("message", "message2")); + lenient().when(config.getIterativeConfig()).thenReturn(Stream.of("message", "message2").map(eventKeyFactory::createEventKey).collect(Collectors.toList())); final UppercaseStringProcessor processor = createObjectUnderTest(); final Record record = getEvent("thisisamessage"); @@ -137,7 +145,7 @@ private Record getEvent(Object message) { } private static Record buildRecordWithEvent(final Map data) { - return new Record<>(JacksonEvent.builder() + return new Record<>(TEST_EVENT_FACTORY.eventBuilder(EventBuilder.class) .withData(data) .withEventType("event") .build()); From 782ad5118c9bb9b6fb752edd5d0914309795f456 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 10 Jul 2024 10:50:39 -0500 Subject: [PATCH 060/159] Update the rename_keys and delete_entries processors to use the EventKey. (#4636) Signed-off-by: David Venable --- .../mutate-event-processors/build.gradle | 2 ++ .../mutateevent/DeleteEntryProcessor.java | 6 +++-- .../DeleteEntryProcessorConfig.java | 10 ++++++-- .../mutateevent/RenameKeyProcessorConfig.java | 15 +++++++---- .../DeleteEntryProcessorTests.java | 25 ++++++++++++------- .../mutateevent/RenameKeyProcessorTests.java | 9 ++++++- 6 files changed, 48 insertions(+), 19 deletions(-) diff --git a/data-prepper-plugins/mutate-event-processors/build.gradle b/data-prepper-plugins/mutate-event-processors/build.gradle index 3fbbc37254..e4b0c63cea 100644 --- a/data-prepper-plugins/mutate-event-processors/build.gradle +++ b/data-prepper-plugins/mutate-event-processors/build.gradle @@ -22,4 +22,6 @@ dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:common') implementation 'com.fasterxml.jackson.core:jackson-databind' + testImplementation project(':data-prepper-test-event') + testImplementation testLibs.slf4j.simple } \ No newline at end of file diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessor.java index d7c902a32c..cfadf70d03 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -17,6 +18,7 @@ import org.slf4j.LoggerFactory; import java.util.Collection; +import java.util.List; import java.util.Objects; import static org.opensearch.dataprepper.logging.DataPrepperMarkers.EVENT; @@ -25,7 +27,7 @@ public class DeleteEntryProcessor extends AbstractProcessor, Record> { private static final Logger LOG = LoggerFactory.getLogger(DeleteEntryProcessor.class); - private final String[] entries; + private final List entries; private final String deleteWhen; private final ExpressionEvaluator expressionEvaluator; @@ -49,7 +51,7 @@ public Collection> doExecute(final Collection> recor } - for (String entry : entries) { + for (final EventKey entry : entries) { recordEvent.delete(entry); } } catch (final Exception e) { diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java index 8470576a7b..a60c2b08bf 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java @@ -8,17 +8,23 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import java.util.List; public class DeleteEntryProcessorConfig { @NotEmpty @NotNull @JsonProperty("with_keys") - private String[] withKeys; + @EventKeyConfiguration(EventKeyFactory.EventAction.DELETE) + private List<@NotNull @NotEmpty EventKey> withKeys; @JsonProperty("delete_when") private String deleteWhen; - public String[] getWithKeys() { + public List getWithKeys() { return withKeys; } diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java index f1e723ad5a..d1ee0178a6 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java @@ -9,6 +9,9 @@ import jakarta.validation.Valid; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import java.util.List; @@ -17,12 +20,14 @@ public static class Entry { @NotEmpty @NotNull @JsonProperty("from_key") - private String fromKey; + @EventKeyConfiguration({EventKeyFactory.EventAction.GET, EventKeyFactory.EventAction.DELETE}) + private EventKey fromKey; @NotEmpty @NotNull @JsonProperty("to_key") - private String toKey; + @EventKeyConfiguration(EventKeyFactory.EventAction.PUT) + private EventKey toKey; @JsonProperty("overwrite_if_to_key_exists") private boolean overwriteIfToKeyExists = false; @@ -30,11 +35,11 @@ public static class Entry { @JsonProperty("rename_when") private String renameWhen; - public String getFromKey() { + public EventKey getFromKey() { return fromKey; } - public String getToKey() { + public EventKey getToKey() { return toKey; } @@ -44,7 +49,7 @@ public boolean getOverwriteIfToKeyExists() { public String getRenameWhen() { return renameWhen; } - public Entry(final String fromKey, final String toKey, final boolean overwriteIfKeyExists, final String renameWhen) { + public Entry(final EventKey fromKey, final EventKey toKey, final boolean overwriteIfKeyExists, final String renameWhen) { this.fromKey = fromKey; this.toKey = toKey; this.overwriteIfToKeyExists = overwriteIfKeyExists; diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorTests.java index 2394a5d958..bc0fb78870 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorTests.java @@ -5,15 +5,17 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +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.event.TestEventKeyFactory; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; import java.util.Collections; import java.util.HashMap; @@ -36,9 +38,11 @@ public class DeleteEntryProcessorTests { @Mock private ExpressionEvaluator expressionEvaluator; + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); + @Test public void testSingleDeleteProcessorTest() { - when(mockConfig.getWithKeys()).thenReturn(new String[] { "message" }); + when(mockConfig.getWithKeys()).thenReturn(List.of(eventKeyFactory.createEventKey("message", EventKeyFactory.EventAction.DELETE))); when(mockConfig.getDeleteWhen()).thenReturn(null); final DeleteEntryProcessor processor = createObjectUnderTest(); @@ -52,7 +56,7 @@ public void testSingleDeleteProcessorTest() { @Test public void testWithKeyDneDeleteProcessorTest() { - when(mockConfig.getWithKeys()).thenReturn(new String[] { "message2" }); + when(mockConfig.getWithKeys()).thenReturn(List.of(eventKeyFactory.createEventKey("message2", EventKeyFactory.EventAction.DELETE))); when(mockConfig.getDeleteWhen()).thenReturn(null); final DeleteEntryProcessor processor = createObjectUnderTest(); @@ -67,7 +71,9 @@ public void testWithKeyDneDeleteProcessorTest() { @Test public void testMultiDeleteProcessorTest() { - when(mockConfig.getWithKeys()).thenReturn(new String[] { "message", "message2" }); + when(mockConfig.getWithKeys()).thenReturn(List.of( + eventKeyFactory.createEventKey("message", EventKeyFactory.EventAction.DELETE), + eventKeyFactory.createEventKey("message2", EventKeyFactory.EventAction.DELETE))); when(mockConfig.getDeleteWhen()).thenReturn(null); final DeleteEntryProcessor processor = createObjectUnderTest(); @@ -83,7 +89,7 @@ public void testMultiDeleteProcessorTest() { @Test public void testKeyIsNotDeleted_when_deleteWhen_returns_false() { - when(mockConfig.getWithKeys()).thenReturn(new String[] { "message" }); + when(mockConfig.getWithKeys()).thenReturn(List.of(eventKeyFactory.createEventKey("message", EventKeyFactory.EventAction.DELETE))); final String deleteWhen = UUID.randomUUID().toString(); when(mockConfig.getDeleteWhen()).thenReturn(deleteWhen); @@ -98,8 +104,9 @@ public void testKeyIsNotDeleted_when_deleteWhen_returns_false() { assertThat(editedRecords.get(0).getData().containsKey("newMessage"), is(true)); } + @Test public void testNestedDeleteProcessorTest() { - when(mockConfig.getWithKeys()).thenReturn(new String[]{"nested/foo"}); + when(mockConfig.getWithKeys()).thenReturn(List.of(eventKeyFactory.createEventKey("nested/foo", EventKeyFactory.EventAction.DELETE))); Map nested = Map.of("foo", "bar", "fizz", 42); diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorTests.java index dfc5a7b595..6ae362bc46 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorTests.java @@ -5,9 +5,12 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +import org.opensearch.dataprepper.event.TestEventKeyFactory; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.Test; @@ -39,6 +42,8 @@ public class RenameKeyProcessorTests { @Mock private ExpressionEvaluator expressionEvaluator; + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); + @Test public void testSingleOverwriteRenameProcessorTests() { when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("message", "newMessage", true, null))); @@ -136,7 +141,9 @@ private RenameKeyProcessor createObjectUnderTest() { } private RenameKeyProcessorConfig.Entry createEntry(final String fromKey, final String toKey, final boolean overwriteIfToKeyExists, final String renameWhen) { - return new RenameKeyProcessorConfig.Entry(fromKey, toKey, overwriteIfToKeyExists, renameWhen); + final EventKey fromEventKey = eventKeyFactory.createEventKey(fromKey); + final EventKey toEventKey = eventKeyFactory.createEventKey(toKey); + return new RenameKeyProcessorConfig.Entry(fromEventKey, toEventKey, overwriteIfToKeyExists, renameWhen); } private List createListOfEntries(final RenameKeyProcessorConfig.Entry... entries) { From 5420162b888b03a8a912d1421e4e3bc56be7f582 Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 11 Jul 2024 10:07:32 -0500 Subject: [PATCH 061/159] Updates to the AWS Lambda Sink tests to fix a flaky test. Also adds SLF4J logging for these tests. (#4723) Signed-off-by: David Venable --- data-prepper-plugins/lambda/build.gradle | 1 + .../plugins/lambda/sink/LambdaSinkServiceTest.java | 2 +- .../lambda/src/test/resources/simplelogger.properties | 8 ++++++++ 3 files changed, 10 insertions(+), 1 deletion(-) create mode 100644 data-prepper-plugins/lambda/src/test/resources/simplelogger.properties diff --git a/data-prepper-plugins/lambda/build.gradle b/data-prepper-plugins/lambda/build.gradle index d0c09c9c8b..8447c3abdf 100644 --- a/data-prepper-plugins/lambda/build.gradle +++ b/data-prepper-plugins/lambda/build.gradle @@ -27,6 +27,7 @@ dependencies { testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-plugins:parse-json-processor') + testImplementation testLibs.slf4j.simple } test { diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java index 4e678c191d..f8ca0f11ec 100644 --- a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java +++ b/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java @@ -294,7 +294,7 @@ public void lambda_sink_test_batch_enabled() throws IOException { when(lambdaSinkConfig.getBatchOptions()).thenReturn(mock(BatchOptions.class)); when(lambdaSinkConfig.getBatchOptions().getBatchKey()).thenReturn(batchKey); when(lambdaSinkConfig.getBatchOptions().getThresholdOptions()).thenReturn(mock(ThresholdOptions.class)); - when(lambdaSinkConfig.getBatchOptions().getThresholdOptions().getEventCount()).thenReturn(maxEvents); + when(lambdaSinkConfig.getBatchOptions().getThresholdOptions().getEventCount()).thenReturn(1); when(lambdaSinkConfig.getBatchOptions().getThresholdOptions().getMaximumSize()).thenReturn(ByteCount.parse(maxSize)); when(lambdaSinkConfig.getBatchOptions().getThresholdOptions().getEventCollectTimeOut()).thenReturn(Duration.ofNanos(10L)); when(lambdaSinkConfig.getAwsAuthenticationOptions()).thenReturn(mock(AwsAuthenticationOptions.class)); diff --git a/data-prepper-plugins/lambda/src/test/resources/simplelogger.properties b/data-prepper-plugins/lambda/src/test/resources/simplelogger.properties new file mode 100644 index 0000000000..f464558cf4 --- /dev/null +++ b/data-prepper-plugins/lambda/src/test/resources/simplelogger.properties @@ -0,0 +1,8 @@ +# +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +# + +org.slf4j.simpleLogger.showDateTime=true +org.slf4j.simpleLogger.dateTimeFormat=yyyy-MM-dd' 'HH:mm:ss.SSS +org.slf4j.simpleLogger.log.org.opensearch.dataprepper.plugins.lambda.sink=trace From 8a1a41626b59b45debd77e7acecf2e6b8dec7c29 Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 11 Jul 2024 13:17:14 -0500 Subject: [PATCH 062/159] Mockito 5 (#4712) Mockito 5 * Synchronize the MetricsTestUtil methods to avoid test failures. * Create a copy of the collections to remove in MetricsTestUtil. * Updated two tests to JUnit 5 and to use mocks instead of actual metrics. Updates to MetricsTestUtil to provide clarity on NPEs. Signed-off-by: David Venable --- .../dataprepper/metrics/MetricsTestUtil.java | 22 +++++-- data-prepper-core/build.gradle | 1 - .../discovery/DnsPeerListProviderTest.java | 66 +++++++++++-------- .../discovery/StaticPeerListProviderTest.java | 53 ++++++++------- .../pipeline/PipelineConnectorTest.java | 2 +- .../pipeline/common/FutureHelperTest.java | 2 +- .../CloudWatchMeterRegistryProviderTest.java | 2 +- .../org.mockito.plugins.MockMaker | 3 - .../org.mockito.plugins.MockMaker | 1 - .../build.gradle | 1 - data-prepper-pipeline-parser/build.gradle | 5 -- .../parser/EventKeyDeserializerTest.java | 4 +- data-prepper-plugin-framework/build.gradle | 1 - .../aggregate-processor/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 1 - .../org.mockito.plugins.MockMaker | 3 - .../cloudwatch-logs/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 3 - data-prepper-plugins/common/build.gradle | 1 - .../decompress-processor/build.gradle | 1 - .../build.gradle | 1 - .../dynamodb-source/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 3 - .../grok-processor/build.gradle | 1 - data-prepper-plugins/http-common/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 3 - .../org.mockito.plugins.MockMaker | 3 - .../org.mockito.plugins.MockMaker | 3 - .../kafka-plugins/build.gradle | 3 - .../org.mockito.plugins.MockMaker | 3 - data-prepper-plugins/mongodb/build.gradle | 1 - data-prepper-plugins/opensearch/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 3 - .../otel-logs-source/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 3 - .../otel-metrics-raw-processor/build.gradle | 1 - .../otel-metrics-source/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 3 - .../org.mockito.plugins.MockMaker | 3 - .../otel-trace-raw-processor/build.gradle | 1 - .../otel-trace-source/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 3 - .../org.mockito.plugins.MockMaker | 3 - data-prepper-plugins/rds-source/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 3 - .../org.mockito.plugins.MockMaker | 3 - .../service-map-stateful/build.gradle | 1 - .../org.mockito.plugins.MockMaker | 3 - .../org.mockito.plugins.MockMaker | 3 - settings.gradle | 2 +- 50 files changed, 89 insertions(+), 148 deletions(-) delete mode 100644 data-prepper-core/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-expression/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/armeria-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/aws-plugin/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/cloudwatch-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/geoip-processor/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/http-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/http-source-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/http-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/opensearch/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/otel-logs-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/otel-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/otel-trace-group-processor/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/otel-trace-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/prometheus-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/s3-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/s3-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/sns-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker delete mode 100644 data-prepper-plugins/sqs-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/MetricsTestUtil.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/MetricsTestUtil.java index a77d9de349..f6c0602f9e 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/MetricsTestUtil.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/metrics/MetricsTestUtil.java @@ -6,25 +6,37 @@ package org.opensearch.dataprepper.metrics; import io.micrometer.core.instrument.Measurement; +import io.micrometer.core.instrument.Meter; import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.Metrics; import io.micrometer.core.instrument.Statistic; import io.micrometer.core.instrument.simple.SimpleMeterRegistry; +import java.util.ArrayList; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.StreamSupport; public class MetricsTestUtil { - public static void initMetrics() { - Metrics.globalRegistry.getRegistries().forEach(meterRegistry -> Metrics.globalRegistry.remove(meterRegistry)); - Metrics.globalRegistry.getMeters().forEach(meter -> Metrics.globalRegistry.remove(meter)); + public static synchronized void initMetrics() { + final Set registries = new HashSet<>(Metrics.globalRegistry.getRegistries()); + registries.forEach(Metrics.globalRegistry::remove); + + final List meters = new ArrayList<>(Metrics.globalRegistry.getMeters()); + meters.forEach(Metrics.globalRegistry::remove); + Metrics.addRegistry(new SimpleMeterRegistry()); } - public static List getMeasurementList(final String meterName) { - return StreamSupport.stream(getRegistry().find(meterName).meter().measure().spliterator(), false) + public static synchronized List getMeasurementList(final String meterName) { + final Meter meter = getRegistry().find(meterName).meter(); + if(meter == null) + throw new RuntimeException("No metrics meter is available for " + meterName); + + return StreamSupport.stream(meter.measure().spliterator(), false) .collect(Collectors.toList()); } diff --git a/data-prepper-core/build.gradle b/data-prepper-core/build.gradle index 429e07069c..080538c5e4 100644 --- a/data-prepper-core/build.gradle +++ b/data-prepper-core/build.gradle @@ -60,7 +60,6 @@ dependencies { implementation 'software.amazon.awssdk:servicediscovery' implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' testImplementation testLibs.junit.vintage - testImplementation testLibs.mockito.inline testImplementation libs.commons.lang3 testImplementation project(':data-prepper-test-event') testImplementation project(':data-prepper-test-common') diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/DnsPeerListProviderTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/DnsPeerListProviderTest.java index 1083eea9f0..3bdee15368 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/DnsPeerListProviderTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/DnsPeerListProviderTest.java @@ -7,30 +7,33 @@ import com.linecorp.armeria.client.Endpoint; import com.linecorp.armeria.client.endpoint.dns.DnsAddressEndpointGroup; -import io.micrometer.core.instrument.Measurement; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; -import org.opensearch.dataprepper.metrics.MetricNames; -import org.opensearch.dataprepper.metrics.MetricsTestUtil; +import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.peerforwarder.HashRing; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.StringJoiner; import java.util.concurrent.CompletableFuture; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import java.util.function.ToDoubleFunction; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +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.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.peerforwarder.discovery.PeerListProvider.PEER_ENDPOINTS; -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(MockitoExtension.class) public class DnsPeerListProviderTest { private static final String ENDPOINT_1 = "10.1.1.1"; @@ -39,8 +42,6 @@ public class DnsPeerListProviderTest { Endpoint.of(ENDPOINT_1), Endpoint.of(ENDPOINT_2) ); - private static final String COMPONENT_SCOPE = "testComponentScope"; - private static final String COMPONENT_ID = "testComponentId"; @Mock private DnsAddressEndpointGroup dnsAddressEndpointGroup; @@ -48,34 +49,33 @@ public class DnsPeerListProviderTest { @Mock private HashRing hashRing; + @Mock private PluginMetrics pluginMetrics; private CompletableFuture completableFuture; private DnsPeerListProvider dnsPeerListProvider; - @Before + @BeforeEach public void setup() { - MetricsTestUtil.initMetrics(); completableFuture = CompletableFuture.completedFuture(null); when(dnsAddressEndpointGroup.whenReady()).thenReturn(completableFuture); - pluginMetrics = PluginMetrics.fromNames(COMPONENT_ID, COMPONENT_SCOPE); dnsPeerListProvider = new DnsPeerListProvider(dnsAddressEndpointGroup, pluginMetrics); } - @Test(expected = NullPointerException.class) + @Test public void testDefaultListProviderWithNullHostname() { - new DnsPeerListProvider(null, pluginMetrics); + assertThrows(NullPointerException.class, () -> new DnsPeerListProvider(null, pluginMetrics)); } - @Test(expected = RuntimeException.class) + @Test public void testConstructWithInterruptedException() throws Exception { CompletableFuture mockFuture = mock(CompletableFuture.class); when(mockFuture.get()).thenThrow(new InterruptedException()); when(dnsAddressEndpointGroup.whenReady()).thenReturn(mockFuture); - new DnsPeerListProvider(dnsAddressEndpointGroup, pluginMetrics); + assertThrows(RuntimeException.class, () -> new DnsPeerListProvider(dnsAddressEndpointGroup, pluginMetrics)); } @Test @@ -90,17 +90,27 @@ public void testGetPeerList() { } @Test - public void testActivePeerCounter() { + public void testActivePeerCounter_with_list() { when(dnsAddressEndpointGroup.endpoints()).thenReturn(ENDPOINT_LIST); - final List endpointsMeasures = MetricsTestUtil.getMeasurementList(new StringJoiner(MetricNames.DELIMITER).add(COMPONENT_SCOPE).add(COMPONENT_ID) - .add(PeerListProvider.PEER_ENDPOINTS).toString()); - assertEquals(1, endpointsMeasures.size()); - final Measurement endpointsMeasure = endpointsMeasures.get(0); - assertEquals(2.0, endpointsMeasure.getValue(), 0); + final ArgumentCaptor> gaugeFunctionCaptor = ArgumentCaptor.forClass(ToDoubleFunction.class); + verify(pluginMetrics).gauge(eq(PEER_ENDPOINTS), eq(dnsAddressEndpointGroup), gaugeFunctionCaptor.capture()); + + final ToDoubleFunction gaugeFunction = gaugeFunctionCaptor.getValue(); + assertThat(gaugeFunction.applyAsDouble(dnsAddressEndpointGroup), equalTo(2.0)); + } + + @Test + public void testActivePeerCounter_with_single() { when(dnsAddressEndpointGroup.endpoints()).thenReturn(Collections.singletonList(Endpoint.of(ENDPOINT_1))); - assertEquals(1.0, endpointsMeasure.getValue(), 0); + + final ArgumentCaptor> gaugeFunctionCaptor = ArgumentCaptor.forClass(ToDoubleFunction.class); + verify(pluginMetrics).gauge(eq(PEER_ENDPOINTS), eq(dnsAddressEndpointGroup), gaugeFunctionCaptor.capture()); + + final ToDoubleFunction gaugeFunction = gaugeFunctionCaptor.getValue(); + + assertThat(gaugeFunction.applyAsDouble(dnsAddressEndpointGroup), equalTo(1.0)); } @Test diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/StaticPeerListProviderTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/StaticPeerListProviderTest.java index 14bc836e36..589329b108 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/StaticPeerListProviderTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/peerforwarder/discovery/StaticPeerListProviderTest.java @@ -5,56 +5,58 @@ package org.opensearch.dataprepper.peerforwarder.discovery; -import io.micrometer.core.instrument.Measurement; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; import org.mockito.Mock; -import org.mockito.junit.MockitoJUnitRunner; -import org.opensearch.dataprepper.metrics.MetricNames; -import org.opensearch.dataprepper.metrics.MetricsTestUtil; +import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.peerforwarder.HashRing; import java.util.Arrays; import java.util.Collections; import java.util.List; -import java.util.StringJoiner; - -import static org.junit.Assert.assertEquals; +import java.util.function.ToDoubleFunction; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; +import static org.opensearch.dataprepper.peerforwarder.discovery.PeerListProvider.PEER_ENDPOINTS; -@RunWith(MockitoJUnitRunner.class) +@ExtendWith(MockitoExtension.class) public class StaticPeerListProviderTest { private static final String ENDPOINT_1 = "10.10.0.1"; private static final String ENDPOINT_2 = "10.10.0.2"; private static final List ENDPOINT_LIST = Arrays.asList(ENDPOINT_1, ENDPOINT_2); - private static final String COMPONENT_SCOPE = "testComponentScope"; - private static final String COMPONENT_ID = "testComponentId"; @Mock private HashRing hashRing; + @Mock private PluginMetrics pluginMetrics; private StaticPeerListProvider staticPeerListProvider; - @Before + @BeforeEach public void setup() { - MetricsTestUtil.initMetrics(); - pluginMetrics = PluginMetrics.fromNames(COMPONENT_ID, COMPONENT_SCOPE); staticPeerListProvider = new StaticPeerListProvider(ENDPOINT_LIST, pluginMetrics); } - @Test(expected = RuntimeException.class) + @Test public void testListProviderWithEmptyList() { - new StaticPeerListProvider(Collections.emptyList(), pluginMetrics); + assertThrows(RuntimeException.class, () -> new StaticPeerListProvider(Collections.emptyList(), pluginMetrics)); } - @Test(expected = RuntimeException.class) + @Test public void testListProviderWithNullList() { - new StaticPeerListProvider(null, pluginMetrics); + assertThrows(RuntimeException.class, () -> new StaticPeerListProvider(null, pluginMetrics)); } @Test @@ -65,11 +67,12 @@ public void testListProviderWithNonEmptyList() { @Test public void testActivePeerCounter() { - final List endpointsMeasures = MetricsTestUtil.getMeasurementList( - new StringJoiner(MetricNames.DELIMITER).add(COMPONENT_SCOPE).add(COMPONENT_ID).add(PeerListProvider.PEER_ENDPOINTS).toString()); - assertEquals(1, endpointsMeasures.size()); - final Measurement endpointsMeasure = endpointsMeasures.get(0); - assertEquals(2.0, endpointsMeasure.getValue(), 0); + final ArgumentCaptor>> gaugeFunctionCaptor = ArgumentCaptor.forClass(ToDoubleFunction.class); + verify(pluginMetrics).gauge(eq(PEER_ENDPOINTS), any(List.class), gaugeFunctionCaptor.capture()); + + final ToDoubleFunction> gaugeFunction = gaugeFunctionCaptor.getValue(); + + assertThat(gaugeFunction.applyAsDouble(ENDPOINT_LIST), equalTo(2.0)); } @Test diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineConnectorTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineConnectorTest.java index fb54d532b7..e2af218c25 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineConnectorTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/PipelineConnectorTest.java @@ -23,7 +23,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.junit.MockitoJUnitRunner; import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBuffer; diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/common/FutureHelperTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/common/FutureHelperTest.java index c572766ac2..ba8a9714de 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/common/FutureHelperTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/common/FutureHelperTest.java @@ -9,7 +9,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.junit.MockitoJUnitRunner; import java.util.Arrays; import java.util.concurrent.ExecutionException; diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/CloudWatchMeterRegistryProviderTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/CloudWatchMeterRegistryProviderTest.java index 53db40d1a6..9dc744981b 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/CloudWatchMeterRegistryProviderTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/CloudWatchMeterRegistryProviderTest.java @@ -9,7 +9,7 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; -import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.junit.MockitoJUnitRunner; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import static org.hamcrest.CoreMatchers.notNullValue; diff --git a/data-prepper-core/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-core/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-core/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-expression/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-expression/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 1f0955d450..0000000000 --- a/data-prepper-expression/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1 +0,0 @@ -mock-maker-inline diff --git a/data-prepper-logstash-configuration/build.gradle b/data-prepper-logstash-configuration/build.gradle index 6e328b7adc..002ae15516 100644 --- a/data-prepper-logstash-configuration/build.gradle +++ b/data-prepper-logstash-configuration/build.gradle @@ -25,7 +25,6 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation libs.commons.lang3 testImplementation testLibs.slf4j.simple - testImplementation testLibs.mockito.inline } generateGrammarSource { diff --git a/data-prepper-pipeline-parser/build.gradle b/data-prepper-pipeline-parser/build.gradle index 53b27d1e99..a94f63fc1d 100644 --- a/data-prepper-pipeline-parser/build.gradle +++ b/data-prepper-pipeline-parser/build.gradle @@ -30,12 +30,7 @@ dependencies { testImplementation testLibs.bundles.junit testImplementation testLibs.bundles.mockito testImplementation testLibs.hamcrest - testImplementation 'org.powermock:powermock-module-junit4:2.0.9' - testImplementation 'org.powermock:powermock-api-mockito2:2.0.9' testImplementation 'org.assertj:assertj-core:3.20.2' - testImplementation 'junit:junit:4.13.2' - testImplementation 'org.powermock:powermock-module-junit4:2.0.9' - testImplementation 'org.powermock:powermock-api-mockito2:2.0.9' compileOnly 'org.projectlombok:lombok:1.18.20' annotationProcessor 'org.projectlombok:lombok:1.18.20' } \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java index c727f0529a..240c14dd37 100644 --- a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/EventKeyDeserializerTest.java @@ -30,8 +30,8 @@ import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.sameInstance; import static org.hamcrest.MatcherAssert.assertThat; -import static org.powermock.api.mockito.PowerMockito.mock; -import static org.powermock.api.mockito.PowerMockito.when; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) class EventKeyDeserializerTest { diff --git a/data-prepper-plugin-framework/build.gradle b/data-prepper-plugin-framework/build.gradle index f77212a6b2..14f03fe15d 100644 --- a/data-prepper-plugin-framework/build.gradle +++ b/data-prepper-plugin-framework/build.gradle @@ -24,5 +24,4 @@ dependencies { } implementation libs.reflections.core implementation 'com.fasterxml.jackson.core:jackson-databind' - testImplementation testLibs.mockito.inline } \ No newline at end of file diff --git a/data-prepper-plugins/aggregate-processor/build.gradle b/data-prepper-plugins/aggregate-processor/build.gradle index 744986e924..9a3eb4551a 100644 --- a/data-prepper-plugins/aggregate-processor/build.gradle +++ b/data-prepper-plugins/aggregate-processor/build.gradle @@ -19,7 +19,6 @@ dependencies { implementation libs.opentelemetry.proto implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'io.micrometer:micrometer-core' - testImplementation testLibs.mockito.inline } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/armeria-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/armeria-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index ca6ee9cea8..0000000000 --- a/data-prepper-plugins/armeria-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1 +0,0 @@ -mock-maker-inline \ No newline at end of file diff --git a/data-prepper-plugins/aws-plugin/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/aws-plugin/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/aws-plugin/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle index dc374997f0..3bbb24f443 100644 --- a/data-prepper-plugins/cloudwatch-logs/build.gradle +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -16,7 +16,6 @@ dependencies { implementation 'org.projectlombok:lombok:1.18.26' implementation 'org.hibernate.validator:hibernate-validator:8.0.0.Final' testImplementation project(path: ':data-prepper-test-common') - testImplementation testLibs.mockito.inline compileOnly 'org.projectlombok:lombok:1.18.24' annotationProcessor 'org.projectlombok:lombok:1.18.24' } diff --git a/data-prepper-plugins/cloudwatch-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/cloudwatch-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/cloudwatch-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-plugins/common/build.gradle b/data-prepper-plugins/common/build.gradle index aec7d7bddc..cdfdeab9ef 100644 --- a/data-prepper-plugins/common/build.gradle +++ b/data-prepper-plugins/common/build.gradle @@ -24,7 +24,6 @@ dependencies { testImplementation project(':data-prepper-plugins:blocking-buffer') testImplementation project(':data-prepper-test-event') testImplementation libs.commons.io - testImplementation testLibs.mockito.inline } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/decompress-processor/build.gradle b/data-prepper-plugins/decompress-processor/build.gradle index 9d67cffc3b..1068830a59 100644 --- a/data-prepper-plugins/decompress-processor/build.gradle +++ b/data-prepper-plugins/decompress-processor/build.gradle @@ -9,5 +9,4 @@ dependencies { implementation project(':data-prepper-plugins:common') implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'io.micrometer:micrometer-core' - testImplementation testLibs.mockito.inline } \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source-coordination-store/build.gradle b/data-prepper-plugins/dynamodb-source-coordination-store/build.gradle index 4b9fb2a8f4..1912c2ae9b 100644 --- a/data-prepper-plugins/dynamodb-source-coordination-store/build.gradle +++ b/data-prepper-plugins/dynamodb-source-coordination-store/build.gradle @@ -10,7 +10,6 @@ dependencies { implementation 'software.amazon.awssdk:dynamodb' implementation 'software.amazon.awssdk:dynamodb-enhanced' implementation 'software.amazon.awssdk:sts' - testImplementation testLibs.mockito.inline } test { diff --git a/data-prepper-plugins/dynamodb-source/build.gradle b/data-prepper-plugins/dynamodb-source/build.gradle index 8fdc037470..3b3046434a 100644 --- a/data-prepper-plugins/dynamodb-source/build.gradle +++ b/data-prepper-plugins/dynamodb-source/build.gradle @@ -25,6 +25,5 @@ dependencies { implementation project(path: ':data-prepper-plugins:buffer-common') - testImplementation testLibs.mockito.inline testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' } \ No newline at end of file diff --git a/data-prepper-plugins/geoip-processor/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/geoip-processor/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/geoip-processor/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-plugins/grok-processor/build.gradle b/data-prepper-plugins/grok-processor/build.gradle index 82a8306a5d..ae4a82a0ee 100644 --- a/data-prepper-plugins/grok-processor/build.gradle +++ b/data-prepper-plugins/grok-processor/build.gradle @@ -12,7 +12,6 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation "io.krakens:java-grok:0.1.9" implementation 'io.micrometer:micrometer-core' - testImplementation testLibs.mockito.inline testImplementation project(':data-prepper-test-common') } diff --git a/data-prepper-plugins/http-common/build.gradle b/data-prepper-plugins/http-common/build.gradle index fa0e1c3efb..54fa5d346d 100644 --- a/data-prepper-plugins/http-common/build.gradle +++ b/data-prepper-plugins/http-common/build.gradle @@ -6,7 +6,6 @@ dependencies { implementation 'org.apache.httpcomponents:httpcore:4.4.16' testImplementation testLibs.bundles.junit - testImplementation testLibs.mockito.inline } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/http-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/http-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/http-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-plugins/http-source-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/http-source-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 78ccc25012..0000000000 --- a/data-prepper-plugins/http-source-common/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline \ No newline at end of file diff --git a/data-prepper-plugins/http-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/http-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 78ccc25012..0000000000 --- a/data-prepper-plugins/http-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline \ No newline at end of file diff --git a/data-prepper-plugins/kafka-plugins/build.gradle b/data-prepper-plugins/kafka-plugins/build.gradle index 0032bed806..046aef949a 100644 --- a/data-prepper-plugins/kafka-plugins/build.gradle +++ b/data-prepper-plugins/kafka-plugins/build.gradle @@ -53,7 +53,6 @@ dependencies { implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:apache-client' - testImplementation testLibs.mockito.inline testImplementation 'org.yaml:snakeyaml:2.2' testImplementation testLibs.spring.test testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' @@ -62,12 +61,10 @@ dependencies { testImplementation project(':data-prepper-core') testImplementation project(':data-prepper-plugin-framework') testImplementation project(':data-prepper-pipeline-parser') - testImplementation testLibs.mockito.inline testImplementation 'org.apache.kafka:kafka_2.13:3.6.1' testImplementation 'org.apache.kafka:kafka_2.13:3.6.1:test' testImplementation 'org.apache.curator:curator-test:5.5.0' testImplementation('com.kjetland:mbknor-jackson-jsonschema_2.13:1.0.39') - testImplementation group: 'org.powermock', name: 'powermock-api-mockito2', version: '2.0.9' testImplementation project(':data-prepper-plugins:otel-metrics-source') testImplementation project(':data-prepper-plugins:otel-proto-common') testImplementation libs.opentelemetry.proto diff --git a/data-prepper-plugins/lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-plugins/mongodb/build.gradle b/data-prepper-plugins/mongodb/build.gradle index ae4a5a9d45..c5495880e6 100644 --- a/data-prepper-plugins/mongodb/build.gradle +++ b/data-prepper-plugins/mongodb/build.gradle @@ -16,7 +16,6 @@ dependencies { implementation project(path: ':data-prepper-plugins:common') - testImplementation testLibs.mockito.inline testImplementation testLibs.bundles.junit testImplementation testLibs.slf4j.simple testImplementation project(path: ':data-prepper-test-common') diff --git a/data-prepper-plugins/opensearch/build.gradle b/data-prepper-plugins/opensearch/build.gradle index bece32eaae..5e7879d8d1 100644 --- a/data-prepper-plugins/opensearch/build.gradle +++ b/data-prepper-plugins/opensearch/build.gradle @@ -44,7 +44,6 @@ dependencies { testImplementation 'net.bytebuddy:byte-buddy:1.14.17' testImplementation 'net.bytebuddy:byte-buddy-agent:1.14.17' testImplementation testLibs.slf4j.simple - testImplementation testLibs.mockito.inline } sourceSets { diff --git a/data-prepper-plugins/opensearch/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/opensearch/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 78ccc25012..0000000000 --- a/data-prepper-plugins/opensearch/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline \ No newline at end of file diff --git a/data-prepper-plugins/otel-logs-source/build.gradle b/data-prepper-plugins/otel-logs-source/build.gradle index 97901da8c3..822e945ba9 100644 --- a/data-prepper-plugins/otel-logs-source/build.gradle +++ b/data-prepper-plugins/otel-logs-source/build.gradle @@ -31,7 +31,6 @@ dependencies { implementation libs.bouncycastle.bcprov implementation libs.bouncycastle.bcpkix testImplementation 'org.assertj:assertj-core:3.25.3' - testImplementation testLibs.mockito.inline testImplementation libs.commons.io } diff --git a/data-prepper-plugins/otel-logs-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/otel-logs-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 78ccc25012..0000000000 --- a/data-prepper-plugins/otel-logs-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline \ No newline at end of file diff --git a/data-prepper-plugins/otel-metrics-raw-processor/build.gradle b/data-prepper-plugins/otel-metrics-raw-processor/build.gradle index af20b2e74b..a4316fca16 100644 --- a/data-prepper-plugins/otel-metrics-raw-processor/build.gradle +++ b/data-prepper-plugins/otel-metrics-raw-processor/build.gradle @@ -22,7 +22,6 @@ dependencies { implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation libs.guava.core testImplementation 'org.assertj:assertj-core:3.25.3' - testImplementation testLibs.mockito.inline } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/otel-metrics-source/build.gradle b/data-prepper-plugins/otel-metrics-source/build.gradle index 25ea578566..96d250d67d 100644 --- a/data-prepper-plugins/otel-metrics-source/build.gradle +++ b/data-prepper-plugins/otel-metrics-source/build.gradle @@ -31,7 +31,6 @@ dependencies { implementation libs.bouncycastle.bcprov implementation libs.bouncycastle.bcpkix testImplementation 'org.assertj:assertj-core:3.25.3' - testImplementation testLibs.mockito.inline testImplementation libs.commons.io } diff --git a/data-prepper-plugins/otel-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/otel-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 78ccc25012..0000000000 --- a/data-prepper-plugins/otel-metrics-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline \ No newline at end of file diff --git a/data-prepper-plugins/otel-trace-group-processor/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/otel-trace-group-processor/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 78ccc25012..0000000000 --- a/data-prepper-plugins/otel-trace-group-processor/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline \ No newline at end of file diff --git a/data-prepper-plugins/otel-trace-raw-processor/build.gradle b/data-prepper-plugins/otel-trace-raw-processor/build.gradle index ff2bfc4a60..2df90630d8 100644 --- a/data-prepper-plugins/otel-trace-raw-processor/build.gradle +++ b/data-prepper-plugins/otel-trace-raw-processor/build.gradle @@ -20,7 +20,6 @@ dependencies { implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' implementation libs.caffeine testImplementation 'org.assertj:assertj-core:3.25.3' - testImplementation testLibs.mockito.inline } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/otel-trace-source/build.gradle b/data-prepper-plugins/otel-trace-source/build.gradle index 39c0869851..d1dcdfa12a 100644 --- a/data-prepper-plugins/otel-trace-source/build.gradle +++ b/data-prepper-plugins/otel-trace-source/build.gradle @@ -29,7 +29,6 @@ dependencies { implementation libs.bouncycastle.bcprov implementation libs.bouncycastle.bcpkix testImplementation 'org.assertj:assertj-core:3.25.3' - testImplementation testLibs.mockito.inline testImplementation testLibs.slf4j.simple testImplementation libs.commons.io } diff --git a/data-prepper-plugins/otel-trace-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/otel-trace-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 78ccc25012..0000000000 --- a/data-prepper-plugins/otel-trace-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/prometheus-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/prometheus-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-plugins/rds-source/build.gradle b/data-prepper-plugins/rds-source/build.gradle index 8372276564..580a312be0 100644 --- a/data-prepper-plugins/rds-source/build.gradle +++ b/data-prepper-plugins/rds-source/build.gradle @@ -20,7 +20,6 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-core' implementation 'com.fasterxml.jackson.core:jackson-databind' - testImplementation testLibs.mockito.inline testImplementation project(path: ':data-prepper-test-common') testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' } diff --git a/data-prepper-plugins/s3-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/s3-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/s3-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-plugins/s3-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/s3-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/s3-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-plugins/service-map-stateful/build.gradle b/data-prepper-plugins/service-map-stateful/build.gradle index 60b9512ed9..fa83d4e6bc 100644 --- a/data-prepper-plugins/service-map-stateful/build.gradle +++ b/data-prepper-plugins/service-map-stateful/build.gradle @@ -19,7 +19,6 @@ dependencies { exclude group: 'com.google.protobuf', module: 'protobuf-java' } implementation libs.protobuf.core - testImplementation testLibs.mockito.inline } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/sns-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/sns-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/sns-sink/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/data-prepper-plugins/sqs-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/sqs-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker deleted file mode 100644 index 23c33feb6d..0000000000 --- a/data-prepper-plugins/sqs-source/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker +++ /dev/null @@ -1,3 +0,0 @@ -# To enable mocking of final classes with vanilla Mockito -# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods -mock-maker-inline diff --git a/settings.gradle b/settings.gradle index ca9fcfbdfb..9d84b2ccf0 100644 --- a/settings.gradle +++ b/settings.gradle @@ -74,7 +74,7 @@ dependencyResolutionManagement { } testLibs { version('junit', '5.8.2') - version('mockito', '3.11.2') + version('mockito', '5.12.0') version('hamcrest', '2.2') version('awaitility', '4.2.0') version('spring', '5.3.28') From 5b1edb6951025bbe9eb19f636d992cdbc74647fa Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 12 Jul 2024 12:40:24 -0500 Subject: [PATCH 063/159] MAINT: backfill documentation into json description for string_converter (#4725) * MAINT: backfill documentation into json description for string_converter Signed-off-by: George Chen --- .../dataprepper/plugins/processor/StringProcessor.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java index aa2930e634..3cf2953e06 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.processor; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.configuration.PluginSetting; @@ -40,6 +41,7 @@ public class StringProcessor implements Processor, Record> private final boolean upperCase; public static class Configuration { + @JsonPropertyDescription("Whether to convert to uppercase (`true`) or lowercase (`false`).") private boolean upperCase = true; public boolean getUpperCase() { From f9dc806d55f8f9a40970cf57c9eae40b20f2a8af Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 12 Jul 2024 12:40:52 -0500 Subject: [PATCH 064/159] MAINT: backfill documentation in json description for otel_traces (#4724) * MAINT: backfill documentation in json property description for otel_traces Signed-off-by: George Chen --- .../processor/oteltrace/OtelTraceRawProcessorConfig.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/data-prepper-plugins/otel-trace-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OtelTraceRawProcessorConfig.java b/data-prepper-plugins/otel-trace-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OtelTraceRawProcessorConfig.java index 553e1ed2d1..6b850f7354 100644 --- a/data-prepper-plugins/otel-trace-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OtelTraceRawProcessorConfig.java +++ b/data-prepper-plugins/otel-trace-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OtelTraceRawProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.oteltrace; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import java.time.Duration; @@ -14,12 +15,17 @@ public class OtelTraceRawProcessorConfig { static final Duration DEFAULT_TRACE_ID_TTL = Duration.ofSeconds(15L); static final long MAX_TRACE_ID_CACHE_SIZE = 1_000_000L; @JsonProperty("trace_flush_interval") + @JsonPropertyDescription("Represents the time interval in seconds to flush all the descendant spans without any " + + "root span. Default is 180.") private long traceFlushInterval = DEFAULT_TG_FLUSH_INTERVAL_SEC; @JsonProperty("trace_group_cache_ttl") + @JsonPropertyDescription("Represents the time-to-live to cache a trace group details. Default is 15 seconds.") private Duration traceGroupCacheTimeToLive = DEFAULT_TRACE_ID_TTL; @JsonProperty("trace_group_cache_max_size") + @JsonPropertyDescription("Represents the maximum size of the cache to store the trace group details from root spans. " + + "Default is 1000000.") private long traceGroupCacheMaxSize = MAX_TRACE_ID_CACHE_SIZE; public long getTraceFlushIntervalSeconds() { From 62682da83e76ddb468da5de34e1b1056090d65a7 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 12 Jul 2024 13:29:16 -0500 Subject: [PATCH 065/159] MAINT: backfill documentation into json description for delete_entries (#4721) * MAINT: backfill documentation into json description for delete_entries Signed-off-by: George Chen --- .../processor/mutateevent/DeleteEntryProcessorConfig.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java index a60c2b08bf..b1df976770 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.model.event.EventKey; @@ -19,9 +20,12 @@ public class DeleteEntryProcessorConfig { @NotNull @JsonProperty("with_keys") @EventKeyConfiguration(EventKeyFactory.EventAction.DELETE) + @JsonPropertyDescription("An array of keys for the entries to be deleted.") private List<@NotNull @NotEmpty EventKey> withKeys; @JsonProperty("delete_when") + @JsonPropertyDescription("Specifies under what condition the `delete_entries` processor should perform deletion. " + + "Default is no condition.") private String deleteWhen; public List getWithKeys() { From 59a4df4207459cb9a009cce99e13ecfba6c93d37 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 12 Jul 2024 13:29:41 -0500 Subject: [PATCH 066/159] MAINT: backfill documentation into json property for substitute_string (#4727) * MAINT: backfill documentation into json property for substitute_string Signed-off-by: George Chen --- .../mutatestring/SubstituteStringProcessorConfig.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java index 5813b7cf0b..4a8f53f0fe 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java @@ -6,14 +6,21 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import org.opensearch.dataprepper.model.event.EventKey; import java.util.List; public class SubstituteStringProcessorConfig implements StringProcessorConfig { public static class Entry { + @JsonPropertyDescription("The key to modify.") private EventKey source; + @JsonPropertyDescription("The Regex String to be replaced. Special regex characters such as `[` and `]` must " + + "be escaped using `\\\\` when using double quotes and `\\ ` when using single quotes. " + + "See [Java Patterns](https://docs.oracle.com/en/java/javase/17/docs/api/java.base/java/util/regex/Pattern.html) " + + "for more information.") private String from; + @JsonPropertyDescription("The String to be substituted for each match of `from`.") private String to; @JsonProperty("substitute_when") @@ -43,6 +50,7 @@ public Entry(final EventKey source, final String from, final String to, final St public Entry() {} } + @JsonPropertyDescription("List of entries. Valid values are `source`, `from`, and `to`.") private List entries; public List getEntries() { From 8fac7cfe06edef6c35fb994ffc504b1b4ec9e56e Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 12 Jul 2024 13:30:05 -0500 Subject: [PATCH 067/159] MAINT: backfill documentation into json description for truncate processor (#4726) * MAINT: backfill documentation into json description for truncate processor Signed-off-by: George Chen --- .../processor/truncate/TruncateProcessorConfig.java | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java b/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java index 7fde949719..02c83f5773 100644 --- a/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java +++ b/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.truncate; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.AssertTrue; @@ -16,18 +17,25 @@ public class TruncateProcessorConfig { public static class Entry { @JsonProperty("source_keys") + @JsonPropertyDescription("The list of source keys that will be modified by the processor. " + + "The default value is an empty list, which indicates that all values will be truncated.") private List sourceKeys; @JsonProperty("start_at") + @JsonPropertyDescription("Where in the string value to start truncation. " + + "Default is `0`, which specifies to start truncation at the beginning of each key's value.") private Integer startAt; @JsonProperty("length") + @JsonPropertyDescription("The length of the string after truncation. " + + "When not specified, the processor will measure the length based on where the string ends.") private Integer length; @JsonProperty("recursive") private Boolean recurse = false; @JsonProperty("truncate_when") + @JsonPropertyDescription("A condition that, when met, determines when the truncate operation is performed.") private String truncateWhen; public Entry(final List sourceKeys, final Integer startAt, final Integer length, final String truncateWhen, final Boolean recurse) { @@ -77,6 +85,7 @@ public boolean isValidConfig() { @NotEmpty @NotNull + @JsonPropertyDescription("A list of entries to add to an event.") private List<@Valid Entry> entries; public List getEntries() { From 67f3595805f07442d8f05823c9959b50358aa4d9 Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 12 Jul 2024 17:35:02 -0500 Subject: [PATCH 068/159] Run tests on the current JVM for Java 17 & 21 / Gradle 8.8 (#4730) Run tests on the current JVM rather than always using Java 11 for the tests. This fixes a problem with our current GitHub tests where we are running against only Java 11 even though we want to run against different Java versions (11, 17, 21). Updates the Gradle version to 8.8. Fix Java 21 support in the AbstractSink by removing usage of Thread::stop which now always throws an UnsupportedOperationException. Use only microsecond precision time when comparing the times in the event_json codec. These tests are failing now on Java 17 and 21 with precision errors. Fixed a randomly failing test in BlockingBufferTests where a value 0 caused an IllegalArgumentException. Logging changes to avoid noise in the Gradle builds in GitHub. Signed-off-by: David Venable --- build.gradle | 3 ++ .../dataprepper/model/sink/AbstractSink.java | 6 ++-- .../dataprepper/model/sink/SinkThread.java | 8 ++++- .../model/sink/AbstractSinkTest.java | 22 +++++++----- data-prepper-core/build.gradle | 3 -- .../avro/AvroAutoSchemaGeneratorTest.java | 4 +-- .../blockingbuffer/BlockingBufferTests.java | 2 +- .../event_json/EventJsonInputCodecTest.java | 34 +++++++++++-------- .../EventJsonInputOutputCodecTest.java | 26 ++++++++------ .../event_json/EventJsonOutputCodecTest.java | 10 +++--- gradle/wrapper/gradle-wrapper.properties | 2 +- gradlew | 2 +- 12 files changed, 72 insertions(+), 50 deletions(-) diff --git a/build.gradle b/build.gradle index f4bbccbcc2..7d7c939d34 100644 --- a/build.gradle +++ b/build.gradle @@ -226,6 +226,9 @@ subprojects { test { useJUnitPlatform() + javaLauncher = javaToolchains.launcherFor { + languageVersion = JavaLanguageVersion.current() + } reports { junitXml.required html.required diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java index 1c3e596265..26dd7e98a6 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java @@ -28,6 +28,7 @@ public abstract class AbstractSink> implements Sink { private Thread retryThread; private int maxRetries; private int waitTimeMs; + private SinkThread sinkThread; public AbstractSink(final PluginSetting pluginSetting, int numRetries, int waitTimeMs) { this.pluginMetrics = PluginMetrics.fromPluginSetting(pluginSetting); @@ -51,7 +52,8 @@ public void initialize() { // the exceptions which are not retryable. doInitialize(); if (!isReady() && retryThread == null) { - retryThread = new Thread(new SinkThread(this, maxRetries, waitTimeMs)); + sinkThread = new SinkThread(this, maxRetries, waitTimeMs); + retryThread = new Thread(sinkThread); retryThread.start(); } } @@ -76,7 +78,7 @@ public void output(Collection records) { @Override public void shutdown() { if (retryThread != null) { - retryThread.stop(); + sinkThread.stop(); } } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java index c304de37af..451cef7dff 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java @@ -10,6 +10,8 @@ class SinkThread implements Runnable { private int maxRetries; private int waitTimeMs; + private volatile boolean isStopped = false; + public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) { this.sink = sink; this.maxRetries = maxRetries; @@ -19,11 +21,15 @@ public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) { @Override public void run() { int numRetries = 0; - while (!sink.isReady() && numRetries++ < maxRetries) { + while (!sink.isReady() && numRetries++ < maxRetries && !isStopped) { try { Thread.sleep(waitTimeMs); sink.doInitialize(); } catch (InterruptedException e){} } } + + public void stop() { + isStopped = true; + } } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java index 3b9fe7c007..8d1af7ea44 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java @@ -11,15 +11,10 @@ import org.opensearch.dataprepper.metrics.MetricNames; import org.opensearch.dataprepper.metrics.MetricsTestUtil; import org.opensearch.dataprepper.model.configuration.PluginSetting; -import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.event.EventHandle; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.mock; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; import java.time.Duration; import java.util.Arrays; @@ -30,6 +25,12 @@ import java.util.UUID; import static org.awaitility.Awaitility.await; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; class AbstractSinkTest { private int count; @@ -71,13 +72,13 @@ void testMetrics() { } @Test - void testSinkNotReady() { + void testSinkNotReady() throws InterruptedException { final String sinkName = "testSink"; final String pipelineName = "pipelineName"; MetricsTestUtil.initMetrics(); PluginSetting pluginSetting = new PluginSetting(sinkName, Collections.emptyMap()); pluginSetting.setPipelineName(pipelineName); - AbstractSink> abstractSink = new AbstractSinkNotReadyImpl(pluginSetting); + AbstractSinkNotReadyImpl abstractSink = new AbstractSinkNotReadyImpl(pluginSetting); abstractSink.initialize(); assertEquals(abstractSink.isReady(), false); assertEquals(abstractSink.getRetryThreadState(), Thread.State.RUNNABLE); @@ -87,7 +88,10 @@ void testSinkNotReady() { await().atMost(Duration.ofSeconds(5)) .until(abstractSink::isReady); assertEquals(abstractSink.getRetryThreadState(), Thread.State.TERMINATED); + int initCountBeforeShutdown = abstractSink.initCount; abstractSink.shutdown(); + Thread.sleep(200); + assertThat(abstractSink.initCount, equalTo(initCountBeforeShutdown)); } @Test diff --git a/data-prepper-core/build.gradle b/data-prepper-core/build.gradle index 080538c5e4..c939129a1c 100644 --- a/data-prepper-core/build.gradle +++ b/data-prepper-core/build.gradle @@ -48,7 +48,6 @@ dependencies { exclude group: 'commons-logging', module: 'commons-logging' } implementation 'software.amazon.cloudwatchlogs:aws-embedded-metrics:2.0.0-beta-1' - testImplementation 'org.apache.logging.log4j:log4j-jpl:2.23.0' testImplementation testLibs.spring.test implementation libs.armeria.core implementation libs.armeria.grpc @@ -89,8 +88,6 @@ task integrationTest(type: Test) { classpath = sourceSets.integrationTest.runtimeClasspath - systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties' - filter { includeTestsMatching '*IT' } diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java index 622eb56a1b..1b66b62c37 100644 --- a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java @@ -17,7 +17,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Random; +import java.util.Timer; import java.util.UUID; import java.util.stream.Stream; @@ -218,7 +218,7 @@ static class SomeUnknownTypesArgumentsProvider implements ArgumentsProvider { @Override public Stream provideArguments(ExtensionContext context) { return Stream.of( - arguments(Random.class), + arguments(Timer.class), arguments(InputStream.class), arguments(File.class) ); diff --git a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java index 194c810ec4..f3f28db174 100644 --- a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java +++ b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java @@ -328,7 +328,7 @@ public Stream provideArguments(final ExtensionContext conte return Stream.of( Arguments.of(0, randomInt + 1, 0.0), Arguments.of(1, 100, 1.0), - Arguments.of(randomInt, randomInt, 100.0), + Arguments.of(randomInt + 1, randomInt + 1, 100.0), Arguments.of(randomInt, randomInt + 250, ((double) randomInt / (randomInt + 250)) * 100), Arguments.of(6, 9, 66.66666666666666), Arguments.of(531, 1000, 53.1), diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java index f85d1c6605..a4b0377963 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java @@ -11,9 +11,12 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; + import static org.mockito.Mockito.when; import static org.mockito.Mockito.mock; + import org.mockito.Mock; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.CoreMatchers.not; @@ -28,6 +31,7 @@ import java.io.ByteArrayInputStream; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.List; import java.util.LinkedList; import java.util.Map; @@ -56,7 +60,7 @@ public EventJsonInputCodec createInputCodec() { @ParameterizedTest @ValueSource(strings = {"", "{}"}) public void emptyTest(String input) throws Exception { - input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["+input+"]}"; + input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":[" + input + "]}"; ByteArrayInputStream inputStream = new ByteArrayInputStream(input.getBytes()); inputCodec = createInputCodec(); Consumer> consumer = mock(Consumer.class); @@ -70,15 +74,15 @@ public void inCompatibleVersionTest() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\""+EventJsonDefines.VERSION+"\":\"3.0\", \""+EventJsonDefines.EVENTS+"\":["; + String input = "{\"" + EventJsonDefines.VERSION + "\":\"3.0\", \"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } input += "]}"; @@ -95,15 +99,15 @@ public void basicTest() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["; + String input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } input += "]}"; @@ -111,8 +115,8 @@ public void basicTest() throws Exception { List> records = new LinkedList<>(); inputCodec.parse(inputStream, records::add); assertThat(records.size(), equalTo(2)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -126,15 +130,15 @@ public void test_with_timeReceivedOverridden() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now().minusSeconds(5); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS).minusSeconds(5); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["; + String input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } input += "]}"; @@ -142,8 +146,8 @@ public void test_with_timeReceivedOverridden() throws Exception { List> records = new LinkedList<>(); inputCodec.parse(inputStream, records::add); assertThat(records.size(), equalTo(2)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), not(equalTo(startTime))); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -159,7 +163,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent)logBuilder.build(); + final JacksonEvent event = (JacksonEvent) logBuilder.build(); return event; } diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java index 85e91e5a55..7ea8c49cd0 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java @@ -6,9 +6,12 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; + import static org.mockito.Mockito.when; import static org.mockito.Mockito.mock; + import org.mockito.Mock; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -22,6 +25,7 @@ import org.opensearch.dataprepper.model.log.JacksonLog; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.List; import java.util.LinkedList; import java.util.Map; @@ -64,7 +68,7 @@ public void basicTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); inputCodec = createInputCodec(); @@ -75,8 +79,8 @@ public void basicTest() throws Exception { inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(1)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -90,7 +94,7 @@ public void multipleEventsTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); inputCodec = createInputCodec(); @@ -103,8 +107,8 @@ public void multipleEventsTest() throws Exception { inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(3)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -122,7 +126,7 @@ public void extendedTest() throws Exception { Set tags = Set.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); List tagsList = tags.stream().collect(Collectors.toList()); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); Instant origTime = startTime.minusSeconds(5); event.getMetadata().setExternalOriginationTime(origTime); @@ -135,11 +139,11 @@ public void extendedTest() throws Exception { outputCodec.complete(outputStream); assertThat(outputCodec.getExtension(), equalTo(EventJsonOutputCodec.EVENT_JSON)); List> records = new LinkedList<>(); -inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); + inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(1)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags(), equalTo(tags)); @@ -157,7 +161,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent)logBuilder.build(); + final JacksonEvent event = (JacksonEvent) logBuilder.build(); return event; } diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java index 51dda545cb..b32d2b62e9 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java @@ -11,6 +11,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mock; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -22,6 +23,7 @@ import org.opensearch.dataprepper.model.log.JacksonLog; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.Map; import java.util.UUID; @@ -49,7 +51,7 @@ public void basicTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); outputCodec.start(outputStream, null, null); @@ -59,10 +61,10 @@ public void basicTest() throws Exception { Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); //String expectedOutput = "{\"version\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\",\""+EventJsonDefines.EVENTS+"\":["; - String expectedOutput = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\",\""+EventJsonDefines.EVENTS+"\":["; + String expectedOutput = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\",\"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - expectedOutput += comma+"{\""+EventJsonDefines.DATA+"\":"+objectMapper.writeValueAsString(dataMap)+","+"\""+EventJsonDefines.METADATA+"\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + expectedOutput += comma + "{\"" + EventJsonDefines.DATA + "\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"" + EventJsonDefines.METADATA + "\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } expectedOutput += "]}"; @@ -78,7 +80,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent)logBuilder.build(); + final JacksonEvent event = (JacksonEvent) logBuilder.build(); return event; } diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index b82aa23a4f..a4413138c9 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-8.7-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.8-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 1aa94a4269..b740cf1339 100755 --- a/gradlew +++ b/gradlew @@ -55,7 +55,7 @@ # Darwin, MinGW, and NonStop. # # (3) This script is generated from the Groovy template -# https://github.com/gradle/gradle/blob/HEAD/subprojects/plugins/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt +# https://github.com/gradle/gradle/blob/HEAD/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt # within the Gradle project. # # You can find Gradle at https://github.com/gradle/gradle/. From b1300c3a47eb560731ee575db6413d7c454e7b80 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 12 Jul 2024 22:28:10 -0500 Subject: [PATCH 069/159] MAINT: add json property description into obfuscate processor (#4706) * MAINT: add json property description Signed-off-by: George Chen --- .../obfuscation/ObfuscationProcessorConfig.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java index b99753bc9f..e5893476e0 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.obfuscation; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.expression.ExpressionEvaluator; @@ -17,6 +18,7 @@ public class ObfuscationProcessorConfig { @JsonProperty("source") + @JsonPropertyDescription("The source field to obfuscate.") @NotEmpty @NotNull private String source; @@ -25,18 +27,29 @@ public class ObfuscationProcessorConfig { private List patterns; @JsonProperty("target") + @JsonPropertyDescription("The new field in which to store the obfuscated value. " + + "This leaves the original source field unchanged. " + + "When no `target` is provided, the source field updates with the obfuscated value.") private String target; @JsonProperty("action") + @JsonPropertyDescription("The obfuscation action. As of Data Prepper 2.3, only the `mask` action is supported.") private PluginModel action; @JsonProperty("obfuscate_when") + @JsonPropertyDescription("Specifies under what condition the Obfuscate processor should perform matching. " + + "Default is no condition.") private String obfuscateWhen; @JsonProperty("tags_on_match_failure") + @JsonPropertyDescription("The tag to add to an event if the obfuscate processor fails to match the pattern.") private List tagsOnMatchFailure; @JsonProperty("single_word_only") + @JsonPropertyDescription("When set to `true`, a word boundary `\b` is added to the pattern, " + + "which causes obfuscation to be applied only to words that are standalone in the input text. " + + "By default, it is false, meaning obfuscation patterns are applied to all occurrences. " + + "Can be used for Data Prepper 2.8 or greater.") private boolean singleWordOnly = false; public ObfuscationProcessorConfig() { From a988177974764424f31dd27a3b1f12c435ed77ac Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Sat, 13 Jul 2024 00:30:56 -0500 Subject: [PATCH 070/159] MAINT: backfill doc in json property description for otel_metrics (#4722) * MAINT: backfill doc in json property description for otel_metrics Signed-off-by: George Chen --- .../otelmetrics/OtelMetricsRawProcessorConfig.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OtelMetricsRawProcessorConfig.java b/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OtelMetricsRawProcessorConfig.java index 9935cc9218..b71a0d1800 100644 --- a/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OtelMetricsRawProcessorConfig.java +++ b/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OtelMetricsRawProcessorConfig.java @@ -6,17 +6,23 @@ package org.opensearch.dataprepper.plugins.processor.otelmetrics; import static org.opensearch.dataprepper.plugins.otel.codec.OTelProtoCodec.DEFAULT_EXPONENTIAL_HISTOGRAM_MAX_ALLOWED_SCALE; + import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; public class OtelMetricsRawProcessorConfig { @JsonProperty("flatten_attributes") + @JsonPropertyDescription("Whether or not to flatten the `attributes` field in the JSON data.") boolean flattenAttributesFlag = true; + @JsonPropertyDescription("Whether or not to calculate histogram buckets.") private Boolean calculateHistogramBuckets = true; + @JsonPropertyDescription("Whether or not to calculate exponential histogram buckets.") private Boolean calculateExponentialHistogramBuckets = true; + @JsonPropertyDescription("Maximum allowed scale in exponential histogram calculation.") private Integer exponentialHistogramMaxAllowedScale = DEFAULT_EXPONENTIAL_HISTOGRAM_MAX_ALLOWED_SCALE; public Boolean getCalculateExponentialHistogramBuckets() { From 37b664b18c0844bdf2cff078b3e8948ce448c1d6 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Sat, 13 Jul 2024 00:31:23 -0500 Subject: [PATCH 071/159] MAINT: backfill documentation in JsonPropertyDescription for split_string (#4720) * MAINT: add documentation in JsonPropertyDescription for split_string processor Signed-off-by: George Chen --- .../mutatestring/SplitStringProcessorConfig.java | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java index 25809819f8..cb8edabfb6 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -20,15 +21,23 @@ public static class Entry { @NotEmpty @NotNull + @JsonPropertyDescription("The key to split.") private EventKey source; @JsonProperty("delimiter_regex") + @JsonPropertyDescription("The regex string responsible for the split. Cannot be defined at the same time as `delimiter`. " + + "At least `delimiter` or `delimiter_regex` must be defined.") private String delimiterRegex; @Size(min = 1, max = 1) + @JsonPropertyDescription("The separator character responsible for the split. " + + "Cannot be defined at the same time as `delimiter_regex`. " + + "At least `delimiter` or `delimiter_regex` must be defined.") private String delimiter; @JsonProperty("split_when") + @JsonPropertyDescription("Specifies under what condition the `split_string` processor should perform splitting. " + + "Default is no condition.") private String splitWhen; public EventKey getSource() { @@ -61,6 +70,7 @@ public List getIterativeConfig() { return entries; } + @JsonPropertyDescription("List of entries. Valid values are `source`, `delimiter`, and `delimiter_regex`.") private List<@Valid Entry> entries; public List getEntries() { From 1ea308bfde80215231942b91f47eecbc7b5a0cfa Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Sat, 13 Jul 2024 00:31:45 -0500 Subject: [PATCH 072/159] MAINT: backfill doc into json property for trim_string (#4728) * MAINT: backfill doc into json property for trim_string Signed-off-by: George Chen --- .../plugins/processor/mutatestring/WithKeysConfig.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java index 05a9c198a6..3660b5d73d 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.model.event.EventKey; @@ -17,6 +18,7 @@ public class WithKeysConfig implements StringProcessorConfig { @NotNull @NotEmpty @JsonProperty("with_keys") + @JsonPropertyDescription("A list of keys to trim the white space from.") private List withKeys; @Override From 731de123b297998cfd158f37be0034ddc43ea237 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Mon, 15 Jul 2024 09:57:24 -0500 Subject: [PATCH 073/159] Load exported S3 files in RDS source (#4718) * Add s3 file loader Signed-off-by: Hai Yan * Make checkExportStatus a callable Signed-off-by: Hai Yan * Fix unit tests Signed-off-by: Hai Yan * Add load status and record converter Signed-off-by: Hai Yan * Update unit tests Signed-off-by: Hai Yan * Restore changes for test Signed-off-by: Hai Yan * Address review comments Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- data-prepper-plugins/rds-source/build.gradle | 2 + .../plugins/source/rds/ClientFactory.java | 8 + .../plugins/source/rds/RdsService.java | 29 +++- .../plugins/source/rds/RdsSource.java | 6 +- .../rds/converter/ExportRecordConverter.java | 36 ++++ .../rds/converter/MetadataKeyAttributes.java | 20 +++ .../rds/coordination/PartitionFactory.java | 5 +- .../partition/DataFilePartition.java | 77 +++++++++ .../state/DataFileProgressState.java | 44 +++++ .../source/rds/export/DataFileLoader.java | 83 +++++++++ .../source/rds/export/DataFileScheduler.java | 163 ++++++++++++++++++ .../source/rds/export/ExportScheduler.java | 130 +++++++++++--- .../source/rds/export/S3ObjectReader.java | 36 ++++ .../source/rds/model/ExportObjectKey.java | 68 ++++++++ .../plugins/source/rds/model/LoadStatus.java | 53 ++++++ .../plugins/source/rds/RdsServiceTest.java | 11 +- .../plugins/source/rds/RdsSourceTest.java | 6 +- .../converter/ExportRecordConverterTest.java | 51 ++++++ .../source/rds/export/DataFileLoaderTest.java | 67 +++++++ .../rds/export/DataFileSchedulerTest.java | 137 +++++++++++++++ .../rds/export/ExportSchedulerTest.java | 40 ++++- .../source/rds/export/S3ObjectReaderTest.java | 56 ++++++ .../source/rds/model/ExportObjectKeyTest.java | 37 ++++ 23 files changed, 1133 insertions(+), 32 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/DataFilePartition.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/S3ObjectReader.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/LoadStatus.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/S3ObjectReaderTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java diff --git a/data-prepper-plugins/rds-source/build.gradle b/data-prepper-plugins/rds-source/build.gradle index 580a312be0..f83b1332eb 100644 --- a/data-prepper-plugins/rds-source/build.gradle +++ b/data-prepper-plugins/rds-source/build.gradle @@ -8,6 +8,7 @@ dependencies { implementation project(path: ':data-prepper-plugins:buffer-common') implementation project(path: ':data-prepper-plugins:http-common') implementation project(path: ':data-prepper-plugins:common') + implementation project(path: ':data-prepper-plugins:parquet-codecs') implementation 'io.micrometer:micrometer-core' @@ -22,4 +23,5 @@ dependencies { testImplementation project(path: ':data-prepper-test-common') testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + testImplementation project(path: ':data-prepper-test-event') } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/ClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/ClientFactory.java index 9cdb2bfa50..7831754f0f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/ClientFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/ClientFactory.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.plugins.source.rds.configuration.AwsAuthenticationConfig; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.s3.S3Client; public class ClientFactory { private final AwsCredentialsProvider awsCredentialsProvider; @@ -32,4 +33,11 @@ public RdsClient buildRdsClient() { .credentialsProvider(awsCredentialsProvider) .build(); } + + public S3Client buildS3Client() { + return S3Client.builder() + .region(awsAuthenticationConfig.getAwsRegion()) + .credentialsProvider(awsCredentialsProvider) + .build(); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index f059dd52bf..77956e6b0e 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -8,13 +8,16 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.s3.S3Client; import java.util.ArrayList; import java.util.List; @@ -24,23 +27,34 @@ public class RdsService { private static final Logger LOG = LoggerFactory.getLogger(RdsService.class); + /** + * Maximum concurrent data loader per node + */ + public static final int DATA_LOADER_MAX_JOB_COUNT = 1; + private final RdsClient rdsClient; + private final S3Client s3Client; private final EnhancedSourceCoordinator sourceCoordinator; + private final EventFactory eventFactory; private final PluginMetrics pluginMetrics; private final RdsSourceConfig sourceConfig; private ExecutorService executor; private LeaderScheduler leaderScheduler; private ExportScheduler exportScheduler; + private DataFileScheduler dataFileScheduler; public RdsService(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig, + final EventFactory eventFactory, final ClientFactory clientFactory, final PluginMetrics pluginMetrics) { this.sourceCoordinator = sourceCoordinator; + this.eventFactory = eventFactory; this.pluginMetrics = pluginMetrics; this.sourceConfig = sourceConfig; rdsClient = clientFactory.buildRdsClient(); + s3Client = clientFactory.buildS3Client(); } /** @@ -54,9 +68,15 @@ public void start(Buffer> buffer) { LOG.info("Start running RDS service"); final List runnableList = new ArrayList<>(); leaderScheduler = new LeaderScheduler(sourceCoordinator, sourceConfig); - exportScheduler = new ExportScheduler(sourceCoordinator, rdsClient, pluginMetrics); runnableList.add(leaderScheduler); - runnableList.add(exportScheduler); + + if (sourceConfig.isExportEnabled()) { + exportScheduler = new ExportScheduler(sourceCoordinator, rdsClient, s3Client, pluginMetrics); + dataFileScheduler = new DataFileScheduler( + sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer); + runnableList.add(exportScheduler); + runnableList.add(dataFileScheduler); + } executor = Executors.newFixedThreadPool(runnableList.size()); runnableList.forEach(executor::submit); @@ -69,7 +89,10 @@ public void start(Buffer> buffer) { public void shutdown() { if (executor != null) { LOG.info("shutdown RDS schedulers"); - exportScheduler.shutdown(); + if (sourceConfig.isExportEnabled()) { + exportScheduler.shutdown(); + dataFileScheduler.shutdown(); + } leaderScheduler.shutdown(); executor.shutdownNow(); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java index a9fe983572..43806c0475 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.Source; import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; @@ -33,15 +34,18 @@ public class RdsSource implements Source>, UsesEnhancedSourceCoord private final ClientFactory clientFactory; private final PluginMetrics pluginMetrics; private final RdsSourceConfig sourceConfig; + private final EventFactory eventFactory; private EnhancedSourceCoordinator sourceCoordinator; private RdsService rdsService; @DataPrepperPluginConstructor public RdsSource(final PluginMetrics pluginMetrics, final RdsSourceConfig sourceConfig, + final EventFactory eventFactory, final AwsCredentialsSupplier awsCredentialsSupplier) { this.pluginMetrics = pluginMetrics; this.sourceConfig = sourceConfig; + this.eventFactory = eventFactory; clientFactory = new ClientFactory(awsCredentialsSupplier, sourceConfig.getAwsAuthenticationConfig()); } @@ -51,7 +55,7 @@ public void start(Buffer> buffer) { Objects.requireNonNull(sourceCoordinator); sourceCoordinator.createPartition(new LeaderPartition()); - rdsService = new RdsService(sourceCoordinator, sourceConfig, clientFactory, pluginMetrics); + rdsService = new RdsService(sourceCoordinator, sourceConfig, eventFactory, clientFactory, pluginMetrics); LOG.info("Start RDS service"); rdsService.start(buffer); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java new file mode 100644 index 0000000000..11932cd512 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java @@ -0,0 +1,36 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.converter; + +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; + +public class ExportRecordConverter { + + private static final Logger LOG = LoggerFactory.getLogger(ExportRecordConverter.class); + + static final String EXPORT_EVENT_TYPE = "EXPORT"; + + public Event convert(Record record, String tableName, String primaryKeyName) { + Event event = record.getData(); + + EventMetadata eventMetadata = event.getMetadata(); + eventMetadata.setAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE, tableName); + eventMetadata.setAttribute(INGESTION_EVENT_TYPE_ATTRIBUTE, EXPORT_EVENT_TYPE); + + final Object primaryKeyValue = record.getData().get(primaryKeyName, Object.class); + eventMetadata.setAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE, primaryKeyValue); + + return event; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java new file mode 100644 index 0000000000..91eecdf07b --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java @@ -0,0 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.converter; + +public class MetadataKeyAttributes { + static final String PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE = "primary_key"; + + static final String EVENT_VERSION_FROM_TIMESTAMP = "document_version"; + + static final String EVENT_TIMESTAMP_METADATA_ATTRIBUTE = "event_timestamp"; + + static final String EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE = "opensearch_action"; + + static final String EVENT_TABLE_NAME_METADATA_ATTRIBUTE = "table_name"; + + static final String INGESTION_EVENT_TYPE_ATTRIBUTE = "ingestion_type"; +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java index db35f5076b..6213263b09 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java @@ -7,6 +7,7 @@ import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; @@ -25,8 +26,10 @@ public EnhancedSourcePartition apply(SourcePartitionStoreItem partitionStoreItem if (LeaderPartition.PARTITION_TYPE.equals(partitionType)) { return new LeaderPartition(partitionStoreItem); - } if (ExportPartition.PARTITION_TYPE.equals(partitionType)) { + } else if (ExportPartition.PARTITION_TYPE.equals(partitionType)) { return new ExportPartition(partitionStoreItem); + } else if (DataFilePartition.PARTITION_TYPE.equals(partitionType)) { + return new DataFilePartition(partitionStoreItem); } else { // Unable to acquire other partitions. return new GlobalState(partitionStoreItem); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/DataFilePartition.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/DataFilePartition.java new file mode 100644 index 0000000000..985f48b652 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/DataFilePartition.java @@ -0,0 +1,77 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.DataFileProgressState; + +import java.util.Optional; + +/** + * An DataFilePartition represents an export data file needs to be loaded. + * The source identifier contains keyword 'DATAFILE' + */ +public class DataFilePartition extends EnhancedSourcePartition { + + public static final String PARTITION_TYPE = "DATAFILE"; + + private final String exportTaskId; + private final String bucket; + private final String key; + private final DataFileProgressState state; + + public DataFilePartition(final SourcePartitionStoreItem sourcePartitionStoreItem) { + + setSourcePartitionStoreItem(sourcePartitionStoreItem); + String[] keySplits = sourcePartitionStoreItem.getSourcePartitionKey().split("\\|"); + exportTaskId = keySplits[0]; + bucket = keySplits[1]; + key = keySplits[2]; + state = convertStringToPartitionProgressState(DataFileProgressState.class, sourcePartitionStoreItem.getPartitionProgressState()); + + } + + public DataFilePartition(final String exportTaskId, + final String bucket, + final String key, + final Optional state) { + this.exportTaskId = exportTaskId; + this.bucket = bucket; + this.key = key; + this.state = state.orElse(null); + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return exportTaskId + "|" + bucket + "|" + key; + } + + @Override + public Optional getProgressState() { + if (state != null) { + return Optional.of(state); + } + return Optional.empty(); + } + + public String getExportTaskId() { + return exportTaskId; + } + + public String getBucket() { + return bucket; + } + + public String getKey() { + return key; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java new file mode 100644 index 0000000000..c65c0bbe01 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DataFileProgressState { + + @JsonProperty("isLoaded") + private boolean isLoaded = false; + + @JsonProperty("totalRecords") + private int totalRecords; + + @JsonProperty("sourceTable") + private String sourceTable; + + public int getTotalRecords() { + return totalRecords; + } + + public void setTotalRecords(int totalRecords) { + this.totalRecords = totalRecords; + } + + public boolean getLoaded() { + return isLoaded; + } + + public void setLoaded(boolean loaded) { + this.isLoaded = loaded; + } + + public String getSourceTable() { + return sourceTable; + } + + public void setSourceTable(String sourceTable) { + this.sourceTable = sourceTable; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java new file mode 100644 index 0000000000..e76a04e99d --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -0,0 +1,83 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.InputStream; + +public class DataFileLoader implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(DataFileLoader.class); + + private final DataFilePartition dataFilePartition; + private final String bucket; + private final String objectKey; + private final S3ObjectReader objectReader; + private final InputCodec codec; + private final BufferAccumulator> bufferAccumulator; + private final ExportRecordConverter recordConverter; + + private DataFileLoader(final DataFilePartition dataFilePartition, + final InputCodec codec, + final BufferAccumulator> bufferAccumulator, + final S3ObjectReader objectReader, + final ExportRecordConverter recordConverter) { + this.dataFilePartition = dataFilePartition; + bucket = dataFilePartition.getBucket(); + objectKey = dataFilePartition.getKey(); + this.objectReader = objectReader; + this.codec = codec; + this.bufferAccumulator = bufferAccumulator; + this.recordConverter = recordConverter; + } + + public static DataFileLoader create(final DataFilePartition dataFilePartition, + final InputCodec codec, + final BufferAccumulator> bufferAccumulator, + final S3ObjectReader objectReader, + final ExportRecordConverter recordConverter) { + return new DataFileLoader(dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter); + } + + @Override + public void run() { + LOG.info("Start loading s3://{}/{}", bucket, objectKey); + + try (InputStream inputStream = objectReader.readFile(bucket, objectKey)) { + + codec.parse(inputStream, record -> { + try { + final String tableName = dataFilePartition.getProgressState().get().getSourceTable(); + // TODO: primary key to be obtained by querying database schema + final String primaryKeyName = "id"; + Record transformedRecord = new Record<>(recordConverter.convert(record, tableName, primaryKeyName)); + bufferAccumulator.add(transformedRecord); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + + LOG.info("Completed loading object s3://{}/{} to buffer", bucket, objectKey); + } catch (Exception e) { + LOG.error("Failed to load object s3://{}/{} to buffer", bucket, objectKey, e); + throw new RuntimeException(e); + } + + try { + bufferAccumulator.flush(); + } catch (Exception e) { + LOG.error("Failed to write events to buffer", e); + } + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java new file mode 100644 index 0000000000..d465d55076 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java @@ -0,0 +1,163 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.codec.parquet.ParquetInputCodec; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.rds.model.LoadStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.S3Client; + +import java.time.Duration; +import java.time.Instant; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.opensearch.dataprepper.plugins.source.rds.RdsService.DATA_LOADER_MAX_JOB_COUNT; + +public class DataFileScheduler implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(DataFileScheduler.class); + + private final AtomicInteger numOfWorkers = new AtomicInteger(0); + + /** + * Default interval to acquire a lease from coordination store + */ + private static final int DEFAULT_LEASE_INTERVAL_MILLIS = 2_000; + + private static final Duration DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT = Duration.ofMinutes(30); + + static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); + static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; + + + private final EnhancedSourceCoordinator sourceCoordinator; + private final ExecutorService executor; + private final RdsSourceConfig sourceConfig; + private final S3ObjectReader objectReader; + private final InputCodec codec; + private final BufferAccumulator> bufferAccumulator; + private final ExportRecordConverter recordConverter; + + private volatile boolean shutdownRequested = false; + + public DataFileScheduler(final EnhancedSourceCoordinator sourceCoordinator, + final RdsSourceConfig sourceConfig, + final S3Client s3Client, + final EventFactory eventFactory, + final Buffer> buffer) { + this.sourceCoordinator = sourceCoordinator; + this.sourceConfig = sourceConfig; + codec = new ParquetInputCodec(eventFactory); + bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); + objectReader = new S3ObjectReader(s3Client); + recordConverter = new ExportRecordConverter(); + executor = Executors.newFixedThreadPool(DATA_LOADER_MAX_JOB_COUNT); + } + + @Override + public void run() { + LOG.debug("Starting Data File Scheduler to process S3 data files for export"); + + while (!shutdownRequested && !Thread.currentThread().isInterrupted()) { + try { + if (numOfWorkers.get() < DATA_LOADER_MAX_JOB_COUNT) { + final Optional sourcePartition = sourceCoordinator.acquireAvailablePartition(DataFilePartition.PARTITION_TYPE); + + if (sourcePartition.isPresent()) { + LOG.debug("Acquired data file partition"); + DataFilePartition dataFilePartition = (DataFilePartition) sourcePartition.get(); + LOG.debug("Start processing data file partition"); + processDataFilePartition(dataFilePartition); + } + } + try { + Thread.sleep(DEFAULT_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException e) { + LOG.info("The DataFileScheduler was interrupted while waiting to retry, stopping processing"); + break; + } + } catch (final Exception e) { + LOG.error("Received an exception while processing an S3 data file, backing off and retrying", e); + try { + Thread.sleep(DEFAULT_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException ex) { + LOG.info("The DataFileScheduler was interrupted while waiting to retry, stopping processing"); + break; + } + } + } + LOG.warn("Data file scheduler is interrupted, stopping all data file loaders..."); + + executor.shutdown(); + } + + public void shutdown() { + shutdownRequested = true; + } + + private void processDataFilePartition(DataFilePartition dataFilePartition) { + Runnable loader = DataFileLoader.create(dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter); + CompletableFuture runLoader = CompletableFuture.runAsync(loader, executor); + + runLoader.whenComplete((v, ex) -> { + if (ex == null) { + // Update global state so we know if all s3 files have been loaded + updateLoadStatus(dataFilePartition.getExportTaskId(), DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT); + sourceCoordinator.completePartition(dataFilePartition); + } else { + LOG.error("There was an exception while processing an S3 data file", (Throwable) ex); + sourceCoordinator.giveUpPartition(dataFilePartition); + } + numOfWorkers.decrementAndGet(); + }); + numOfWorkers.incrementAndGet(); + } + + private void updateLoadStatus(String exportTaskId, Duration timeout) { + + Instant endTime = Instant.now().plus(timeout); + // Keep retrying in case update fails due to conflicts until timed out + while (Instant.now().isBefore(endTime)) { + Optional globalStatePartition = sourceCoordinator.getPartition(exportTaskId); + if (globalStatePartition.isEmpty()) { + LOG.error("Failed to get data file load status for {}", exportTaskId); + return; + } + + GlobalState globalState = (GlobalState) globalStatePartition.get(); + LoadStatus loadStatus = LoadStatus.fromMap(globalState.getProgressState().get()); + loadStatus.setLoadedFiles(loadStatus.getLoadedFiles() + 1); + LOG.info("Current data file load status: total {} loaded {}", loadStatus.getTotalFiles(), loadStatus.getLoadedFiles()); + + globalState.setProgressState(loadStatus.toMap()); + + try { + sourceCoordinator.saveProgressStateForPartition(globalState, null); + // TODO: Stream is enabled and loadStatus.getLoadedFiles() == loadStatus.getTotalFiles(), create global state to indicate that stream can start + break; + } catch (Exception e) { + LOG.error("Failed to update the global status, looks like the status was out of date, will retry.."); + } + } + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index 51db82248b..abcbd2c1f4 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -8,22 +8,36 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.DataFileProgressState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; +import org.opensearch.dataprepper.plugins.source.rds.model.ExportObjectKey; import org.opensearch.dataprepper.plugins.source.rds.model.ExportStatus; +import org.opensearch.dataprepper.plugins.source.rds.model.LoadStatus; import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.rds.RdsClient; +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.util.ArrayList; +import java.util.List; import java.util.Optional; +import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.BiConsumer; +import java.util.stream.Collectors; public class ExportScheduler implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(ExportScheduler.class); @@ -34,8 +48,10 @@ public class ExportScheduler implements Runnable { private static final int DEFAULT_CHECKPOINT_INTERVAL_MILLS = 5 * 60_000; private static final int DEFAULT_CHECK_STATUS_INTERVAL_MILLS = 30 * 1000; private static final Duration DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT = Duration.ofMinutes(60); + static final String PARQUET_SUFFIX = ".parquet"; private final RdsClient rdsClient; + private final S3Client s3Client; private final PluginMetrics pluginMetrics; private final EnhancedSourceCoordinator sourceCoordinator; private final ExecutorService executor; @@ -46,10 +62,12 @@ public class ExportScheduler implements Runnable { public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, final RdsClient rdsClient, + final S3Client s3Client, final PluginMetrics pluginMetrics) { this.pluginMetrics = pluginMetrics; this.sourceCoordinator = sourceCoordinator; this.rdsClient = rdsClient; + this.s3Client = s3Client; this.executor = Executors.newCachedThreadPool(); this.exportTaskManager = new ExportTaskManager(rdsClient); this.snapshotManager = new SnapshotManager(rdsClient); @@ -72,7 +90,8 @@ public void run() { LOG.error("The export to S3 failed, it will be retried"); closeExportPartitionWithError(exportPartition); } else { - CompletableFuture checkStatus = CompletableFuture.supplyAsync(() -> checkExportStatus(exportPartition), executor); + CheckExportStatusRunner checkExportStatusRunner = new CheckExportStatusRunner(sourceCoordinator, exportTaskManager, exportPartition); + CompletableFuture checkStatus = CompletableFuture.supplyAsync(checkExportStatusRunner::call, executor); checkStatus.whenComplete(completeExport(exportPartition)); } } @@ -179,29 +198,46 @@ private String checkSnapshotStatus(String snapshotId, Duration timeout) { throw new RuntimeException("Snapshot status check timed out."); } - private String checkExportStatus(ExportPartition exportPartition) { - long lastCheckpointTime = System.currentTimeMillis(); - String exportTaskId = exportPartition.getProgressState().get().getExportTaskId(); + static class CheckExportStatusRunner implements Callable { + private final EnhancedSourceCoordinator sourceCoordinator; + private final ExportTaskManager exportTaskManager; + private final ExportPartition exportPartition; - LOG.debug("Start checking the status of export {}", exportTaskId); - while (true) { - if (System.currentTimeMillis() - lastCheckpointTime > DEFAULT_CHECKPOINT_INTERVAL_MILLS) { - sourceCoordinator.saveProgressStateForPartition(exportPartition, null); - lastCheckpointTime = System.currentTimeMillis(); - } + CheckExportStatusRunner(EnhancedSourceCoordinator sourceCoordinator, ExportTaskManager exportTaskManager, ExportPartition exportPartition) { + this.sourceCoordinator = sourceCoordinator; + this.exportTaskManager = exportTaskManager; + this.exportPartition = exportPartition; + } - // Valid statuses are: CANCELED, CANCELING, COMPLETE, FAILED, IN_PROGRESS, STARTING - String status = exportTaskManager.checkExportStatus(exportTaskId); - LOG.debug("Current export status is {}.", status); - if (ExportStatus.isTerminal(status)) { - LOG.info("Export {} is completed with final status {}", exportTaskId, status); - return status; - } - LOG.debug("Export {} is still running in progress. Wait and check later", exportTaskId); - try { - Thread.sleep(DEFAULT_CHECK_STATUS_INTERVAL_MILLS); - } catch (InterruptedException e) { - throw new RuntimeException(e); + @Override + public String call() { + return checkExportStatus(exportPartition); + } + + private String checkExportStatus(ExportPartition exportPartition) { + long lastCheckpointTime = System.currentTimeMillis(); + String exportTaskId = exportPartition.getProgressState().get().getExportTaskId(); + + LOG.debug("Start checking the status of export {}", exportTaskId); + while (true) { + if (System.currentTimeMillis() - lastCheckpointTime > DEFAULT_CHECKPOINT_INTERVAL_MILLS) { + sourceCoordinator.saveProgressStateForPartition(exportPartition, null); + lastCheckpointTime = System.currentTimeMillis(); + } + + // Valid statuses are: CANCELED, CANCELING, COMPLETE, FAILED, IN_PROGRESS, STARTING + String status = exportTaskManager.checkExportStatus(exportTaskId); + LOG.debug("Current export status is {}.", status); + if (ExportStatus.isTerminal(status)) { + LOG.info("Export {} is completed with final status {}", exportTaskId, status); + return status; + } + LOG.debug("Export {} is still running in progress. Wait and check later", exportTaskId); + try { + Thread.sleep(DEFAULT_CHECK_STATUS_INTERVAL_MILLS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } } } } @@ -219,11 +255,61 @@ private BiConsumer completeExport(ExportPartition exportParti } LOG.info("Export for {} completed successfully", exportPartition.getPartitionKey()); + ExportProgressState state = exportPartition.getProgressState().get(); + String bucket = state.getBucket(); + String prefix = state.getPrefix(); + String exportTaskId = state.getExportTaskId(); + + // Create data file partitions for processing S3 files + List dataFileObjectKeys = getDataFileObjectKeys(bucket, prefix, exportTaskId); + createDataFilePartitions(bucket, exportTaskId, dataFileObjectKeys); + completeExportPartition(exportPartition); } }; } + private List getDataFileObjectKeys(String bucket, String prefix, String exportTaskId) { + LOG.debug("Fetching object keys for export data files."); + ListObjectsV2Request.Builder requestBuilder = ListObjectsV2Request.builder() + .bucket(bucket) + .prefix(prefix + "/" + exportTaskId); + + List objectKeys = new ArrayList<>(); + ListObjectsV2Response response = null; + do { + String nextToken = response == null ? null : response.nextContinuationToken(); + response = s3Client.listObjectsV2(requestBuilder + .continuationToken(nextToken) + .build()); + objectKeys.addAll(response.contents().stream() + .map(S3Object::key) + .filter(key -> key.endsWith(PARQUET_SUFFIX)) + .collect(Collectors.toList())); + + } while (response.isTruncated()); + return objectKeys; + } + + private void createDataFilePartitions(String bucket, String exportTaskId, List dataFileObjectKeys) { + LOG.info("Total of {} data files generated for export {}", dataFileObjectKeys.size(), exportTaskId); + AtomicInteger totalFiles = new AtomicInteger(); + for (final String objectKey : dataFileObjectKeys) { + DataFileProgressState progressState = new DataFileProgressState(); + ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKey); + String table = exportObjectKey.getTableName(); + progressState.setSourceTable(table); + + DataFilePartition dataFilePartition = new DataFilePartition(exportTaskId, bucket, objectKey, Optional.of(progressState)); + sourceCoordinator.createPartition(dataFilePartition); + totalFiles.getAndIncrement(); + } + + // Create a global state to track overall progress for data file processing + LoadStatus loadStatus = new LoadStatus(totalFiles.get(), 0); + sourceCoordinator.createPartition(new GlobalState(exportTaskId, loadStatus.toMap())); + } + private void completeExportPartition(ExportPartition exportPartition) { ExportProgressState progressState = exportPartition.getProgressState().get(); progressState.setStatus("Completed"); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/S3ObjectReader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/S3ObjectReader.java new file mode 100644 index 0000000000..39c0079198 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/S3ObjectReader.java @@ -0,0 +1,36 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; + +import java.io.InputStream; + +public class S3ObjectReader { + + private static final Logger LOG = LoggerFactory.getLogger(S3ObjectReader.class); + + private final S3Client s3Client; + + public S3ObjectReader(S3Client s3Client) { + this.s3Client = s3Client; + } + + public InputStream readFile(String bucketName, String s3Key) { + LOG.debug("Read file from s3://{}/{}", bucketName, s3Key); + + GetObjectRequest objectRequest = GetObjectRequest.builder() + .bucket(bucketName) + .key(s3Key) + .build(); + + return s3Client.getObject(objectRequest); + } + +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java new file mode 100644 index 0000000000..c69dcc7651 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java @@ -0,0 +1,68 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +/** + * Represents the object key for an object exported to S3 by RDS. + * The object key has this structure: "{prefix}/{export task ID}/{database name}/{table name}/{numbered folder}/{file name}" + */ +public class ExportObjectKey { + + private final String prefix; + private final String exportTaskId; + private final String databaseName; + private final String tableName; + private final String numberedFolder; + private final String fileName; + + ExportObjectKey(final String prefix, final String exportTaskId, final String databaseName, final String tableName, final String numberedFolder, final String fileName) { + this.prefix = prefix; + this.exportTaskId = exportTaskId; + this.databaseName = databaseName; + this.tableName = tableName; + this.numberedFolder = numberedFolder; + this.fileName = fileName; + } + + public static ExportObjectKey fromString(final String objectKeyString) { + + final String[] parts = objectKeyString.split("/"); + if (parts.length != 6) { + throw new IllegalArgumentException("Export object key is not valid: " + objectKeyString); + } + final String prefix = parts[0]; + final String exportTaskId = parts[1]; + final String databaseName = parts[2]; + final String tableName = parts[3]; + final String numberedFolder = parts[4]; + final String fileName = parts[5]; + return new ExportObjectKey(prefix, exportTaskId, databaseName, tableName, numberedFolder, fileName); + } + + public String getPrefix() { + return prefix; + } + + public String getExportTaskId() { + return exportTaskId; + } + + public String getDatabaseName() { + return databaseName; + } + + public String getTableName() { + return tableName; + } + + public String getNumberedFolder() { + return numberedFolder; + } + + public String getFileName() { + return fileName; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/LoadStatus.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/LoadStatus.java new file mode 100644 index 0000000000..a2762c1b38 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/LoadStatus.java @@ -0,0 +1,53 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +import java.util.Map; + +public class LoadStatus { + + private static final String TOTAL_FILES = "totalFiles"; + private static final String LOADED_FILES = "loadedFiles"; + + private int totalFiles; + + private int loadedFiles; + + public LoadStatus(int totalFiles, int loadedFiles) { + this.totalFiles = totalFiles; + this.loadedFiles = loadedFiles; + } + + public int getTotalFiles() { + return totalFiles; + } + + public void setTotalFiles(int totalFiles) { + this.totalFiles = totalFiles; + } + + public int getLoadedFiles() { + return loadedFiles; + } + + public void setLoadedFiles(int loadedFiles) { + this.loadedFiles = loadedFiles; + } + + public Map toMap() { + return Map.of( + TOTAL_FILES, totalFiles, + LOADED_FILES, loadedFiles + ); + } + + public static LoadStatus fromMap(Map map) { + return new LoadStatus( + ((Number) map.get(TOTAL_FILES)).intValue(), + ((Number) map.get(LOADED_FILES)).intValue() + ); + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java index 6aaa0b0bd5..7a18dd6159 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java @@ -14,8 +14,10 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; import software.amazon.awssdk.services.rds.RdsClient; @@ -47,6 +49,9 @@ class RdsServiceTest { @Mock private ExecutorService executor; + @Mock + private EventFactory eventFactory; + @Mock private ClientFactory clientFactory; @@ -59,8 +64,9 @@ void setUp() { } @Test - void test_normal_service_start() { + void test_normal_service_start_when_export_is_enabled() { RdsService rdsService = createObjectUnderTest(); + when(sourceConfig.isExportEnabled()).thenReturn(true); try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); rdsService.start(buffer); @@ -68,6 +74,7 @@ void test_normal_service_start() { verify(executor).submit(any(LeaderScheduler.class)); verify(executor).submit(any(ExportScheduler.class)); + verify(executor).submit(any(DataFileScheduler.class)); } @Test @@ -83,6 +90,6 @@ void test_service_shutdown_calls_executor_shutdownNow() { } private RdsService createObjectUnderTest() { - return new RdsService(sourceCoordinator, sourceConfig, clientFactory, pluginMetrics); + return new RdsService(sourceCoordinator, sourceConfig, eventFactory, clientFactory, pluginMetrics); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java index edd409e5e4..682f16ed51 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java @@ -12,6 +12,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.plugins.source.rds.configuration.AwsAuthenticationConfig; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -27,6 +28,9 @@ class RdsSourceTest { @Mock private RdsSourceConfig sourceConfig; + @Mock + private EventFactory eventFactory; + @Mock AwsCredentialsSupplier awsCredentialsSupplier; @@ -45,6 +49,6 @@ void test_when_buffer_is_null_then_start_throws_exception() { } private RdsSource createObjectUnderTest() { - return new RdsSource(pluginMetrics, sourceConfig, awsCredentialsSupplier); + return new RdsSource(pluginMetrics, sourceConfig, eventFactory, awsCredentialsSupplier); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java new file mode 100644 index 0000000000..79c5597c3b --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java @@ -0,0 +1,51 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.converter; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.event.TestEventFactory; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.record.Record; + +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.sameInstance; +import static org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter.EXPORT_EVENT_TYPE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; + +@ExtendWith(MockitoExtension.class) +class ExportRecordConverterTest { + + @Test + void test_convert() { + final String tableName = UUID.randomUUID().toString(); + final String primaryKeyName = UUID.randomUUID().toString(); + final String primaryKeyValue = UUID.randomUUID().toString(); + final Event testEvent = TestEventFactory.getTestEventFactory().eventBuilder(EventBuilder.class) + .withEventType("EVENT") + .withData(Map.of(primaryKeyName, primaryKeyValue)) + .build(); + + Record testRecord = new Record<>(testEvent); + + ExportRecordConverter exportRecordConverter = new ExportRecordConverter(); + Event actualEvent = exportRecordConverter.convert(testRecord, tableName, primaryKeyName); + + // Assert + assertThat(actualEvent.getMetadata().getAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE), equalTo(tableName)); + assertThat(actualEvent.getMetadata().getAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE), equalTo(primaryKeyValue)); + assertThat(actualEvent.getMetadata().getAttribute(INGESTION_EVENT_TYPE_ATTRIBUTE), equalTo(EXPORT_EVENT_TYPE)); + assertThat(actualEvent, sameInstance(testRecord.getData())); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java new file mode 100644 index 0000000000..1ed91bc031 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java @@ -0,0 +1,67 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +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.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; + +import java.io.InputStream; +import java.util.UUID; +import java.util.function.Consumer; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class DataFileLoaderTest { + + @Mock + private DataFilePartition dataFilePartition; + + @Mock + private BufferAccumulator> bufferAccumulator; + + @Mock + private InputCodec codec; + + @Mock + private S3ObjectReader s3ObjectReader; + + @Mock + private ExportRecordConverter recordConverter; + + @Test + void test_run() throws Exception { + final String bucket = UUID.randomUUID().toString(); + final String key = UUID.randomUUID().toString(); + when(dataFilePartition.getBucket()).thenReturn(bucket); + when(dataFilePartition.getKey()).thenReturn(key); + + InputStream inputStream = mock(InputStream.class); + when(s3ObjectReader.readFile(bucket, key)).thenReturn(inputStream); + + DataFileLoader objectUnderTest = createObjectUnderTest(); + objectUnderTest.run(); + + verify(codec).parse(eq(inputStream), any(Consumer.class)); + verify(bufferAccumulator).flush(); + } + + private DataFileLoader createObjectUnderTest() { + return DataFileLoader.create(dataFilePartition, codec, bufferAccumulator, s3ObjectReader, recordConverter); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java new file mode 100644 index 0000000000..ee0d0e2852 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java @@ -0,0 +1,137 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.rds.model.LoadStatus; +import software.amazon.awssdk.services.s3.S3Client; + +import java.time.Duration; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class DataFileSchedulerTest { + + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + + @Mock + private RdsSourceConfig sourceConfig; + + @Mock + private S3Client s3Client; + + @Mock + private EventFactory eventFactory; + + @Mock + private Buffer> buffer; + + @Mock + private DataFilePartition dataFilePartition; + + private Random random; + + @BeforeEach + void setUp() { + random = new Random(); + } + + @Test + void test_given_no_datafile_partition_then_no_export() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(DataFilePartition.PARTITION_TYPE)).thenReturn(Optional.empty()); + + final DataFileScheduler objectUnderTest = createObjectUnderTest(); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(objectUnderTest); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(DataFilePartition.PARTITION_TYPE)); + Thread.sleep(100); + executorService.shutdownNow(); + + verifyNoInteractions(s3Client, buffer); + } + + @Test + void test_given_available_datafile_partition_then_load_datafile() { + DataFileScheduler objectUnderTest = createObjectUnderTest(); + final String exportTaskId = UUID.randomUUID().toString(); + when(dataFilePartition.getExportTaskId()).thenReturn(exportTaskId); + + when(sourceCoordinator.acquireAvailablePartition(DataFilePartition.PARTITION_TYPE)).thenReturn(Optional.of(dataFilePartition)); + final GlobalState globalStatePartition = mock(GlobalState.class); + final int totalFiles = random.nextInt() + 1; + final Map loadStatusMap = new LoadStatus(totalFiles, totalFiles - 1).toMap(); + when(globalStatePartition.getProgressState()).thenReturn(Optional.of(loadStatusMap)); + when(sourceCoordinator.getPartition(exportTaskId)).thenReturn(Optional.of(globalStatePartition)); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> { + // MockedStatic needs to be created on the same thread it's used + try (MockedStatic dataFileLoaderMockedStatic = mockStatic(DataFileLoader.class)) { + DataFileLoader dataFileLoader = mock(DataFileLoader.class); + dataFileLoaderMockedStatic.when(() -> DataFileLoader.create( + eq(dataFilePartition), any(InputCodec.class), any(BufferAccumulator.class), any(S3ObjectReader.class), any(ExportRecordConverter.class))) + .thenReturn(dataFileLoader); + doNothing().when(dataFileLoader).run(); + objectUnderTest.run(); + } + }); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).completePartition(dataFilePartition)); + executorService.shutdownNow(); + + verify(sourceCoordinator).completePartition(dataFilePartition); + } + + @Test + void test_shutdown() { + DataFileScheduler objectUnderTest = createObjectUnderTest(); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(objectUnderTest); + + objectUnderTest.shutdown(); + + verifyNoMoreInteractions(sourceCoordinator); + executorService.shutdownNow(); + } + + private DataFileScheduler createObjectUnderTest() { + return new DataFileScheduler(sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java index d0560ab30d..32aff02a57 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java @@ -15,6 +15,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; import software.amazon.awssdk.services.rds.RdsClient; @@ -27,9 +28,14 @@ import software.amazon.awssdk.services.rds.model.DescribeExportTasksResponse; import software.amazon.awssdk.services.rds.model.StartExportTaskRequest; import software.amazon.awssdk.services.rds.model.StartExportTaskResponse; +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.util.List; import java.util.Optional; import java.util.UUID; import java.util.concurrent.ExecutorService; @@ -44,6 +50,7 @@ import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.PARQUET_SUFFIX; @ExtendWith(MockitoExtension.class) @@ -55,6 +62,9 @@ class ExportSchedulerTest { @Mock private RdsClient rdsClient; + @Mock + private S3Client s3Client; + @Mock private PluginMetrics pluginMetrics; @@ -96,6 +106,18 @@ void test_given_export_partition_and_task_id_then_complete_export() throws Inter when(describeExportTasksResponse.exportTasks().get(0).status()).thenReturn("COMPLETE"); when(rdsClient.describeExportTasks(any(DescribeExportTasksRequest.class))).thenReturn(describeExportTasksResponse); + // Mock list s3 objects response + ListObjectsV2Response listObjectsV2Response = mock(ListObjectsV2Response.class); + String exportTaskId = UUID.randomUUID().toString(); + String tableName = UUID.randomUUID().toString(); + // objectKey needs to have this structure: "{prefix}/{export task ID}/{database name}/{table name}/{numbered folder}/{file name}" + S3Object s3Object = S3Object.builder() + .key("prefix/" + exportTaskId + "/my_db/" + tableName + "/1/file1" + PARQUET_SUFFIX) + .build(); + when(listObjectsV2Response.contents()).thenReturn(List.of(s3Object)); + when(listObjectsV2Response.isTruncated()).thenReturn(false); + when(s3Client.listObjectsV2(any(ListObjectsV2Request.class))).thenReturn(listObjectsV2Response); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(exportScheduler); await().atMost(Duration.ofSeconds(1)) @@ -103,6 +125,7 @@ void test_given_export_partition_and_task_id_then_complete_export() throws Inter Thread.sleep(100); executorService.shutdownNow(); + verify(sourceCoordinator).createPartition(any(DataFilePartition.class)); verify(sourceCoordinator).completePartition(exportPartition); verify(rdsClient, never()).startExportTask(any(StartExportTaskRequest.class)); verify(rdsClient, never()).createDBSnapshot(any(CreateDbSnapshotRequest.class)); @@ -110,7 +133,7 @@ void test_given_export_partition_and_task_id_then_complete_export() throws Inter @Test - void test_given_export_partition_and_no_task_id_then_start_and_complete_export() throws InterruptedException { + void test_given_export_partition_without_task_id_then_start_and_complete_export() throws InterruptedException { when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.of(exportPartition)); when(exportPartition.getPartitionKey()).thenReturn(UUID.randomUUID().toString()); when(exportProgressState.getExportTaskId()).thenReturn(null).thenReturn(UUID.randomUUID().toString()); @@ -142,6 +165,18 @@ void test_given_export_partition_and_no_task_id_then_start_and_complete_export() when(describeExportTasksResponse.exportTasks().get(0).status()).thenReturn("COMPLETE"); when(rdsClient.describeExportTasks(any(DescribeExportTasksRequest.class))).thenReturn(describeExportTasksResponse); + // Mock list s3 objects response + ListObjectsV2Response listObjectsV2Response = mock(ListObjectsV2Response.class); + String exportTaskId = UUID.randomUUID().toString(); + String tableName = UUID.randomUUID().toString(); + // objectKey needs to have this structure: "{prefix}/{export task ID}/{database name}/{table name}/{numbered folder}/{file name}" + S3Object s3Object = S3Object.builder() + .key("prefix/" + exportTaskId + "/my_db/" + tableName + "/1/file1" + PARQUET_SUFFIX) + .build(); + when(listObjectsV2Response.contents()).thenReturn(List.of(s3Object)); + when(listObjectsV2Response.isTruncated()).thenReturn(false); + when(s3Client.listObjectsV2(any(ListObjectsV2Request.class))).thenReturn(listObjectsV2Response); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(exportScheduler); await().atMost(Duration.ofSeconds(1)) @@ -151,6 +186,7 @@ void test_given_export_partition_and_no_task_id_then_start_and_complete_export() verify(rdsClient).createDBSnapshot(any(CreateDbSnapshotRequest.class)); verify(rdsClient).startExportTask(any(StartExportTaskRequest.class)); + verify(sourceCoordinator).createPartition(any(DataFilePartition.class)); verify(sourceCoordinator).completePartition(exportPartition); } @@ -166,6 +202,6 @@ void test_shutDown() { } private ExportScheduler createObjectUnderTest() { - return new ExportScheduler(sourceCoordinator, rdsClient, pluginMetrics); + return new ExportScheduler(sourceCoordinator, rdsClient, s3Client, pluginMetrics); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/S3ObjectReaderTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/S3ObjectReaderTest.java new file mode 100644 index 0000000000..44aa22f6ad --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/S3ObjectReaderTest.java @@ -0,0 +1,56 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; + +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.verify; + +@ExtendWith(MockitoExtension.class) +class S3ObjectReaderTest { + + @Mock + private S3Client s3Client; + + private S3ObjectReader s3ObjectReader; + + + @BeforeEach + void setUp() { + s3ObjectReader = createObjectUnderTest(); + } + + @Test + void test_readFile() { + final String bucketName = UUID.randomUUID().toString(); + final String key = UUID.randomUUID().toString(); + + + s3ObjectReader.readFile(bucketName, key); + + ArgumentCaptor getObjectRequestArgumentCaptor = ArgumentCaptor.forClass(GetObjectRequest.class); + verify(s3Client).getObject(getObjectRequestArgumentCaptor.capture()); + + GetObjectRequest request = getObjectRequestArgumentCaptor.getValue(); + assertThat(request.bucket(), equalTo(bucketName)); + assertThat(request.key(), equalTo(key)); + } + + private S3ObjectReader createObjectUnderTest() { + return new S3ObjectReader(s3Client); + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java new file mode 100644 index 0000000000..7056114572 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java @@ -0,0 +1,37 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class ExportObjectKeyTest { + + @Test + void test_fromString_with_valid_input_string() { + final String objectKeyString = "prefix/export-task-id/db-name/table-name/1/file-name.parquet"; + final ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKeyString); + + assertThat(exportObjectKey.getPrefix(), equalTo("prefix")); + assertThat(exportObjectKey.getExportTaskId(), equalTo("export-task-id")); + assertThat(exportObjectKey.getDatabaseName(), equalTo("db-name")); + assertThat(exportObjectKey.getTableName(), equalTo("table-name")); + assertThat(exportObjectKey.getNumberedFolder(), equalTo("1")); + assertThat(exportObjectKey.getFileName(), equalTo("file-name.parquet")); + } + + @Test + void test_fromString_with_invalid_input_string() { + final String objectKeyString = "prefix/export-task-id/db-name/table-name/1/"; + + Throwable exception = assertThrows(IllegalArgumentException.class, () -> ExportObjectKey.fromString(objectKeyString)); + assertThat(exception.getMessage(), containsString("Export object key is not valid: " + objectKeyString)); + } +} \ No newline at end of file From 286edc2a9e1d72c9a49e3d932142a969f3282369 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Mon, 15 Jul 2024 10:05:10 -0500 Subject: [PATCH 074/159] REF: grok processor with the latest config model (#4731) * REF: grok processor with the latest config model Signed-off-by: George Chen --- .../plugins/processor/grok/GrokProcessor.java | 24 ++-- .../processor/grok/GrokProcessorConfig.java | 118 ++++++++--------- .../grok/GrokProcessorConfigTests.java | 37 +++--- .../processor/grok/GrokProcessorIT.java | 54 +++++--- .../processor/grok/GrokProcessorTests.java | 121 ++++++++++-------- 5 files changed, 193 insertions(+), 161 deletions(-) diff --git a/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessor.java b/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessor.java index 8b8b7f2e90..8cc9c6a716 100644 --- a/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessor.java +++ b/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessor.java @@ -12,10 +12,10 @@ import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.Timer; import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.annotations.SingleThread; -import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; @@ -59,7 +59,7 @@ @SingleThread -@DataPrepperPlugin(name = "grok", pluginType = Processor.class) +@DataPrepperPlugin(name = "grok", pluginType = Processor.class, pluginConfigurationType = GrokProcessorConfig.class) public class GrokProcessor extends AbstractProcessor, Record> { static final long EXECUTOR_SERVICE_SHUTDOWN_TIMEOUT = 300L; @@ -89,20 +89,28 @@ public class GrokProcessor extends AbstractProcessor, Record(grokProcessorConfig.getkeysToOverwrite()); this.grokCompiler = grokCompiler; this.fieldToGrok = new LinkedHashMap<>(); this.executorService = executorService; this.expressionEvaluator = expressionEvaluator; this.tagsOnMatchFailure = grokProcessorConfig.getTagsOnMatchFailure(); - this.tagsOnTimeout = grokProcessorConfig.getTagsOnTimeout(); + this.tagsOnTimeout = grokProcessorConfig.getTagsOnTimeout().isEmpty() ? + grokProcessorConfig.getTagsOnMatchFailure() : grokProcessorConfig.getTagsOnTimeout(); grokProcessingMatchCounter = pluginMetrics.counter(GROK_PROCESSING_MATCH); grokProcessingMismatchCounter = pluginMetrics.counter(GROK_PROCESSING_MISMATCH); grokProcessingErrorsCounter = pluginMetrics.counter(GROK_PROCESSING_ERRORS); diff --git a/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfig.java b/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfig.java index de9daf91d5..2d2ae1ef41 100644 --- a/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfig.java +++ b/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.grok; -import org.opensearch.dataprepper.model.configuration.PluginSetting; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -39,69 +41,57 @@ public class GrokProcessorConfig { static final int DEFAULT_TIMEOUT_MILLIS = 30000; static final String DEFAULT_TARGET_KEY = null; - private final boolean breakOnMatch; - private final boolean keepEmptyCaptures; - private final Map> match; - private final boolean namedCapturesOnly; - private final List keysToOverwrite; - private final List patternsDirectories; - private final String patternsFilesGlob; - private final Map patternDefinitions; - private final int timeoutMillis; - private final String targetKey; - private final String grokWhen; - private final List tagsOnMatchFailure; - private final List tagsOnTimeout; - - private final boolean includePerformanceMetadata; - - private GrokProcessorConfig(final boolean breakOnMatch, - final boolean keepEmptyCaptures, - final Map> match, - final boolean namedCapturesOnly, - final List keysToOverwrite, - final List patternsDirectories, - final String patternsFilesGlob, - final Map patternDefinitions, - final int timeoutMillis, - final String targetKey, - final String grokWhen, - final List tagsOnMatchFailure, - final List tagsOnTimeout, - final boolean includePerformanceMetadata) { - - this.breakOnMatch = breakOnMatch; - this.keepEmptyCaptures = keepEmptyCaptures; - this.match = match; - this.namedCapturesOnly = namedCapturesOnly; - this.keysToOverwrite = keysToOverwrite; - this.patternsDirectories = patternsDirectories; - this.patternsFilesGlob = patternsFilesGlob; - this.patternDefinitions = patternDefinitions; - this.timeoutMillis = timeoutMillis; - this.targetKey = targetKey; - this.grokWhen = grokWhen; - this.tagsOnMatchFailure = tagsOnMatchFailure; - this.tagsOnTimeout = tagsOnTimeout.isEmpty() ? tagsOnMatchFailure : tagsOnTimeout; - this.includePerformanceMetadata = includePerformanceMetadata; - } - - public static GrokProcessorConfig buildConfig(final PluginSetting pluginSetting) { - return new GrokProcessorConfig(pluginSetting.getBooleanOrDefault(BREAK_ON_MATCH, DEFAULT_BREAK_ON_MATCH), - pluginSetting.getBooleanOrDefault(KEEP_EMPTY_CAPTURES, DEFAULT_KEEP_EMPTY_CAPTURES), - pluginSetting.getTypedListMap(MATCH, String.class, String.class), - pluginSetting.getBooleanOrDefault(NAMED_CAPTURES_ONLY, DEFAULT_NAMED_CAPTURES_ONLY), - pluginSetting.getTypedList(KEYS_TO_OVERWRITE, String.class), - pluginSetting.getTypedList(PATTERNS_DIRECTORIES, String.class), - pluginSetting.getStringOrDefault(PATTERNS_FILES_GLOB, DEFAULT_PATTERNS_FILES_GLOB), - pluginSetting.getTypedMap(PATTERN_DEFINITIONS, String.class, String.class), - pluginSetting.getIntegerOrDefault(TIMEOUT_MILLIS, DEFAULT_TIMEOUT_MILLIS), - pluginSetting.getStringOrDefault(TARGET_KEY, DEFAULT_TARGET_KEY), - pluginSetting.getStringOrDefault(GROK_WHEN, null), - pluginSetting.getTypedList(TAGS_ON_MATCH_FAILURE, String.class), - pluginSetting.getTypedList(TAGS_ON_TIMEOUT, String.class), - pluginSetting.getBooleanOrDefault(INCLUDE_PERFORMANCE_METADATA, false)); - } + @JsonProperty(BREAK_ON_MATCH) + @JsonPropertyDescription("Specifies whether to match all patterns (`false`) or stop once the first successful " + + "match is found (`true`). Default is `true`.") + private boolean breakOnMatch = DEFAULT_BREAK_ON_MATCH; + @JsonProperty(KEEP_EMPTY_CAPTURES) + @JsonPropertyDescription("Enables the preservation of `null` captures from the processed output. Default is `false`.") + private boolean keepEmptyCaptures = DEFAULT_KEEP_EMPTY_CAPTURES; + @JsonProperty(MATCH) + @JsonPropertyDescription("Specifies which keys should match specific patterns. Default is an empty response body.") + private Map> match = Collections.emptyMap(); + @JsonProperty(NAMED_CAPTURES_ONLY) + @JsonPropertyDescription("Specifies whether to keep only named captures. Default is `true`.") + private boolean namedCapturesOnly = DEFAULT_NAMED_CAPTURES_ONLY; + @JsonProperty(KEYS_TO_OVERWRITE) + @JsonPropertyDescription("Specifies which existing keys will be overwritten if there is a capture with the same key value. " + + "Default is `[]`.") + private List keysToOverwrite = Collections.emptyList(); + @JsonProperty(PATTERNS_DIRECTORIES) + @JsonPropertyDescription("Specifies which directory paths contain the custom pattern files. Default is an empty list.") + private List patternsDirectories = Collections.emptyList(); + @JsonProperty(PATTERNS_FILES_GLOB) + @JsonPropertyDescription("Specifies which pattern files to use from the directories specified for " + + "`pattern_directories`. Default is `*`.") + private String patternsFilesGlob = DEFAULT_PATTERNS_FILES_GLOB; + @JsonProperty(PATTERN_DEFINITIONS) + @JsonPropertyDescription("Allows for a custom pattern that can be used inline inside the response body. " + + "Default is an empty response body.") + private Map patternDefinitions = Collections.emptyMap(); + @JsonProperty(TIMEOUT_MILLIS) + @JsonPropertyDescription("The maximum amount of time during which matching occurs. " + + "Setting to `0` prevents any matching from occurring. Default is `30,000`.") + private int timeoutMillis = DEFAULT_TIMEOUT_MILLIS; + @JsonProperty(TARGET_KEY) + @JsonPropertyDescription("Specifies a parent-level key used to store all captures. Default value is `null`.") + private String targetKey = DEFAULT_TARGET_KEY; + @JsonProperty(GROK_WHEN) + @JsonPropertyDescription("Specifies under what condition the `grok` processor should perform matching. " + + "Default is no condition.") + private String grokWhen; + @JsonProperty(TAGS_ON_MATCH_FAILURE) + @JsonPropertyDescription("A `List` of `String`s that specifies the tags to be set in the event when grok fails to " + + "match or an unknown exception occurs while matching. This tag may be used in conditional expressions in " + + "other parts of the configuration") + private List tagsOnMatchFailure = Collections.emptyList(); + @JsonProperty(TAGS_ON_TIMEOUT) + @JsonPropertyDescription("A `List` of `String`s that specifies the tags to be set in the event when grok match times out.") + private List tagsOnTimeout = Collections.emptyList(); + @JsonProperty(INCLUDE_PERFORMANCE_METADATA) + @JsonPropertyDescription("A `Boolean` on whether to include performance metadata into event metadata, " + + "e.g. _total_grok_patterns_attempted, _total_grok_processing_time.") + private boolean includePerformanceMetadata = false; public boolean isBreakOnMatch() { return breakOnMatch; diff --git a/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfigTests.java b/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfigTests.java index eb69968a96..37c5ec9cb1 100644 --- a/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfigTests.java +++ b/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfigTests.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.processor.grok; +import com.fasterxml.jackson.databind.ObjectMapper; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -27,6 +28,7 @@ import static org.opensearch.dataprepper.plugins.processor.grok.GrokProcessorConfig.DEFAULT_TIMEOUT_MILLIS; public class GrokProcessorConfigTests { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final String PLUGIN_NAME = "grok"; private static final Map> TEST_MATCH = new HashMap<>(); @@ -62,7 +64,8 @@ public static void setUp() { @Test public void testDefault() { - final GrokProcessorConfig grokProcessorConfig = GrokProcessorConfig.buildConfig(new PluginSetting(PLUGIN_NAME, null)); + final GrokProcessorConfig grokProcessorConfig = OBJECT_MAPPER.convertValue( + Collections.emptyMap(), GrokProcessorConfig.class); assertThat(grokProcessorConfig.isBreakOnMatch(), equalTo(DEFAULT_BREAK_ON_MATCH)); assertThat(grokProcessorConfig.isKeepEmptyCaptures(), equalTo(DEFAULT_KEEP_EMPTY_CAPTURES)); @@ -95,7 +98,8 @@ public void testValidConfig() { TEST_TARGET_KEY, true); - final GrokProcessorConfig grokProcessorConfig = GrokProcessorConfig.buildConfig(validPluginSetting); + final GrokProcessorConfig grokProcessorConfig = OBJECT_MAPPER.convertValue( + validPluginSetting.getSettings(), GrokProcessorConfig.class); assertThat(grokProcessorConfig.isBreakOnMatch(), equalTo(false)); assertThat(grokProcessorConfig.isKeepEmptyCaptures(), equalTo(true)); @@ -127,7 +131,8 @@ public void testInvalidConfig() { invalidPluginSetting.getSettings().put(GrokProcessorConfig.MATCH, TEST_INVALID_MATCH); - assertThrows(IllegalArgumentException.class, () -> GrokProcessorConfig.buildConfig(invalidPluginSetting)); + assertThrows(IllegalArgumentException.class, () -> OBJECT_MAPPER.convertValue( + invalidPluginSetting.getSettings(), GrokProcessorConfig.class)); } private PluginSetting completePluginSettingForGrokProcessor(final boolean breakOnMatch, @@ -160,33 +165,22 @@ private PluginSetting completePluginSettingForGrokProcessor(final boolean breakO @Test void getTagsOnMatchFailure_returns_tagOnMatch() { final List tagsOnMatch = List.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); - final GrokProcessorConfig objectUnderTest = GrokProcessorConfig.buildConfig(new PluginSetting(PLUGIN_NAME, - Map.of(GrokProcessorConfig.TAGS_ON_MATCH_FAILURE, tagsOnMatch) - )); + final GrokProcessorConfig objectUnderTest = OBJECT_MAPPER.convertValue( + Map.of(GrokProcessorConfig.TAGS_ON_MATCH_FAILURE, tagsOnMatch), GrokProcessorConfig.class); assertThat(objectUnderTest.getTagsOnMatchFailure(), equalTo(tagsOnMatch)); } - @Test - void getTagsOnTimeout_returns_tagsOnMatch_if_no_tagsOnTimeout() { - final List tagsOnMatch = List.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); - final GrokProcessorConfig objectUnderTest = GrokProcessorConfig.buildConfig(new PluginSetting(PLUGIN_NAME, - Map.of(GrokProcessorConfig.TAGS_ON_MATCH_FAILURE, tagsOnMatch) - )); - - assertThat(objectUnderTest.getTagsOnTimeout(), equalTo(tagsOnMatch)); - } - @Test void getTagsOnTimeout_returns_tagsOnTimeout_if_present() { final List tagsOnMatch = List.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); final List tagsOnTimeout = List.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); - final GrokProcessorConfig objectUnderTest = GrokProcessorConfig.buildConfig(new PluginSetting(PLUGIN_NAME, + final GrokProcessorConfig objectUnderTest = OBJECT_MAPPER.convertValue( Map.of( GrokProcessorConfig.TAGS_ON_MATCH_FAILURE, tagsOnMatch, GrokProcessorConfig.TAGS_ON_TIMEOUT, tagsOnTimeout - ) - )); + ), + GrokProcessorConfig.class); assertThat(objectUnderTest.getTagsOnTimeout(), equalTo(tagsOnTimeout)); } @@ -194,9 +188,8 @@ void getTagsOnTimeout_returns_tagsOnTimeout_if_present() { @Test void getTagsOnTimeout_returns_tagsOnTimeout_if_present_and_no_tagsOnMatch() { final List tagsOnTimeout = List.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); - final GrokProcessorConfig objectUnderTest = GrokProcessorConfig.buildConfig(new PluginSetting(PLUGIN_NAME, - Map.of(GrokProcessorConfig.TAGS_ON_TIMEOUT, tagsOnTimeout) - )); + final GrokProcessorConfig objectUnderTest = OBJECT_MAPPER.convertValue( + Map.of(GrokProcessorConfig.TAGS_ON_TIMEOUT, tagsOnTimeout), GrokProcessorConfig.class); assertThat(objectUnderTest.getTagsOnTimeout(), equalTo(tagsOnTimeout)); } diff --git a/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorIT.java b/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorIT.java index 1c8d0036c2..f6fa090405 100644 --- a/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorIT.java +++ b/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorIT.java @@ -16,6 +16,7 @@ import org.junit.jupiter.params.provider.MethodSource; import org.mockito.Mock; import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; @@ -38,6 +39,8 @@ public class GrokProcessorIT { private PluginSetting pluginSetting; + private PluginMetrics pluginMetrics; + private GrokProcessorConfig grokProcessorConfig; private GrokProcessor grokProcessor; private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference>() {}; @@ -65,6 +68,8 @@ public void setup() { null); pluginSetting.setPipelineName("grokPipeline"); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + pluginMetrics = PluginMetrics.fromPluginSetting(pluginSetting); // This is a COMMONAPACHELOG pattern with the following format // COMMONAPACHELOG %{IPORHOST:clientip} %{USER:ident} %{USER:auth} \[%{HTTPDATE:timestamp}\] "(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest})" %{NUMBER:response} (?:%{NUMBER:bytes}|-) @@ -115,7 +120,8 @@ public void testMatchNoCapturesWithExistingAndNonExistingKey() throws JsonProces matchConfig.put("bad_key", Collections.singletonList(nonMatchingPattern)); pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", messageInput); @@ -135,7 +141,8 @@ public void testSingleMatchSinglePatternWithDefaults() throws JsonProcessingExce matchConfig.put("message", Collections.singletonList("%{COMMONAPACHELOG}")); pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", messageInput); @@ -173,7 +180,8 @@ public void testSingleMatchMultiplePatternWithBreakOnMatchFalse() throws JsonPro pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); pluginSetting.getSettings().put(GrokProcessorConfig.BREAK_ON_MATCH, false); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", messageInput); @@ -208,7 +216,8 @@ public void testSingleMatchTypeConversionWithDefaults() throws JsonProcessingExc matchConfig.put("message", Collections.singletonList("\"(?:%{WORD:verb} %{NOTSPACE:request}(?: HTTP/%{NUMBER:httpversion})?|%{DATA:rawrequest})\" %{NUMBER:response:int} (?:%{NUMBER:bytes:float}|-)")); pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", messageInput); @@ -240,7 +249,8 @@ public void testMultipleMatchWithBreakOnMatchFalse() throws JsonProcessingExcept pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); pluginSetting.getSettings().put(GrokProcessorConfig.BREAK_ON_MATCH, false); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", messageInput); @@ -278,7 +288,8 @@ public void testMatchWithKeepEmptyCapturesTrue() throws JsonProcessingException pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); pluginSetting.getSettings().put(GrokProcessorConfig.KEEP_EMPTY_CAPTURES, true); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", messageInput); @@ -314,7 +325,8 @@ public void testMatchWithNamedCapturesOnlyFalse() throws JsonProcessingException pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); pluginSetting.getSettings().put(GrokProcessorConfig.NAMED_CAPTURES_ONLY, false); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", "This is my greedy data before matching 192.0.2.1 123456"); @@ -346,7 +358,8 @@ public void testPatternDefinitions() throws JsonProcessingException { pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); pluginSetting.getSettings().put(GrokProcessorConfig.PATTERN_DEFINITIONS, patternDefinitions); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", "This is my greedy data before matching with my phone number 123-456-789"); @@ -389,7 +402,8 @@ public void testPatternsDirWithDefaultPatternsFilesGlob() throws JsonProcessingE pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); pluginSetting.getSettings().put(GrokProcessorConfig.PATTERNS_DIRECTORIES, patternsDirectories); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Record resultRecord = buildRecordWithEvent(resultData); @@ -422,7 +436,8 @@ public void testPatternsDirWithCustomPatternsFilesGlob() throws JsonProcessingEx pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); pluginSetting.getSettings().put(GrokProcessorConfig.PATTERNS_DIRECTORIES, patternsDirectories); pluginSetting.getSettings().put(GrokProcessorConfig.PATTERNS_FILES_GLOB, "*1.txt"); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Record resultRecord = buildRecordWithEvent(resultData); @@ -436,8 +451,10 @@ public void testPatternsDirWithCustomPatternsFilesGlob() throws JsonProcessingEx matchConfigWithPatterns2Pattern.put("message", Collections.singletonList("My birthday is %{CUSTOMBIRTHDAYPATTERN:my_birthday}")); pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfigWithPatterns2Pattern); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); - Throwable throwable = assertThrows(IllegalArgumentException.class, () -> new GrokProcessor(pluginSetting, expressionEvaluator)); + Throwable throwable = assertThrows(IllegalArgumentException.class, () -> new GrokProcessor( + pluginMetrics, grokProcessorConfig, expressionEvaluator)); assertThat("No definition for key 'CUSTOMBIRTHDAYPATTERN' found, aborting", equalTo(throwable.getMessage())); } @@ -447,7 +464,8 @@ public void testMatchWithNamedCapturesSyntax() throws JsonProcessingException { matchConfig.put("message", Collections.singletonList("%{GREEDYDATA:greedy_data} (?\\d\\d\\d-\\d\\d\\d-\\d\\d\\d)")); pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", "This is my greedy data before matching with my phone number 123-456-789"); @@ -477,7 +495,8 @@ public void testMatchWithNoCapturesAndTags() throws JsonProcessingException { pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); pluginSetting.getSettings().put(GrokProcessorConfig.TAGS_ON_MATCH_FAILURE, List.of(tagOnMatchFailure1, tagOnMatchFailure2)); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("log", "This is my greedy data before matching with my phone number 123-456-789"); @@ -495,14 +514,16 @@ public void testMatchWithNoCapturesAndTags() throws JsonProcessingException { @Test public void testCompileNonRegisteredPatternThrowsIllegalArgumentException() { - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map> matchConfig = new HashMap<>(); matchConfig.put("message", Collections.singletonList("%{NONEXISTENTPATTERN}")); pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); - assertThrows(IllegalArgumentException.class, () -> new GrokProcessor(pluginSetting, expressionEvaluator)); + assertThrows(IllegalArgumentException.class, () -> new GrokProcessor( + pluginMetrics, grokProcessorConfig, expressionEvaluator)); } @ParameterizedTest @@ -512,7 +533,8 @@ void testDataPrepperBuiltInGrokPatterns(final String matchPattern, final String matchConfig.put("message", Collections.singletonList(matchPattern)); pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); - grokProcessor = new GrokProcessor(pluginSetting, expressionEvaluator); + grokProcessorConfig = OBJECT_MAPPER.convertValue(pluginSetting.getSettings(), GrokProcessorConfig.class); + grokProcessor = new GrokProcessor(pluginMetrics, grokProcessorConfig, expressionEvaluator); final Map testData = new HashMap(); testData.put("message", logInput); diff --git a/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorTests.java b/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorTests.java index e9d17121d8..aedad1fe5c 100644 --- a/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorTests.java +++ b/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorTests.java @@ -20,11 +20,9 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mock; -import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; @@ -52,7 +50,6 @@ import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; import static org.mockito.Mockito.lenient; -import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; @@ -109,23 +106,22 @@ public class GrokProcessorTests { @Mock private ExpressionEvaluator expressionEvaluator; - - private PluginSetting pluginSetting; + @Mock + private GrokProcessorConfig grokProcessorConfig; private final String PLUGIN_NAME = "grok"; private Map capture; private final Map> matchConfig = new HashMap<>(); @BeforeEach public void setup() throws TimeoutException, ExecutionException, InterruptedException { - pluginSetting = getDefaultPluginSetting(); - pluginSetting.setPipelineName("grokPipeline"); + configureDefaultGrokProcessorConfig(); final List matchPatterns = new ArrayList<>(); matchPatterns.add("%{PATTERN1}"); matchPatterns.add("%{PATTERN2}"); matchConfig.put("message", matchPatterns); - pluginSetting.getSettings().put(GrokProcessorConfig.MATCH, matchConfig); + when(grokProcessorConfig.getMatch()).thenReturn(matchConfig); lenient().when(pluginMetrics.counter(GrokProcessor.GROK_PROCESSING_MATCH)).thenReturn(grokProcessingMatchCounter); lenient().when(pluginMetrics.counter(GrokProcessor.GROK_PROCESSING_MISMATCH)).thenReturn(grokProcessingMismatchCounter); @@ -155,15 +151,13 @@ public void setup() throws TimeoutException, ExecutionException, InterruptedExce } private GrokProcessor createObjectUnderTest() { - try (MockedStatic pluginMetricsMockedStatic = mockStatic(PluginMetrics.class)) { - pluginMetricsMockedStatic.when(() -> PluginMetrics.fromPluginSetting(pluginSetting)).thenReturn(pluginMetrics); - return new GrokProcessor(pluginSetting, grokCompiler, executorService, expressionEvaluator); - } + return new GrokProcessor( + pluginMetrics, grokProcessorConfig, grokCompiler, executorService, expressionEvaluator); } @Test public void testMatchMerge() throws JsonProcessingException, ExecutionException, InterruptedException, TimeoutException { - pluginSetting.getSettings().put(GrokProcessorConfig.INCLUDE_PERFORMANCE_METADATA, false); + when(grokProcessorConfig.getIncludePerformanceMetadata()).thenReturn(false); grokProcessor = createObjectUnderTest(); @@ -202,7 +196,7 @@ public void testMatchMerge() throws JsonProcessingException, ExecutionException, @Test public void testTarget() throws JsonProcessingException, ExecutionException, InterruptedException, TimeoutException { - pluginSetting.getSettings().put(GrokProcessorConfig.TARGET_KEY, "test_target"); + when(grokProcessorConfig.getTargetKey()).thenReturn("test_target"); grokProcessor = createObjectUnderTest(); capture.put("key_capture_1", "value_capture_1"); @@ -238,7 +232,7 @@ public void testTarget() throws JsonProcessingException, ExecutionException, Int @Test public void testOverwrite() throws JsonProcessingException { - pluginSetting.getSettings().put(GrokProcessorConfig.KEYS_TO_OVERWRITE, Collections.singletonList("message")); + when(grokProcessorConfig.getkeysToOverwrite()).thenReturn(Collections.singletonList("message")); grokProcessor = createObjectUnderTest(); capture.put("key_capture_1", "value_capture_1"); @@ -423,7 +417,7 @@ public void testThatTimeoutExceptionIsCaughtAndProcessingContinues() throws Json @Test public void testThatProcessingWithTimeoutMillisOfZeroDoesNotInteractWithExecutorServiceAndReturnsCorrectResult() throws JsonProcessingException { - pluginSetting.getSettings().put(GrokProcessorConfig.TIMEOUT_MILLIS, 0); + when(grokProcessorConfig.getTimeoutMillis()).thenReturn(0); grokProcessor = createObjectUnderTest(); capture.put("key_capture_1", "value_capture_1"); @@ -528,7 +522,7 @@ public void testNoCaptures() throws JsonProcessingException { @Test public void testMatchOnSecondPattern() throws JsonProcessingException { - pluginSetting.getSettings().put(GrokProcessorConfig.INCLUDE_PERFORMANCE_METADATA, true); + when(grokProcessorConfig.getIncludePerformanceMetadata()).thenReturn(true); when(match.capture()).thenReturn(Collections.emptyMap()); when(grokSecondMatch.match(messageInput)).thenReturn(secondMatch); @@ -556,7 +550,7 @@ public void testMatchOnSecondPattern() throws JsonProcessingException { @Test public void testMatchOnSecondPatternWithExistingMetadataForTotalPatternMatches() throws JsonProcessingException { - pluginSetting.getSettings().put(GrokProcessorConfig.INCLUDE_PERFORMANCE_METADATA, true); + when(grokProcessorConfig.getIncludePerformanceMetadata()).thenReturn(true); when(match.capture()).thenReturn(Collections.emptyMap()); when(grokSecondMatch.match(messageInput)).thenReturn(secondMatch); @@ -598,8 +592,10 @@ void setUp() { tagOnMatchFailure2 = UUID.randomUUID().toString(); tagOnTimeout1 = UUID.randomUUID().toString(); tagOnTimeout2 = UUID.randomUUID().toString(); - pluginSetting.getSettings().put(GrokProcessorConfig.TAGS_ON_MATCH_FAILURE, List.of(tagOnMatchFailure1, tagOnMatchFailure2)); - pluginSetting.getSettings().put(GrokProcessorConfig.TAGS_ON_TIMEOUT, List.of(tagOnTimeout1, tagOnTimeout2)); + when(grokProcessorConfig.getTagsOnMatchFailure()).thenReturn( + List.of(tagOnMatchFailure1, tagOnMatchFailure2)); + when(grokProcessorConfig.getTagsOnTimeout()).thenReturn( + List.of(tagOnTimeout1, tagOnTimeout2)); } @Test @@ -654,6 +650,34 @@ public void timeout_exception_tags_the_event() throws JsonProcessingException, T verifyNoInteractions(grokProcessingErrorsCounter, grokProcessingMismatchCounter); } + @Test + public void timeout_exception_tags_the_event_with_tags_on_match_failure() + throws JsonProcessingException, TimeoutException, ExecutionException, InterruptedException { + when(grokProcessorConfig.getTagsOnTimeout()).thenReturn(Collections.emptyList()); + when(task.get(GrokProcessorConfig.DEFAULT_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS)).thenThrow(TimeoutException.class); + + grokProcessor = createObjectUnderTest(); + + capture.put("key_capture_1", "value_capture_1"); + capture.put("key_capture_2", "value_capture_2"); + capture.put("key_capture_3", "value_capture_3"); + + final Map testData = new HashMap(); + testData.put("message", messageInput); + final Record record = buildRecordWithEvent(testData); + + final List> grokkedRecords = (List>) grokProcessor.doExecute(Collections.singletonList(record)); + + assertThat(grokkedRecords.size(), equalTo(1)); + assertThat(grokkedRecords.get(0), notNullValue()); + assertRecordsAreEqual(grokkedRecords.get(0), record); + assertThat(record.getData().getMetadata().getTags(), hasItem(tagOnMatchFailure1)); + assertThat(record.getData().getMetadata().getTags(), hasItem(tagOnMatchFailure2)); + verify(grokProcessingTimeoutsCounter, times(1)).increment(); + verify(grokProcessingTime, times(1)).record(any(Runnable.class)); + verifyNoInteractions(grokProcessingErrorsCounter, grokProcessingMismatchCounter); + } + @ParameterizedTest @ValueSource(classes = {ExecutionException.class, InterruptedException.class, RuntimeException.class}) public void execution_exception_tags_the_event(Class exceptionClass) throws JsonProcessingException, TimeoutException, ExecutionException, InterruptedException { @@ -720,7 +744,7 @@ public void testBreakOnMatchTrue() throws JsonProcessingException { @Test public void testBreakOnMatchFalse() throws JsonProcessingException { - pluginSetting.getSettings().put(GrokProcessorConfig.BREAK_ON_MATCH, false); + when(grokProcessorConfig.isBreakOnMatch()).thenReturn(false); grokProcessor = createObjectUnderTest(); when(grokSecondMatch.match(messageInput)).thenReturn(secondMatch); @@ -756,10 +780,8 @@ public void testBreakOnMatchFalse() throws JsonProcessingException { } } - private PluginSetting getDefaultPluginSetting() { - - return completePluginSettingForGrokProcessor( - GrokProcessorConfig.DEFAULT_BREAK_ON_MATCH, + private void configureDefaultGrokProcessorConfig() { + completeMockGrokProcessorConfig(GrokProcessorConfig.DEFAULT_BREAK_ON_MATCH, GrokProcessorConfig.DEFAULT_KEEP_EMPTY_CAPTURES, matchConfig, GrokProcessorConfig.DEFAULT_NAMED_CAPTURES_ONLY, @@ -775,7 +797,7 @@ private PluginSetting getDefaultPluginSetting() { @Test public void testNoGrok_when_GrokWhen_returns_false() throws JsonProcessingException { final String grokWhen = UUID.randomUUID().toString(); - pluginSetting.getSettings().put(GrokProcessorConfig.GROK_WHEN, grokWhen); + when(grokProcessorConfig.getGrokWhen()).thenReturn(grokWhen); grokProcessor = createObjectUnderTest(); capture.put("key_capture_1", "value_capture_1"); @@ -796,31 +818,28 @@ public void testNoGrok_when_GrokWhen_returns_false() throws JsonProcessingExcept verifyNoInteractions(grok, grokSecondMatch); } - private PluginSetting completePluginSettingForGrokProcessor(final boolean breakOnMatch, - final boolean keepEmptyCaptures, - final Map> match, - final boolean namedCapturesOnly, - final List keysToOverwrite, - final List patternsDirectories, - final String patternsFilesGlob, - final Map patternDefinitions, - final int timeoutMillis, - final String targetKey, - final String grokWhen) { - final Map settings = new HashMap<>(); - settings.put(GrokProcessorConfig.BREAK_ON_MATCH, breakOnMatch); - settings.put(GrokProcessorConfig.NAMED_CAPTURES_ONLY, namedCapturesOnly); - settings.put(GrokProcessorConfig.MATCH, match); - settings.put(GrokProcessorConfig.KEEP_EMPTY_CAPTURES, keepEmptyCaptures); - settings.put(GrokProcessorConfig.KEYS_TO_OVERWRITE, keysToOverwrite); - settings.put(GrokProcessorConfig.PATTERNS_DIRECTORIES, patternsDirectories); - settings.put(GrokProcessorConfig.PATTERN_DEFINITIONS, patternDefinitions); - settings.put(GrokProcessorConfig.PATTERNS_FILES_GLOB, patternsFilesGlob); - settings.put(GrokProcessorConfig.TIMEOUT_MILLIS, timeoutMillis); - settings.put(GrokProcessorConfig.TARGET_KEY, targetKey); - settings.put(GrokProcessorConfig.GROK_WHEN, grokWhen); - - return new PluginSetting(PLUGIN_NAME, settings); + private void completeMockGrokProcessorConfig(final boolean breakOnMatch, + final boolean keepEmptyCaptures, + final Map> match, + final boolean namedCapturesOnly, + final List keysToOverwrite, + final List patternsDirectories, + final String patternsFilesGlob, + final Map patternDefinitions, + final int timeoutMillis, + final String targetKey, + final String grokWhen) { + lenient().when(grokProcessorConfig.isBreakOnMatch()).thenReturn(breakOnMatch); + lenient().when(grokProcessorConfig.isNamedCapturesOnly()).thenReturn(namedCapturesOnly); + lenient().when(grokProcessorConfig.getMatch()).thenReturn(match); + lenient().when(grokProcessorConfig.isKeepEmptyCaptures()).thenReturn(keepEmptyCaptures); + lenient().when(grokProcessorConfig.getkeysToOverwrite()).thenReturn(keysToOverwrite); + lenient().when(grokProcessorConfig.getPatternsDirectories()).thenReturn(patternsDirectories); + lenient().when(grokProcessorConfig.getPatternDefinitions()).thenReturn(patternDefinitions); + lenient().when(grokProcessorConfig.getPatternsFilesGlob()).thenReturn(patternsFilesGlob); + lenient().when(grokProcessorConfig.getTimeoutMillis()).thenReturn(timeoutMillis); + lenient().when(grokProcessorConfig.getTargetKey()).thenReturn(targetKey); + lenient().when(grokProcessorConfig.getGrokWhen()).thenReturn(grokWhen); } private void assertRecordsAreEqual(final Record first, final Record second) throws JsonProcessingException { From aeac95328c830c76788a3baf65b6bea8c16c58bf Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Mon, 15 Jul 2024 10:30:14 -0500 Subject: [PATCH 075/159] MAINT: add documentation in json property description for date processor (#4719) * MAINT: add documentation in json property description for date processor Signed-off-by: George Chen --- .../processor/date/DateProcessorConfig.java | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java index a74b2e9d38..aed3a38674 100644 --- a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java +++ b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import java.time.ZoneId; @@ -24,8 +25,16 @@ public class DateProcessorConfig { public static class DateMatch { @JsonProperty("key") + @JsonPropertyDescription("Represents the event key against which to match patterns. " + + "Required if `match` is configured. ") private String key; @JsonProperty("patterns") + @JsonPropertyDescription("A list of possible patterns that the timestamp value of the key can have. The patterns " + + "are based on a sequence of letters and symbols. The `patterns` support all the patterns listed in the " + + "Java [DatetimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) reference. " + + "The timestamp value also supports `epoch_second`, `epoch_milli`, and `epoch_nano` values, " + + "which represent the timestamp as the number of seconds, milliseconds, and nanoseconds since the epoch. " + + "Epoch values always use the UTC time zone.") private List patterns; public DateMatch() { @@ -82,30 +91,57 @@ public static boolean isValidPattern(final String pattern) { } @JsonProperty("from_time_received") + @JsonPropertyDescription("When `true`, the timestamp from the event metadata, " + + "which is the time at which the source receives the event, is added to the event data. " + + "This option cannot be defined at the same time as `match`. Default is `false`.") private Boolean fromTimeReceived = DEFAULT_FROM_TIME_RECEIVED; @JsonProperty("to_origination_metadata") + @JsonPropertyDescription("When `true`, the matched time is also added to the event's metadata as an instance of " + + "`Instant`. Default is `false`.") private Boolean toOriginationMetadata = DEFAULT_TO_ORIGINATION_METADATA; @JsonProperty("match") + @JsonPropertyDescription("The date match configuration. " + + "This option cannot be defined at the same time as `from_time_received`. There is no default value.") private List match; @JsonProperty("destination") + @JsonPropertyDescription("The field used to store the timestamp parsed by the date processor. " + + "Can be used with both `match` and `from_time_received`. Default is `@timestamp`.") private String destination = DEFAULT_DESTINATION; @JsonProperty("output_format") + @JsonPropertyDescription("Determines the format of the timestamp added to an event. " + + "Default is `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`.") private String outputFormat = DEFAULT_OUTPUT_FORMAT; @JsonProperty("source_timezone") + @JsonPropertyDescription("The time zone used to parse dates, including when the zone or offset cannot be extracted " + + "from the value. If the zone or offset are part of the value, then the time zone is ignored. " + + "A list of all the available time zones is contained in the **TZ database name** column of " + + "[the list of database time zones](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones#List).") private String sourceTimezone = DEFAULT_SOURCE_TIMEZONE; @JsonProperty("destination_timezone") + @JsonPropertyDescription("The time zone used for storing the timestamp in the `destination` field. " + + "A list of all the available time zones is contained in the **TZ database name** column of " + + "[the list of database time zones](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones#List).") private String destinationTimezone = DEFAULT_DESTINATION_TIMEZONE; @JsonProperty("locale") + @JsonPropertyDescription("The location used for parsing dates. Commonly used for parsing month names (`MMM`). " + + "The value can contain language, country, or variant fields in IETF BCP 47, such as `en-US`, " + + "or a string representation of the " + + "[locale](https://docs.oracle.com/javase/8/docs/api/java/util/Locale.html) object, such as `en_US`. " + + "A full list of locale fields, including language, country, and variant, can be found in " + + "[the language subtag registry](https://www.iana.org/assignments/language-subtag-registry/language-subtag-registry). " + + "Default is `Locale.ROOT`.") private String locale; @JsonProperty("date_when") + @JsonPropertyDescription("Specifies under what condition the `date` processor should perform matching. " + + "Default is no condition.") private String dateWhen; @JsonIgnore From c4455a7785bc2da4358067c217be7085e0bc8d0f Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Mon, 15 Jul 2024 11:02:40 -0500 Subject: [PATCH 076/159] REF: service-map processor with the latest config model (#4734) * REF: service-map processor with the latest config model Signed-off-by: George Chen --- .../service-map-stateful/build.gradle | 1 + .../processor/ServiceMapProcessorConfig.java | 14 ++++++- .../ServiceMapStatefulProcessor.java | 25 +++++++----- .../ServiceMapProcessorConfigTest.java | 38 ++++++++++++++++++ .../ServiceMapStatefulProcessorTest.java | 39 +++++++++++-------- 5 files changed, 91 insertions(+), 26 deletions(-) create mode 100644 data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfigTest.java diff --git a/data-prepper-plugins/service-map-stateful/build.gradle b/data-prepper-plugins/service-map-stateful/build.gradle index fa83d4e6bc..ab2300f020 100644 --- a/data-prepper-plugins/service-map-stateful/build.gradle +++ b/data-prepper-plugins/service-map-stateful/build.gradle @@ -19,6 +19,7 @@ dependencies { exclude group: 'com.google.protobuf', module: 'protobuf-java' } implementation libs.protobuf.core + testImplementation project(':data-prepper-test-common') } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfig.java b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfig.java index 8c337b2737..7f72fb5286 100644 --- a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfig.java +++ b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfig.java @@ -5,8 +5,20 @@ package org.opensearch.dataprepper.plugins.processor; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; + public class ServiceMapProcessorConfig { - static final String WINDOW_DURATION = "window_duration"; + private static final String WINDOW_DURATION = "window_duration"; static final int DEFAULT_WINDOW_DURATION = 180; static final String DEFAULT_DB_PATH = "data/service-map/"; + + @JsonProperty(WINDOW_DURATION) + @JsonPropertyDescription("Represents the fixed time window, in seconds, " + + "during which service map relationships are evaluated. Default value is 180.") + private int windowDuration = DEFAULT_WINDOW_DURATION; + + public int getWindowDuration() { + return windowDuration; + } } diff --git a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java index c02ccb17d6..75041a09b4 100644 --- a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java +++ b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java @@ -6,9 +6,11 @@ package org.opensearch.dataprepper.plugins.processor; import org.apache.commons.codec.DecoderException; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.annotations.SingleThread; -import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.peerforwarder.RequiresPeerForwarding; @@ -40,7 +42,8 @@ import java.util.concurrent.atomic.AtomicInteger; @SingleThread -@DataPrepperPlugin(name = "service_map", deprecatedName = "service_map_stateful", pluginType = Processor.class) +@DataPrepperPlugin(name = "service_map", deprecatedName = "service_map_stateful", pluginType = Processor.class, + pluginConfigurationType = ServiceMapProcessorConfig.class) public class ServiceMapStatefulProcessor extends AbstractProcessor, Record> implements RequiresPeerForwarding { static final String SPANS_DB_SIZE = "spansDbSize"; @@ -75,20 +78,24 @@ public class ServiceMapStatefulProcessor extends AbstractProcessor private final int thisProcessorId; - public ServiceMapStatefulProcessor(final PluginSetting pluginSetting) { - this(pluginSetting.getIntegerOrDefault(ServiceMapProcessorConfig.WINDOW_DURATION, ServiceMapProcessorConfig.DEFAULT_WINDOW_DURATION) * TO_MILLIS, + @DataPrepperPluginConstructor + public ServiceMapStatefulProcessor( + final ServiceMapProcessorConfig serviceMapProcessorConfig, + final PluginMetrics pluginMetrics, + final PipelineDescription pipelineDescription) { + this((long) serviceMapProcessorConfig.getWindowDuration() * TO_MILLIS, new File(ServiceMapProcessorConfig.DEFAULT_DB_PATH), Clock.systemUTC(), - pluginSetting.getNumberOfProcessWorkers(), - pluginSetting); + pipelineDescription.getNumberOfProcessWorkers(), + pluginMetrics); } - public ServiceMapStatefulProcessor(final long windowDurationMillis, + ServiceMapStatefulProcessor(final long windowDurationMillis, final File databasePath, final Clock clock, final int processWorkers, - final PluginSetting pluginSetting) { - super(pluginSetting); + final PluginMetrics pluginMetrics) { + super(pluginMetrics); ServiceMapStatefulProcessor.clock = clock; this.thisProcessorId = processorsCreated.getAndIncrement(); diff --git a/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfigTest.java b/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfigTest.java new file mode 100644 index 0000000000..35ef3b0c07 --- /dev/null +++ b/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfigTest.java @@ -0,0 +1,38 @@ +package org.opensearch.dataprepper.plugins.processor; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; + +import java.util.Random; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.opensearch.dataprepper.plugins.processor.ServiceMapProcessorConfig.DEFAULT_WINDOW_DURATION; + +class ServiceMapProcessorConfigTest { + private ServiceMapProcessorConfig serviceMapProcessorConfig; + Random random; + + @BeforeEach + void setUp() { + serviceMapProcessorConfig = new ServiceMapProcessorConfig(); + random = new Random(); + } + + @Test + void testDefaultConfig() { + assertThat(serviceMapProcessorConfig.getWindowDuration(), equalTo(DEFAULT_WINDOW_DURATION)); + } + + @Test + void testGetter() throws NoSuchFieldException, IllegalAccessException { + final int windowDuration = 1 + random.nextInt(300); + ReflectivelySetField.setField( + ServiceMapProcessorConfig.class, + serviceMapProcessorConfig, + "windowDuration", + windowDuration); + assertThat(serviceMapProcessorConfig.getWindowDuration(), equalTo(windowDuration)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java b/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java index 28789615aa..b565642e19 100644 --- a/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java +++ b/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java @@ -14,6 +14,8 @@ import org.mockito.Mockito; import org.opensearch.dataprepper.metrics.MetricNames; import org.opensearch.dataprepper.metrics.MetricsTestUtil; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.trace.Span; @@ -43,6 +45,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.processor.ServiceMapProcessorConfig.DEFAULT_WINDOW_DURATION; public class ServiceMapStatefulProcessorTest { @@ -54,12 +57,20 @@ public class ServiceMapStatefulProcessorTest { private static final String PAYMENT_SERVICE = "PAY"; private static final String CART_SERVICE = "CART"; private PluginSetting pluginSetting; + private PluginMetrics pluginMetrics; + private PipelineDescription pipelineDescription; + private ServiceMapProcessorConfig serviceMapProcessorConfig; @BeforeEach public void setup() throws NoSuchFieldException, IllegalAccessException { resetServiceMapStatefulProcessorStatic(); MetricsTestUtil.initMetrics(); pluginSetting = mock(PluginSetting.class); + pipelineDescription = mock(PipelineDescription.class); + serviceMapProcessorConfig = mock(ServiceMapProcessorConfig.class); + when(serviceMapProcessorConfig.getWindowDuration()).thenReturn(DEFAULT_WINDOW_DURATION); + pluginMetrics = PluginMetrics.fromNames( + "testServiceMapProcessor", "testPipelineName"); when(pluginSetting.getName()).thenReturn("testServiceMapProcessor"); when(pluginSetting.getPipelineName()).thenReturn("testPipelineName"); } @@ -116,13 +127,11 @@ private Set evaluateEdges(Set serv } @Test - public void testPluginSettingConstructor() { - - final PluginSetting pluginSetting = new PluginSetting("testPluginSetting", Collections.emptyMap()); - pluginSetting.setProcessWorkers(4); - pluginSetting.setPipelineName("TestPipeline"); + public void testDataPrepperConstructor() { + when(pipelineDescription.getNumberOfProcessWorkers()).thenReturn(4); //Nothing is accessible to validate, so just verify that no exception is thrown. - final ServiceMapStatefulProcessor serviceMapStatefulProcessor = new ServiceMapStatefulProcessor(pluginSetting); + final ServiceMapStatefulProcessor serviceMapStatefulProcessor = new ServiceMapStatefulProcessor( + serviceMapProcessorConfig, pluginMetrics, pipelineDescription); } @Test @@ -132,8 +141,8 @@ public void testTraceGroupsWithEventRecordData() throws Exception { Mockito.when(clock.instant()).thenReturn(Instant.now()); ExecutorService threadpool = Executors.newCachedThreadPool(); final File path = new File(ServiceMapProcessorConfig.DEFAULT_DB_PATH); - final ServiceMapStatefulProcessor serviceMapStateful1 = new ServiceMapStatefulProcessor(100, path, clock, 2, pluginSetting); - final ServiceMapStatefulProcessor serviceMapStateful2 = new ServiceMapStatefulProcessor(100, path, clock, 2, pluginSetting); + final ServiceMapStatefulProcessor serviceMapStateful1 = new ServiceMapStatefulProcessor(100, path, clock, 2, pluginMetrics); + final ServiceMapStatefulProcessor serviceMapStateful2 = new ServiceMapStatefulProcessor(100, path, clock, 2, pluginMetrics); final byte[] rootSpanId1Bytes = ServiceMapTestUtils.getRandomBytes(8); final byte[] rootSpanId2Bytes = ServiceMapTestUtils.getRandomBytes(8); @@ -327,8 +336,8 @@ public void testTraceGroupsWithIsolatedServiceEventRecordData() throws Exception Mockito.when(clock.instant()).thenReturn(Instant.now()); ExecutorService threadpool = Executors.newCachedThreadPool(); final File path = new File(ServiceMapProcessorConfig.DEFAULT_DB_PATH); - final ServiceMapStatefulProcessor serviceMapStateful1 = new ServiceMapStatefulProcessor(100, path, clock, 2, pluginSetting); - final ServiceMapStatefulProcessor serviceMapStateful2 = new ServiceMapStatefulProcessor(100, path, clock, 2, pluginSetting); + final ServiceMapStatefulProcessor serviceMapStateful1 = new ServiceMapStatefulProcessor(100, path, clock, 2, pluginMetrics); + final ServiceMapStatefulProcessor serviceMapStateful2 = new ServiceMapStatefulProcessor(100, path, clock, 2, pluginMetrics); final byte[] rootSpanIdBytes = ServiceMapTestUtils.getRandomBytes(8); final byte[] traceIdBytes = ServiceMapTestUtils.getRandomBytes(16); @@ -383,7 +392,7 @@ public void testTraceGroupsWithIsolatedServiceEventRecordData() throws Exception @Test public void testPrepareForShutdownWithEventRecordData() { final File path = new File(ServiceMapProcessorConfig.DEFAULT_DB_PATH); - final ServiceMapStatefulProcessor serviceMapStateful = new ServiceMapStatefulProcessor(100, path, Clock.systemUTC(), 1, pluginSetting); + final ServiceMapStatefulProcessor serviceMapStateful = new ServiceMapStatefulProcessor(100, path, Clock.systemUTC(), 1, pluginMetrics); final byte[] rootSpanId1Bytes = ServiceMapTestUtils.getRandomBytes(8); final byte[] traceId1Bytes = ServiceMapTestUtils.getRandomBytes(16); @@ -411,11 +420,9 @@ public void testPrepareForShutdownWithEventRecordData() { @Test public void testGetIdentificationKeys() { - final PluginSetting pluginSetting = new PluginSetting("testPluginSetting", Collections.emptyMap()); - pluginSetting.setProcessWorkers(4); - pluginSetting.setPipelineName("TestPipeline"); - - final ServiceMapStatefulProcessor serviceMapStatefulProcessor = new ServiceMapStatefulProcessor(pluginSetting); + when(pipelineDescription.getNumberOfProcessWorkers()).thenReturn(4); + final ServiceMapStatefulProcessor serviceMapStatefulProcessor = new ServiceMapStatefulProcessor( + serviceMapProcessorConfig, pluginMetrics, pipelineDescription); final Collection expectedIdentificationKeys = serviceMapStatefulProcessor.getIdentificationKeys(); assertThat(expectedIdentificationKeys, equalTo(Collections.singleton("traceId"))); From c09a8ee9dd43ab0c0f23ec384640122c265c6def Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 15 Jul 2024 13:55:11 -0500 Subject: [PATCH 077/159] Updating the Python dependencies to resolve CVEs. Resolves #4715, #4713, 4714. (#4733) Signed-off-by: David Venable --- .../sample-app/requirements.txt | 10 +++++----- .../otel-span-exporter/requirements.txt | 18 +++++++++--------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/examples/trace-analytics-sample-app/sample-app/requirements.txt b/examples/trace-analytics-sample-app/sample-app/requirements.txt index df780b836b..a24bef87af 100644 --- a/examples/trace-analytics-sample-app/sample-app/requirements.txt +++ b/examples/trace-analytics-sample-app/sample-app/requirements.txt @@ -1,10 +1,10 @@ dash==2.15.0 mysql-connector==2.2.9 -opentelemetry-exporter-otlp==1.20.0 -opentelemetry-instrumentation-flask==0.41b0 -opentelemetry-instrumentation-mysql==0.41b0 -opentelemetry-instrumentation-requests==0.41b0 -opentelemetry-sdk==1.20.0 +opentelemetry-exporter-otlp==1.25.0 +opentelemetry-instrumentation-flask==0.46b0 +opentelemetry-instrumentation-mysql==0.46b0 +opentelemetry-instrumentation-requests==0.46b0 +opentelemetry-sdk==1.25.0 protobuf==3.20.3 urllib3==2.2.2 werkzeug==3.0.3 \ No newline at end of file diff --git a/release/smoke-tests/otel-span-exporter/requirements.txt b/release/smoke-tests/otel-span-exporter/requirements.txt index 6968658846..f2e5b97c35 100644 --- a/release/smoke-tests/otel-span-exporter/requirements.txt +++ b/release/smoke-tests/otel-span-exporter/requirements.txt @@ -1,17 +1,17 @@ backoff==1.10.0 -certifi==2023.7.22 +certifi==2024.07.04 charset-normalizer==2.0.9 Deprecated==1.2.13 googleapis-common-protos==1.53.0 grpcio==1.53.2 -idna==3.3 -opentelemetry-api==1.7.1 -opentelemetry-exporter-otlp==1.7.1 -opentelemetry-exporter-otlp-proto-grpc==1.7.1 -opentelemetry-exporter-otlp-proto-http==1.7.1 -opentelemetry-proto==1.7.1 -opentelemetry-sdk==1.7.1 -opentelemetry-semantic-conventions==0.26b1 +idna==3.7 +opentelemetry-api==1.25.0 +opentelemetry-exporter-otlp==1.25.0 +opentelemetry-exporter-otlp-proto-grpc==1.25.0 +opentelemetry-exporter-otlp-proto-http==1.25.0 +opentelemetry-proto==1.25.0 +opentelemetry-sdk==1.25.0 +opentelemetry-semantic-conventions==0.46b0 protobuf==3.19.5 requests==2.32.3 six==1.16.0 From 418a2a5e82180797d0b4d7a6fe7afac3e261e3e5 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 17 Jul 2024 17:49:55 -0500 Subject: [PATCH 078/159] Updates Jackson to 2.17.2. Related to #4729. (#4744) Signed-off-by: David Venable --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 7d7c939d34..3dccd497cf 100644 --- a/build.gradle +++ b/build.gradle @@ -69,7 +69,7 @@ subprojects { } } dependencies { - implementation platform('com.fasterxml.jackson:jackson-bom:2.16.1') + implementation platform('com.fasterxml.jackson:jackson-bom:2.17.2') implementation platform('org.eclipse.jetty:jetty-bom:9.4.53.v20231009') implementation platform('io.micrometer:micrometer-bom:1.10.5') implementation libs.guava.core From aa58f3ab682cce7887988f776cf02b4f886b10f5 Mon Sep 17 00:00:00 2001 From: Katherine Shen <40495707+shenkw1@users.noreply.github.com> Date: Fri, 19 Jul 2024 12:04:50 -0500 Subject: [PATCH 079/159] MAINT: add json property descriptions for kv configs (#4747) add json property descriptions for kv configs Signed-off-by: Katherine Shen --- .../keyvalue/KeyValueProcessorConfig.java | 80 ++++++++++++++++++- 1 file changed, 79 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java index 84cdb868e9..bcc8eb0a27 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.keyvalue; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.AssertTrue; @@ -35,87 +36,163 @@ public class KeyValueProcessorConfig { static final boolean DEFAULT_RECURSIVE = false; @NotEmpty + @JsonPropertyDescription("The message field to be parsed. Optional. Default value is `message`.") private String source = DEFAULT_SOURCE; + @JsonPropertyDescription("The destination field for the parsed source. The parsed source overwrites the " + + "preexisting data for that key. Optional. If `destination` is set to `null`, the parsed fields will be " + + "written to the root of the event. Default value is `parsed_message`.") private String destination = DEFAULT_DESTINATION; @JsonProperty("field_delimiter_regex") + @JsonPropertyDescription("A regular expression specifying the delimiter that separates key-value pairs. " + + "Special regular expression characters such as `[` and `]` must be escaped with `\\\\`. " + + "Cannot be defined at the same time as `field_split_characters`. Optional. " + + "If this option is not defined, `field_split_characters` is used.") private String fieldDelimiterRegex; @JsonProperty("field_split_characters") + @JsonPropertyDescription("A string of characters specifying the delimiter that separates key-value pairs. " + + "Special regular expression characters such as `[` and `]` must be escaped with `\\\\`. " + + "Cannot be defined at the same time as `field_delimiter_regex`. Optional. Default value is `&`.") private String fieldSplitCharacters = DEFAULT_FIELD_SPLIT_CHARACTERS; @JsonProperty("include_keys") + @JsonPropertyDescription("An array specifying the keys that should be added for parsing. " + + "By default, all keys will be added.") @NotNull private List includeKeys = DEFAULT_INCLUDE_KEYS; @JsonProperty("exclude_keys") + @JsonPropertyDescription("An array specifying the parsed keys that should not be added to the event. " + + "By default, no keys will be excluded.") @NotNull private List excludeKeys = DEFAULT_EXCLUDE_KEYS; @JsonProperty("default_values") + @JsonPropertyDescription("A map specifying the default keys and their values that should be added " + + "to the event in case these keys do not exist in the source field being parsed. " + + "If the default key already exists in the message, the value is not changed. " + + "The `include_keys` filter will be applied to the message before `default_values`.") @NotNull private Map defaultValues = DEFAULT_DEFAULT_VALUES; @JsonProperty("key_value_delimiter_regex") + @JsonPropertyDescription("A regular expression specifying the delimiter that separates the key and value " + + "within a key-value pair. Special regular expression characters such as `[` and `]` must be escaped with " + + "`\\\\`. This option cannot be defined at the same time as `value_split_characters`. Optional. " + + "If this option is not defined, `value_split_characters` is used.") private String keyValueDelimiterRegex; @JsonProperty("value_split_characters") + @JsonPropertyDescription("A string of characters specifying the delimiter that separates the key and value within " + + "a key-value pair. Special regular expression characters such as `[` and `]` must be escaped with `\\\\`. " + + "Cannot be defined at the same time as `key_value_delimiter_regex`. Optional. Default value is `=`.") private String valueSplitCharacters = DEFAULT_VALUE_SPLIT_CHARACTERS; @JsonProperty("non_match_value") + @JsonPropertyDescription("When a key-value pair cannot be successfully split, the key-value pair is " + + "placed in the `key` field, and the specified value is placed in the `value` field. " + + "Optional. Default value is `null`.") private Object nonMatchValue = DEFAULT_NON_MATCH_VALUE; + @JsonPropertyDescription("A prefix to append before all keys. Optional. Default value is an empty string.") @NotNull private String prefix = DEFAULT_PREFIX; @JsonProperty("delete_key_regex") + @JsonPropertyDescription("A regular expression specifying the characters to delete from the key. " + + "Special regular expression characters such as `[` and `]` must be escaped with `\\\\`. Cannot be an " + + "empty string. Optional. No default value.") @NotNull private String deleteKeyRegex = DEFAULT_DELETE_KEY_REGEX; @JsonProperty("delete_value_regex") + @JsonPropertyDescription("A regular expression specifying the characters to delete from the value. " + + "Special regular expression characters such as `[` and `]` must be escaped with `\\\\`. " + + "Cannot be an empty string. Optional. No default value.") @NotNull private String deleteValueRegex = DEFAULT_DELETE_VALUE_REGEX; @JsonProperty("transform_key") + @JsonPropertyDescription("When to lowercase, uppercase, or capitalize keys.") @NotNull private String transformKey = DEFAULT_TRANSFORM_KEY; @JsonProperty("whitespace") + @JsonPropertyDescription("Specifies whether to be lenient or strict with the acceptance of " + + "unnecessary white space surrounding the configured value-split sequence. Default is `lenient`.") @NotNull private String whitespace = DEFAULT_WHITESPACE; @JsonProperty("skip_duplicate_values") + @JsonPropertyDescription("A Boolean option for removing duplicate key-value pairs. When set to `true`, " + + "only one unique key-value pair will be preserved. Default is `false`.") @NotNull private boolean skipDuplicateValues = DEFAULT_SKIP_DUPLICATE_VALUES; @JsonProperty("remove_brackets") + @JsonPropertyDescription("Specifies whether to treat square brackets, angle brackets, and parentheses " + + "as value “wrappers” that should be removed from the value. Default is `false`.") @NotNull private boolean removeBrackets = DEFAULT_REMOVE_BRACKETS; @JsonProperty("value_grouping") + @JsonPropertyDescription("Specifies whether to group values using predefined value grouping delimiters: " + + "`{...}`, `[...]`, `<...>`, `(...)`, `\"...\"`, `'...'`, `http://... (space)`, and `https:// (space)`. " + + "If this flag is enabled, then the content between the delimiters is considered to be one entity and " + + "is not parsed for key-value pairs. Default is `false`. If `value_grouping` is `true`, then " + + "`{\"key1=[a=b,c=d]&key2=value2\"}` parses to `{\"key1\": \"[a=b,c=d]\", \"key2\": \"value2\"}`.") private boolean valueGrouping = DEFAULT_VALUE_GROUPING; @JsonProperty("recursive") + @JsonPropertyDescription("Specifies whether to recursively obtain additional key-value pairs from values. " + + "The extra key-value pairs will be stored as sub-keys of the root key. Default is `false`. " + + "The levels of recursive parsing must be defined by different brackets for each level: " + + "`[]`, `()`, and `<>`, in this order. Any other configurations specified will only be applied " + + "to the outmost keys.\n" + + "When `recursive` is `true`:\n" + + "`remove_brackets` cannot also be `true`;\n" + + "`skip_duplicate_values` will always be `true`;\n" + + "`whitespace` will always be `\"strict\"`.") @NotNull private boolean recursive = DEFAULT_RECURSIVE; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("When a `kv` operation causes a runtime exception within the processor, " + + "the operation is safely stopped without crashing the processor, and the event is tagged " + + "with the provided tags.") private List tagsOnFailure; @JsonProperty("overwrite_if_destination_exists") + @JsonPropertyDescription("Specifies whether to overwrite existing fields if there are key conflicts " + + "when writing parsed fields to the event. Default is `true`.") private boolean overwriteIfDestinationExists = true; @JsonProperty("drop_keys_with_no_value") + @JsonPropertyDescription("Specifies whether keys should be dropped if they have a null value. Default is `false`. " + + "If `drop_keys_with_no_value` is set to `true`, " + + "then `{\"key1=value1&key2\"}` parses to `{\"key1\": \"value1\"}`.") private boolean dropKeysWithNoValue = false; @JsonProperty("key_value_when") + @JsonPropertyDescription("Allows you to specify a [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"`, that will be evaluated to determine whether " + + "the processor should be applied to the event.") private String keyValueWhen; @JsonProperty("strict_grouping") + @JsonPropertyDescription("When enabled, groups with unmatched end characters yield errors. " + + "The event is ignored after the errors are logged. " + + "Specifies whether strict grouping should be enabled when the `value_grouping` " + + "or `string_literal_character` options are used. Default is `false`.") private boolean strictGrouping = false; @JsonProperty("string_literal_character") + @JsonPropertyDescription("When this option is used, any text contained within the specified quotation " + + "mark character will be ignored and excluded from key-value parsing. " + + "Can be set to either a single quotation mark (`'`) or a double quotation mark (`\"`). " + + "Default is `null`.") @Size(min = 0, max = 1, message = "string_literal_character may only have character") private String stringLiteralCharacter = null; @@ -124,7 +201,8 @@ boolean isValidValueGroupingAndFieldDelimiterRegex() { return (!valueGrouping || fieldDelimiterRegex == null); } - @AssertTrue(message = "Invalid Configuration. String literal character config is valid only when value_grouping is enabled, and only double quote (\") and single quote are (') are valid string literal characters.") + @AssertTrue(message = "Invalid Configuration. String literal character config is valid only when value_grouping is enabled, " + + "and only double quote (\") and single quote are (') are valid string literal characters.") boolean isValidStringLiteralConfig() { if (stringLiteralCharacter == null) return true; From afe84648d662ddc469a6cbf9f22ccd6ff9dfda2c Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 19 Jul 2024 13:40:57 -0500 Subject: [PATCH 080/159] Improve the SQS shutdown process such that it does not prevent the pipeline from shutting down and no longer results in failures. Resolves #4575 (#4748) The previous approach to shutting down the SQS thread closed the SqsClient. However, with acknowledgments enabled, asynchronous callbacks would result in further attempts to either ChangeVisibilityTimeout or DeleteMessages. These were failing because the client was closed. Also, the threads would remain and prevent Data Prepper from correctly shutting down. With this change, we correctly stop each processing thread. Then we close the client. Additionally, the SqsWorker now checks that it is not stopped before attempting to change the message visibility or delete messages. Additionally, I found some missing test cases. Also, modifying this code and especially unit testing it is becoming more difficult, so I performed some refactoring to move message parsing out of the SqsWorker. Signed-off-by: David Venable --- .../plugins/source/s3/SqsService.java | 22 +- .../plugins/source/s3/SqsWorker.java | 66 ++-- .../source/s3/parser/ParsedMessage.java | 17 +- .../source/s3/parser/SqsMessageParser.java | 44 +++ .../plugins/source/s3/SqsWorkerTest.java | 320 +++++++++++------- .../source/s3/parser/ParsedMessageTest.java | 222 ++++++++---- .../S3EventBridgeNotificationParserTest.java | 2 +- .../parser/S3EventNotificationParserTest.java | 6 +- .../s3/parser/SqsMessageParserTest.java | 96 ++++++ 9 files changed, 559 insertions(+), 236 deletions(-) create mode 100644 data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/SqsMessageParser.java create mode 100644 data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/SqsMessageParserTest.java diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsService.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsService.java index b05d2806d4..c674be5f68 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsService.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsService.java @@ -17,9 +17,12 @@ import software.amazon.awssdk.services.sqs.SqsClient; import java.time.Duration; +import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.Executors; import java.util.concurrent.ExecutorService; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class SqsService { private static final Logger LOG = LoggerFactory.getLogger(SqsService.class); @@ -34,6 +37,7 @@ public class SqsService { private final PluginMetrics pluginMetrics; private final AcknowledgementSetManager acknowledgementSetManager; private final ExecutorService executorService; + private final List sqsWorkers; public SqsService(final AcknowledgementSetManager acknowledgementSetManager, final S3SourceConfig s3SourceConfig, @@ -46,18 +50,20 @@ public SqsService(final AcknowledgementSetManager acknowledgementSetManager, this.acknowledgementSetManager = acknowledgementSetManager; this.sqsClient = createSqsClient(credentialsProvider); executorService = Executors.newFixedThreadPool(s3SourceConfig.getNumWorkers(), BackgroundThreadFactory.defaultExecutorThreadFactory("s3-source-sqs")); - } - public void start() { final Backoff backoff = Backoff.exponential(INITIAL_DELAY, MAXIMUM_DELAY).withJitter(JITTER_RATE) .withMaxAttempts(Integer.MAX_VALUE); - for (int i = 0; i < s3SourceConfig.getNumWorkers(); i++) { - executorService.submit(new SqsWorker(acknowledgementSetManager, sqsClient, s3Accessor, s3SourceConfig, pluginMetrics, backoff)); - } + sqsWorkers = IntStream.range(0, s3SourceConfig.getNumWorkers()) + .mapToObj(i -> new SqsWorker(acknowledgementSetManager, sqsClient, s3Accessor, s3SourceConfig, pluginMetrics, backoff)) + .collect(Collectors.toList()); + } + + public void start() { + sqsWorkers.forEach(executorService::submit); } SqsClient createSqsClient(final AwsCredentialsProvider credentialsProvider) { - LOG.info("Creating SQS client"); + LOG.debug("Creating SQS client"); return SqsClient.builder() .region(s3SourceConfig.getAwsAuthenticationOptions().getAwsRegion()) .credentialsProvider(credentialsProvider) @@ -68,8 +74,8 @@ SqsClient createSqsClient(final AwsCredentialsProvider credentialsProvider) { } public void stop() { - sqsClient.close(); executorService.shutdown(); + sqsWorkers.forEach(SqsWorker::stop); try { if (!executorService.awaitTermination(SHUTDOWN_TIMEOUT, TimeUnit.SECONDS)) { LOG.warn("Failed to terminate SqsWorkers"); @@ -82,5 +88,7 @@ public void stop() { Thread.currentThread().interrupt(); } } + + sqsClient.close(); } } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java index b3404cebf6..3c5fba0701 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java @@ -5,7 +5,6 @@ package org.opensearch.dataprepper.plugins.source.s3; -import com.fasterxml.jackson.databind.ObjectMapper; import com.linecorp.armeria.client.retry.Backoff; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.Timer; @@ -20,8 +19,7 @@ import org.opensearch.dataprepper.plugins.source.s3.filter.S3EventFilter; import org.opensearch.dataprepper.plugins.source.s3.filter.S3ObjectCreatedFilter; import org.opensearch.dataprepper.plugins.source.s3.parser.ParsedMessage; -import org.opensearch.dataprepper.plugins.source.s3.parser.S3EventBridgeNotificationParser; -import org.opensearch.dataprepper.plugins.source.s3.parser.S3EventNotificationParser; +import org.opensearch.dataprepper.plugins.source.s3.parser.SqsMessageParser; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.exception.SdkException; @@ -75,11 +73,10 @@ public class SqsWorker implements Runnable { private final Counter sqsVisibilityTimeoutChangeFailedCount; private final Timer sqsMessageDelayTimer; private final Backoff standardBackoff; + private final SqsMessageParser sqsMessageParser; private int failedAttemptCount; private final boolean endToEndAcknowledgementsEnabled; private final AcknowledgementSetManager acknowledgementSetManager; - - private final ObjectMapper objectMapper = new ObjectMapper(); private volatile boolean isStopped = false; private Map parsedMessageVisibilityTimesMap; @@ -98,6 +95,7 @@ public SqsWorker(final AcknowledgementSetManager acknowledgementSetManager, sqsOptions = s3SourceConfig.getSqsOptions(); objectCreatedFilter = new S3ObjectCreatedFilter(); evenBridgeObjectCreatedFilter = new EventBridgeObjectCreatedFilter(); + sqsMessageParser = new SqsMessageParser(s3SourceConfig); failedAttemptCount = 0; parsedMessageVisibilityTimesMap = new HashMap<>(); @@ -139,7 +137,7 @@ int processSqsMessages() { if (!sqsMessages.isEmpty()) { sqsMessagesReceivedCounter.increment(sqsMessages.size()); - final Collection s3MessageEventNotificationRecords = getS3MessageEventNotificationRecords(sqsMessages); + final Collection s3MessageEventNotificationRecords = sqsMessageParser.parseSqsMessages(sqsMessages); // build s3ObjectReference from S3EventNotificationRecord if event name starts with ObjectCreated final List deleteMessageBatchRequestEntries = processS3EventNotificationRecords(s3MessageEventNotificationRecords); @@ -191,22 +189,6 @@ private ReceiveMessageRequest createReceiveMessageRequest() { .build(); } - private Collection getS3MessageEventNotificationRecords(final List sqsMessages) { - return sqsMessages.stream() - .map(this::convertS3EventMessages) - .collect(Collectors.toList()); - } - - private ParsedMessage convertS3EventMessages(final Message message) { - if (s3SourceConfig.getNotificationSource().equals(NotificationSourceOption.S3)) { - return new S3EventNotificationParser().parseMessage(message, objectMapper); - } - else if (s3SourceConfig.getNotificationSource().equals(NotificationSourceOption.EVENTBRIDGE)) { - return new S3EventBridgeNotificationParser().parseMessage(message, objectMapper); - } - return new ParsedMessage(message, true); - } - private List processS3EventNotificationRecords(final Collection s3EventNotificationRecords) { final List deleteMessageBatchRequestEntryCollection = new ArrayList<>(); final List parsedMessagesToRead = new ArrayList<>(); @@ -276,21 +258,7 @@ && isEventBridgeEventTypeCreated(parsedMessage)) { return; } parsedMessageVisibilityTimesMap.put(parsedMessage, newValue); - final ChangeMessageVisibilityRequest changeMessageVisibilityRequest = ChangeMessageVisibilityRequest.builder() - .visibilityTimeout(newVisibilityTimeoutSeconds) - .queueUrl(sqsOptions.getSqsUrl()) - .receiptHandle(parsedMessage.getMessage().receiptHandle()) - .build(); - - try { - sqsClient.changeMessageVisibility(changeMessageVisibilityRequest); - sqsVisibilityTimeoutChangedCount.increment(); - LOG.debug("Set visibility timeout for message {} to {}", parsedMessage.getMessage().messageId(), newVisibilityTimeoutSeconds); - } catch (Exception e) { - LOG.error("Failed to set visibility timeout for message {} to {}", parsedMessage.getMessage().messageId(), newVisibilityTimeoutSeconds, e); - sqsVisibilityTimeoutChangeFailedCount.increment(); - } - + increaseVisibilityTimeout(parsedMessage, newVisibilityTimeoutSeconds); }, Duration.ofSeconds(progressCheckInterval)); } @@ -308,6 +276,27 @@ && isEventBridgeEventTypeCreated(parsedMessage)) { return deleteMessageBatchRequestEntryCollection; } + private void increaseVisibilityTimeout(final ParsedMessage parsedMessage, final int newVisibilityTimeoutSeconds) { + if(isStopped) { + LOG.info("Some messages are pending completion of acknowledgments. Data Prepper will not increase the visibility timeout because it is shutting down. {}", parsedMessage); + return; + } + final ChangeMessageVisibilityRequest changeMessageVisibilityRequest = ChangeMessageVisibilityRequest.builder() + .visibilityTimeout(newVisibilityTimeoutSeconds) + .queueUrl(sqsOptions.getSqsUrl()) + .receiptHandle(parsedMessage.getMessage().receiptHandle()) + .build(); + + try { + sqsClient.changeMessageVisibility(changeMessageVisibilityRequest); + sqsVisibilityTimeoutChangedCount.increment(); + LOG.debug("Set visibility timeout for message {} to {}", parsedMessage.getMessage().messageId(), newVisibilityTimeoutSeconds); + } catch (Exception e) { + LOG.error("Failed to set visibility timeout for message {} to {}", parsedMessage.getMessage().messageId(), newVisibilityTimeoutSeconds, e); + sqsVisibilityTimeoutChangeFailedCount.increment(); + } + } + private Optional processS3Object( final ParsedMessage parsedMessage, final S3ObjectReference s3ObjectReference, @@ -328,6 +317,8 @@ private Optional processS3Object( } private void deleteSqsMessages(final List deleteMessageBatchRequestEntryCollection) { + if(isStopped) + return; if (deleteMessageBatchRequestEntryCollection.size() == 0) { return; } @@ -396,6 +387,5 @@ private S3ObjectReference populateS3Reference(final String bucketName, final Str void stop() { isStopped = true; - Thread.currentThread().interrupt(); } } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessage.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessage.java index 18bbc58499..ed68dff063 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessage.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessage.java @@ -11,6 +11,7 @@ import software.amazon.awssdk.services.sqs.model.Message; import java.util.List; +import java.util.Objects; public class ParsedMessage { private final Message message; @@ -24,14 +25,14 @@ public class ParsedMessage { private String detailType; public ParsedMessage(final Message message, final boolean failedParsing) { - this.message = message; + this.message = Objects.requireNonNull(message); this.failedParsing = failedParsing; this.emptyNotification = true; } - // S3EventNotification contains only one S3EventNotificationRecord ParsedMessage(final Message message, final List notificationRecords) { - this.message = message; + this.message = Objects.requireNonNull(message); + // S3EventNotification contains only one S3EventNotificationRecord this.bucketName = notificationRecords.get(0).getS3().getBucket().getName(); this.objectKey = notificationRecords.get(0).getS3().getObject().getUrlDecodedKey(); this.objectSize = notificationRecords.get(0).getS3().getObject().getSizeAsLong(); @@ -42,7 +43,7 @@ public ParsedMessage(final Message message, final boolean failedParsing) { } ParsedMessage(final Message message, final S3EventBridgeNotification eventBridgeNotification) { - this.message = message; + this.message = Objects.requireNonNull(message); this.bucketName = eventBridgeNotification.getDetail().getBucket().getName(); this.objectKey = eventBridgeNotification.getDetail().getObject().getUrlDecodedKey(); this.objectSize = eventBridgeNotification.getDetail().getObject().getSize(); @@ -85,4 +86,12 @@ public boolean isEmptyNotification() { public String getDetailType() { return detailType; } + + @Override + public String toString() { + return "Message{" + + "messageId=" + message.messageId() + + ", objectKey=" + objectKey + + '}'; + } } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/SqsMessageParser.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/SqsMessageParser.java new file mode 100644 index 0000000000..ea40e3f041 --- /dev/null +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/SqsMessageParser.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.s3.parser; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.opensearch.dataprepper.plugins.source.s3.S3SourceConfig; +import software.amazon.awssdk.services.sqs.model.Message; + +import java.util.Collection; +import java.util.stream.Collectors; + +public class SqsMessageParser { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private final S3SourceConfig s3SourceConfig; + private final S3NotificationParser s3NotificationParser; + + public SqsMessageParser(final S3SourceConfig s3SourceConfig) { + this.s3SourceConfig = s3SourceConfig; + s3NotificationParser = createNotificationParser(s3SourceConfig); + } + + public Collection parseSqsMessages(final Collection sqsMessages) { + return sqsMessages.stream() + .map(this::convertS3EventMessages) + .collect(Collectors.toList()); + } + + private ParsedMessage convertS3EventMessages(final Message message) { + return s3NotificationParser.parseMessage(message, OBJECT_MAPPER); + } + + private static S3NotificationParser createNotificationParser(final S3SourceConfig s3SourceConfig) { + switch (s3SourceConfig.getNotificationSource()) { + case EVENTBRIDGE: + return new S3EventBridgeNotificationParser(); + case S3: + default: + return new S3EventNotificationParser(); + } + } +} diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerTest.java index 50ed879f4a..ada789cea6 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerTest.java @@ -12,6 +12,7 @@ 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.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -19,19 +20,21 @@ import org.junit.jupiter.params.provider.ArgumentsSource; import org.junit.jupiter.params.provider.ValueSource; import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; -import org.opensearch.dataprepper.plugins.source.s3.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.model.acknowledgements.ProgressCheck; import org.opensearch.dataprepper.plugins.source.s3.configuration.NotificationSourceOption; import org.opensearch.dataprepper.plugins.source.s3.configuration.OnErrorOption; import org.opensearch.dataprepper.plugins.source.s3.configuration.SqsOptions; import org.opensearch.dataprepper.plugins.source.s3.exception.SqsRetriesExhaustedException; import org.opensearch.dataprepper.plugins.source.s3.filter.S3EventFilter; import org.opensearch.dataprepper.plugins.source.s3.filter.S3ObjectCreatedFilter; -import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.sqs.SqsClient; import software.amazon.awssdk.services.sqs.model.BatchResultErrorEntry; +import software.amazon.awssdk.services.sqs.model.ChangeMessageVisibilityRequest; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResultEntry; @@ -50,6 +53,7 @@ import java.util.Collections; import java.util.List; import java.util.UUID; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.stream.Stream; @@ -65,20 +69,23 @@ import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; +import static org.mockito.Mockito.reset; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.s3.SqsWorker.ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME; +import static org.opensearch.dataprepper.plugins.source.s3.SqsWorker.S3_OBJECTS_EMPTY_METRIC_NAME; import static org.opensearch.dataprepper.plugins.source.s3.SqsWorker.SQS_MESSAGES_DELETED_METRIC_NAME; import static org.opensearch.dataprepper.plugins.source.s3.SqsWorker.SQS_MESSAGES_DELETE_FAILED_METRIC_NAME; import static org.opensearch.dataprepper.plugins.source.s3.SqsWorker.SQS_MESSAGES_FAILED_METRIC_NAME; import static org.opensearch.dataprepper.plugins.source.s3.SqsWorker.SQS_MESSAGES_RECEIVED_METRIC_NAME; import static org.opensearch.dataprepper.plugins.source.s3.SqsWorker.SQS_MESSAGE_DELAY_METRIC_NAME; -import static org.opensearch.dataprepper.plugins.source.s3.SqsWorker.S3_OBJECTS_EMPTY_METRIC_NAME; +import static org.opensearch.dataprepper.plugins.source.s3.SqsWorker.SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME; +@ExtendWith(MockitoExtension.class) class SqsWorkerTest { - private SqsWorker sqsWorker; private SqsClient sqsClient; private S3Service s3Service; private S3SourceConfig s3SourceConfig; @@ -90,10 +97,13 @@ class SqsWorkerTest { private Counter sqsMessagesFailedCounter; private Counter sqsMessagesDeleteFailedCounter; private Counter s3ObjectsEmptyCounter; + @Mock + private Counter sqsVisibilityTimeoutChangedCount; private Timer sqsMessageDelayTimer; private AcknowledgementSetManager acknowledgementSetManager; private AcknowledgementSet acknowledgementSet; private SqsOptions sqsOptions; + private String queueUrl; @BeforeEach void setUp() { @@ -105,15 +115,11 @@ void setUp() { objectCreatedFilter = new S3ObjectCreatedFilter(); backoff = mock(Backoff.class); - AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); - when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.US_EAST_1); - sqsOptions = mock(SqsOptions.class); - when(sqsOptions.getSqsUrl()).thenReturn("https://sqs.us-east-2.amazonaws.com/123456789012/MyQueue"); + queueUrl = "https://sqs.us-east-2.amazonaws.com/123456789012/" + UUID.randomUUID(); + when(sqsOptions.getSqsUrl()).thenReturn(queueUrl); - when(s3SourceConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); when(s3SourceConfig.getSqsOptions()).thenReturn(sqsOptions); - when(s3SourceConfig.getOnErrorOption()).thenReturn(OnErrorOption.RETAIN_MESSAGES); when(s3SourceConfig.getAcknowledgements()).thenReturn(false); when(s3SourceConfig.getNotificationSource()).thenReturn(NotificationSourceOption.S3); @@ -130,8 +136,12 @@ void setUp() { when(pluginMetrics.counter(SQS_MESSAGES_DELETE_FAILED_METRIC_NAME)).thenReturn(sqsMessagesDeleteFailedCounter); when(pluginMetrics.counter(S3_OBJECTS_EMPTY_METRIC_NAME)).thenReturn(s3ObjectsEmptyCounter); when(pluginMetrics.timer(SQS_MESSAGE_DELAY_METRIC_NAME)).thenReturn(sqsMessageDelayTimer); + when(pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME)).thenReturn(mock(Counter.class)); + when(pluginMetrics.counter(SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME)).thenReturn(sqsVisibilityTimeoutChangedCount); + } - sqsWorker = new SqsWorker(acknowledgementSetManager, sqsClient, s3Service, s3SourceConfig, pluginMetrics, backoff); + private SqsWorker createObjectUnderTest() { + return new SqsWorker(acknowledgementSetManager, sqsClient, s3Service, s3SourceConfig, pluginMetrics, backoff); } @AfterEach @@ -167,7 +177,7 @@ void processSqsMessages_should_return_number_of_messages_processed(final String when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); verify(sqsClient).deleteMessageBatch(deleteMessageBatchRequestArgumentCaptor.capture()); final DeleteMessageBatchRequest actualDeleteMessageBatchRequest = deleteMessageBatchRequestArgumentCaptor.getValue(); @@ -190,93 +200,6 @@ void processSqsMessages_should_return_number_of_messages_processed(final String assertThat(actualDelay, greaterThanOrEqualTo(Duration.ofHours(1).minus(Duration.ofSeconds(5)))); } - @ParameterizedTest - @ValueSource(strings = {"ObjectCreated:Put", "ObjectCreated:Post", "ObjectCreated:Copy", "ObjectCreated:CompleteMultipartUpload"}) - void processSqsMessages_should_return_number_of_messages_processed_with_acknowledgements(final String eventName) throws IOException { - when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); - when(s3SourceConfig.getAcknowledgements()).thenReturn(true); - sqsWorker = new SqsWorker(acknowledgementSetManager, sqsClient, s3Service, s3SourceConfig, pluginMetrics, backoff); - Instant startTime = Instant.now().minus(1, ChronoUnit.HOURS); - final Message message = mock(Message.class); - when(message.body()).thenReturn(createEventNotification(eventName, startTime)); - final String testReceiptHandle = UUID.randomUUID().toString(); - when(message.messageId()).thenReturn(testReceiptHandle); - when(message.receiptHandle()).thenReturn(testReceiptHandle); - - final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); - when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - - final int messagesProcessed = sqsWorker.processSqsMessages(); - final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); - - final ArgumentCaptor durationArgumentCaptor = ArgumentCaptor.forClass(Duration.class); - verify(sqsMessageDelayTimer).record(durationArgumentCaptor.capture()); - Duration actualDelay = durationArgumentCaptor.getValue(); - - assertThat(messagesProcessed, equalTo(1)); - verify(s3Service).addS3Object(any(S3ObjectReference.class), any()); - verify(acknowledgementSetManager).create(any(), any(Duration.class)); - verify(sqsMessagesReceivedCounter).increment(1); - verifyNoInteractions(sqsMessagesDeletedCounter); - assertThat(actualDelay, lessThanOrEqualTo(Duration.ofHours(1).plus(Duration.ofSeconds(5)))); - assertThat(actualDelay, greaterThanOrEqualTo(Duration.ofHours(1).minus(Duration.ofSeconds(5)))); - } - - @ParameterizedTest - @ValueSource(strings = {"ObjectCreated:Put", "ObjectCreated:Post", "ObjectCreated:Copy", "ObjectCreated:CompleteMultipartUpload"}) - void processSqsMessages_should_return_number_of_messages_processed_with_acknowledgements_and_progress_check(final String eventName) throws IOException { - when(sqsOptions.getVisibilityDuplicateProtection()).thenReturn(true); - when(sqsOptions.getVisibilityTimeout()).thenReturn(Duration.ofSeconds(6)); - when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); - when(s3SourceConfig.getAcknowledgements()).thenReturn(true); - sqsWorker = new SqsWorker(acknowledgementSetManager, sqsClient, s3Service, s3SourceConfig, pluginMetrics, backoff); - Instant startTime = Instant.now().minus(1, ChronoUnit.HOURS); - final Message message = mock(Message.class); - when(message.body()).thenReturn(createEventNotification(eventName, startTime)); - final String testReceiptHandle = UUID.randomUUID().toString(); - when(message.messageId()).thenReturn(testReceiptHandle); - when(message.receiptHandle()).thenReturn(testReceiptHandle); - - final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); - when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - - final int messagesProcessed = sqsWorker.processSqsMessages(); - final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); - - final ArgumentCaptor durationArgumentCaptor = ArgumentCaptor.forClass(Duration.class); - verify(sqsMessageDelayTimer).record(durationArgumentCaptor.capture()); - Duration actualDelay = durationArgumentCaptor.getValue(); - - assertThat(messagesProcessed, equalTo(1)); - verify(s3Service).addS3Object(any(S3ObjectReference.class), any()); - verify(acknowledgementSetManager).create(any(), any(Duration.class)); - verify(acknowledgementSet).addProgressCheck(any(), any(Duration.class)); - verify(sqsMessagesReceivedCounter).increment(1); - verifyNoInteractions(sqsMessagesDeletedCounter); - assertThat(actualDelay, lessThanOrEqualTo(Duration.ofHours(1).plus(Duration.ofSeconds(5)))); - assertThat(actualDelay, greaterThanOrEqualTo(Duration.ofHours(1).minus(Duration.ofSeconds(5)))); - } - - @ParameterizedTest - @ValueSource(strings = {"", "{\"foo\": \"bar\""}) - void processSqsMessages_should_not_interact_with_S3Service_if_input_is_not_valid_JSON(String inputString) { - final Message message = mock(Message.class); - when(message.body()).thenReturn(inputString); - - final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); - when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - - final int messagesProcessed = sqsWorker.processSqsMessages(); - assertThat(messagesProcessed, equalTo(1)); - verifyNoInteractions(s3Service); - verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); - verify(sqsMessagesReceivedCounter).increment(1); - verify(sqsMessagesFailedCounter).increment(); - } - @Test void processSqsMessages_should_not_interact_with_S3Service_and_delete_message_if_TestEvent() { final String messageId = UUID.randomUUID().toString(); @@ -291,7 +214,7 @@ void processSqsMessages_should_not_interact_with_S3Service_and_delete_message_if when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); assertThat(messagesProcessed, equalTo(1)); verifyNoInteractions(s3Service); @@ -324,7 +247,7 @@ void processSqsMessages_should_not_interact_with_S3Service_and_delete_message_if when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); assertThat(messagesProcessed, equalTo(1)); verifyNoInteractions(s3Service); @@ -354,7 +277,7 @@ void processSqsMessages_with_irrelevant_eventName_should_return_number_of_messag when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); assertThat(messagesProcessed, equalTo(1)); verifyNoInteractions(s3Service); @@ -378,7 +301,7 @@ void processSqsMessages_should_invoke_delete_if_input_is_not_valid_JSON_and_dele when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); verify(sqsClient).deleteMessageBatch(deleteMessageBatchRequestArgumentCaptor.capture()); final DeleteMessageBatchRequest actualDeleteMessageBatchRequest = deleteMessageBatchRequestArgumentCaptor.getValue(); @@ -410,7 +333,7 @@ void processSqsMessages_should_return_number_of_messages_processed_when_using_Ev when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); verify(sqsClient).deleteMessageBatch(deleteMessageBatchRequestArgumentCaptor.capture()); final DeleteMessageBatchRequest actualDeleteMessageBatchRequest = deleteMessageBatchRequestArgumentCaptor.getValue(); @@ -447,7 +370,7 @@ void processSqsMessages_should_return_number_of_messages_processed_when_using_Se when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); verify(sqsClient).deleteMessageBatch(deleteMessageBatchRequestArgumentCaptor.capture()); final DeleteMessageBatchRequest actualDeleteMessageBatchRequest = deleteMessageBatchRequestArgumentCaptor.getValue(); @@ -502,7 +425,7 @@ void processSqsMessages_should_report_correct_metrics_for_DeleteMessages_when_so when(deleteMessageBatchResponse.failed()).thenReturn(failedDeletes); when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenReturn(deleteMessageBatchResponse); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); verify(sqsClient).deleteMessageBatch(deleteMessageBatchRequestArgumentCaptor.capture()); @@ -542,7 +465,7 @@ void processSqsMessages_should_report_correct_metrics_for_DeleteMessages_when_re when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenThrow(exClass); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); verify(sqsClient).deleteMessageBatch(deleteMessageBatchRequestArgumentCaptor.capture()); @@ -565,7 +488,7 @@ void processSqsMessages_should_report_correct_metrics_for_DeleteMessages_when_re @Test void processSqsMessages_should_return_zero_messages_when_a_SqsException_is_thrown() { when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenThrow(SqsException.class); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); assertThat(messagesProcessed, equalTo(0)); verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); } @@ -573,7 +496,7 @@ void processSqsMessages_should_return_zero_messages_when_a_SqsException_is_throw @Test void processSqsMessages_should_return_zero_messages_with_backoff_when_a_SqsException_is_thrown() { when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenThrow(SqsException.class); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); verify(backoff).nextDelayMillis(1); assertThat(messagesProcessed, equalTo(0)); } @@ -582,7 +505,8 @@ void processSqsMessages_should_return_zero_messages_with_backoff_when_a_SqsExcep void processSqsMessages_should_throw_when_a_SqsException_is_thrown_with_max_retries() { when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenThrow(SqsException.class); when(backoff.nextDelayMillis(anyInt())).thenReturn((long) -1); - assertThrows(SqsRetriesExhaustedException.class, () -> sqsWorker.processSqsMessages()); + SqsWorker objectUnderTest = createObjectUnderTest(); + assertThrows(SqsRetriesExhaustedException.class, () -> objectUnderTest.processSqsMessages()); } @ParameterizedTest @@ -591,11 +515,13 @@ void processSqsMessages_should_return_zero_messages_when_messages_are_not_S3Even final Message message = mock(Message.class); when(message.body()).thenReturn(inputString); + when(s3SourceConfig.getOnErrorOption()).thenReturn(OnErrorOption.RETAIN_MESSAGES); + final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - final int messagesProcessed = sqsWorker.processSqsMessages(); + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); assertThat(messagesProcessed, equalTo(1)); verifyNoInteractions(s3Service); verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); @@ -605,6 +531,7 @@ void processSqsMessages_should_return_zero_messages_when_messages_are_not_S3Even @Test void populateS3Reference_should_interact_with_getUrlDecodedKey() throws NoSuchMethodException, InvocationTargetException, IllegalAccessException { + reset(sqsOptions); // Using reflection to unit test a private method as part of bug fix. Class params[] = new Class[2]; params[0] = String.class; @@ -617,21 +544,176 @@ void populateS3Reference_should_interact_with_getUrlDecodedKey() throws NoSuchMe final S3EventNotification.S3ObjectEntity s3ObjectEntity = mock(S3EventNotification.S3ObjectEntity.class); final S3EventNotification.S3BucketEntity s3BucketEntity = mock(S3EventNotification.S3BucketEntity.class); - when(s3EventNotificationRecord.getS3()).thenReturn(s3Entity); - when(s3Entity.getBucket()).thenReturn(s3BucketEntity); - when(s3Entity.getObject()).thenReturn(s3ObjectEntity); - when(s3BucketEntity.getName()).thenReturn("test-bucket-name"); - when(s3ObjectEntity.getUrlDecodedKey()).thenReturn("test-key"); - - final S3ObjectReference s3ObjectReference = (S3ObjectReference) method.invoke(sqsWorker, "test-bucket-name", "test-key"); + final S3ObjectReference s3ObjectReference = (S3ObjectReference) method.invoke(createObjectUnderTest(), "test-bucket-name", "test-key"); assertThat(s3ObjectReference, notNullValue()); assertThat(s3ObjectReference.getBucketName(), equalTo("test-bucket-name")); assertThat(s3ObjectReference.getKey(), equalTo("test-key")); -// verify(s3ObjectEntity).getUrlDecodedKey(); verifyNoMoreInteractions(s3ObjectEntity); } + + @ParameterizedTest + @ValueSource(strings = {"ObjectCreated:Put", "ObjectCreated:Post", "ObjectCreated:Copy", "ObjectCreated:CompleteMultipartUpload"}) + void processSqsMessages_should_return_number_of_messages_processed_with_acknowledgements(final String eventName) throws IOException { + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + when(s3SourceConfig.getAcknowledgements()).thenReturn(true); + Instant startTime = Instant.now().minus(1, ChronoUnit.HOURS); + final Message message = mock(Message.class); + when(message.body()).thenReturn(createEventNotification(eventName, startTime)); + final String testReceiptHandle = UUID.randomUUID().toString(); + when(message.messageId()).thenReturn(testReceiptHandle); + when(message.receiptHandle()).thenReturn(testReceiptHandle); + + final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); + when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); + + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); + final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); + + final ArgumentCaptor durationArgumentCaptor = ArgumentCaptor.forClass(Duration.class); + verify(sqsMessageDelayTimer).record(durationArgumentCaptor.capture()); + Duration actualDelay = durationArgumentCaptor.getValue(); + + assertThat(messagesProcessed, equalTo(1)); + verify(s3Service).addS3Object(any(S3ObjectReference.class), any()); + verify(acknowledgementSetManager).create(any(), any(Duration.class)); + verify(sqsMessagesReceivedCounter).increment(1); + verifyNoInteractions(sqsMessagesDeletedCounter); + assertThat(actualDelay, lessThanOrEqualTo(Duration.ofHours(1).plus(Duration.ofSeconds(5)))); + assertThat(actualDelay, greaterThanOrEqualTo(Duration.ofHours(1).minus(Duration.ofSeconds(5)))); + } + + @ParameterizedTest + @ValueSource(strings = {"ObjectCreated:Put", "ObjectCreated:Post", "ObjectCreated:Copy", "ObjectCreated:CompleteMultipartUpload"}) + void processSqsMessages_should_return_number_of_messages_processed_with_acknowledgements_and_progress_check(final String eventName) throws IOException { + when(sqsOptions.getVisibilityDuplicateProtection()).thenReturn(true); + when(sqsOptions.getVisibilityTimeout()).thenReturn(Duration.ofSeconds(6)); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + when(s3SourceConfig.getAcknowledgements()).thenReturn(true); + Instant startTime = Instant.now().minus(1, ChronoUnit.HOURS); + final Message message = mock(Message.class); + when(message.body()).thenReturn(createEventNotification(eventName, startTime)); + final String testReceiptHandle = UUID.randomUUID().toString(); + when(message.messageId()).thenReturn(testReceiptHandle); + when(message.receiptHandle()).thenReturn(testReceiptHandle); + + final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); + when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); + + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); + final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); + + final ArgumentCaptor durationArgumentCaptor = ArgumentCaptor.forClass(Duration.class); + verify(sqsMessageDelayTimer).record(durationArgumentCaptor.capture()); + Duration actualDelay = durationArgumentCaptor.getValue(); + + assertThat(messagesProcessed, equalTo(1)); + verify(s3Service).addS3Object(any(S3ObjectReference.class), any()); + verify(acknowledgementSetManager).create(any(), any(Duration.class)); + verify(acknowledgementSet).addProgressCheck(any(), any(Duration.class)); + verify(sqsMessagesReceivedCounter).increment(1); + verifyNoInteractions(sqsMessagesDeletedCounter); + assertThat(actualDelay, lessThanOrEqualTo(Duration.ofHours(1).plus(Duration.ofSeconds(5)))); + assertThat(actualDelay, greaterThanOrEqualTo(Duration.ofHours(1).minus(Duration.ofSeconds(5)))); + } + + @ParameterizedTest + @ValueSource(strings = {"", "{\"foo\": \"bar\""}) + void processSqsMessages_should_not_interact_with_S3Service_if_input_is_not_valid_JSON(String inputString) { + final Message message = mock(Message.class); + when(message.body()).thenReturn(inputString); + + when(s3SourceConfig.getOnErrorOption()).thenReturn(OnErrorOption.RETAIN_MESSAGES); + + final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); + when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); + + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); + assertThat(messagesProcessed, equalTo(1)); + verifyNoInteractions(s3Service); + verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); + verify(sqsMessagesReceivedCounter).increment(1); + verify(sqsMessagesFailedCounter).increment(); + } + + @Test + void processSqsMessages_should_update_visibility_timeout_when_progress_changes() throws IOException { + when(sqsOptions.getVisibilityDuplicateProtection()).thenReturn(true); + when(sqsOptions.getVisibilityTimeout()).thenReturn(Duration.ofMillis(1)); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + when(s3SourceConfig.getAcknowledgements()).thenReturn(true); + Instant startTime = Instant.now().minus(1, ChronoUnit.HOURS); + final Message message = mock(Message.class); + when(message.body()).thenReturn(createEventNotification("ObjectCreated:Put", startTime)); + final String testReceiptHandle = UUID.randomUUID().toString(); + when(message.messageId()).thenReturn(testReceiptHandle); + when(message.receiptHandle()).thenReturn(testReceiptHandle); + + final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); + when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); + + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); + + assertThat(messagesProcessed, equalTo(1)); + verify(s3Service).addS3Object(any(S3ObjectReference.class), any()); + verify(acknowledgementSetManager).create(any(), any(Duration.class)); + + ArgumentCaptor> progressConsumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSet).addProgressCheck(progressConsumerArgumentCaptor.capture(), any(Duration.class)); + final Consumer actualConsumer = progressConsumerArgumentCaptor.getValue(); + final ProgressCheck progressCheck = mock(ProgressCheck.class); + actualConsumer.accept(progressCheck); + + ArgumentCaptor changeMessageVisibilityRequestArgumentCaptor = ArgumentCaptor.forClass(ChangeMessageVisibilityRequest.class); + verify(sqsClient).changeMessageVisibility(changeMessageVisibilityRequestArgumentCaptor.capture()); + ChangeMessageVisibilityRequest actualChangeVisibilityRequest = changeMessageVisibilityRequestArgumentCaptor.getValue(); + assertThat(actualChangeVisibilityRequest.queueUrl(), equalTo(queueUrl)); + assertThat(actualChangeVisibilityRequest.receiptHandle(), equalTo(testReceiptHandle)); + verify(sqsMessagesReceivedCounter).increment(1); + verify(sqsMessageDelayTimer).record(any(Duration.class)); + } + + @Test + void processSqsMessages_should_stop_updating_visibility_timeout_after_stop() throws IOException { + when(sqsOptions.getVisibilityDuplicateProtection()).thenReturn(true); + when(sqsOptions.getVisibilityTimeout()).thenReturn(Duration.ofMillis(1)); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + when(s3SourceConfig.getAcknowledgements()).thenReturn(true); + Instant startTime = Instant.now().minus(1, ChronoUnit.HOURS); + final Message message = mock(Message.class); + when(message.body()).thenReturn(createEventNotification("ObjectCreated:Put", startTime)); + final String testReceiptHandle = UUID.randomUUID().toString(); + when(message.messageId()).thenReturn(testReceiptHandle); + when(message.receiptHandle()).thenReturn(testReceiptHandle); + + final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); + when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); + + SqsWorker objectUnderTest = createObjectUnderTest(); + final int messagesProcessed = objectUnderTest.processSqsMessages(); + objectUnderTest.stop(); + + assertThat(messagesProcessed, equalTo(1)); + verify(s3Service).addS3Object(any(S3ObjectReference.class), any()); + verify(acknowledgementSetManager).create(any(), any(Duration.class)); + + ArgumentCaptor> progressConsumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSet).addProgressCheck(progressConsumerArgumentCaptor.capture(), any(Duration.class)); + final Consumer actualConsumer = progressConsumerArgumentCaptor.getValue(); + final ProgressCheck progressCheck = mock(ProgressCheck.class); + actualConsumer.accept(progressCheck); + + verify(sqsClient, never()).changeMessageVisibility(any(ChangeMessageVisibilityRequest.class)); + verify(sqsMessagesReceivedCounter).increment(1); + verify(sqsMessageDelayTimer).record(any(Duration.class)); + } + private static String createPutNotification(final Instant startTime) { return createEventNotification("ObjectCreated:Put", startTime); } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessageTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessageTest.java index 3acec973e1..51f3abad06 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessageTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessageTest.java @@ -2,6 +2,7 @@ import org.joda.time.DateTime; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.plugins.source.s3.S3EventBridgeNotification; import org.opensearch.dataprepper.plugins.source.s3.S3EventNotification; @@ -12,33 +13,31 @@ import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; class ParsedMessageTest { private static final Random RANDOM = new Random(); private Message message; - private S3EventNotification.S3Entity s3Entity; - private S3EventNotification.S3BucketEntity s3BucketEntity; - private S3EventNotification.S3ObjectEntity s3ObjectEntity; - private S3EventNotification.S3EventNotificationRecord s3EventNotificationRecord; - private S3EventBridgeNotification s3EventBridgeNotification; - private S3EventBridgeNotification.Detail detail; - private S3EventBridgeNotification.Bucket bucket; - private S3EventBridgeNotification.Object object; + private String testBucketName; + private String testDecodedObjectKey; + private long testSize; @BeforeEach void setUp() { message = mock(Message.class); - s3Entity = mock(S3EventNotification.S3Entity.class); - s3BucketEntity = mock(S3EventNotification.S3BucketEntity.class); - s3ObjectEntity = mock(S3EventNotification.S3ObjectEntity.class); - s3EventNotificationRecord = mock(S3EventNotification.S3EventNotificationRecord.class); - s3EventBridgeNotification = mock(S3EventBridgeNotification.class); - detail = mock(S3EventBridgeNotification.Detail.class); - bucket = mock(S3EventBridgeNotification.Bucket.class); - object = mock(S3EventBridgeNotification.Object.class); + testBucketName = UUID.randomUUID().toString(); + testDecodedObjectKey = UUID.randomUUID().toString(); + testSize = RANDOM.nextInt(1_000_000_000) + 1; + } + + @Test + void constructor_with_failed_parsing_throws_if_Message_is_null() { + assertThrows(NullPointerException.class, () -> new ParsedMessage(null, true)); } @Test @@ -50,61 +49,156 @@ void test_parsed_message_with_failed_parsing() { } @Test - void test_parsed_message_with_S3EventNotificationRecord() { - final String testBucketName = UUID.randomUUID().toString(); - final String testDecodedObjectKey = UUID.randomUUID().toString(); - final String testEventName = UUID.randomUUID().toString(); - final DateTime testEventTime = DateTime.now(); - final long testSize = RANDOM.nextLong(); - - when(s3EventNotificationRecord.getS3()).thenReturn(s3Entity); - when(s3Entity.getBucket()).thenReturn(s3BucketEntity); - when(s3Entity.getObject()).thenReturn(s3ObjectEntity); - when(s3ObjectEntity.getSizeAsLong()).thenReturn(testSize); - when(s3BucketEntity.getName()).thenReturn(testBucketName); - when(s3ObjectEntity.getUrlDecodedKey()).thenReturn(testDecodedObjectKey); - when(s3EventNotificationRecord.getEventName()).thenReturn(testEventName); - when(s3EventNotificationRecord.getEventTime()).thenReturn(testEventTime); - - final ParsedMessage parsedMessage = new ParsedMessage(message, List.of(s3EventNotificationRecord)); + void toString_with_failed_parsing_and_messageId() { + final String messageId = UUID.randomUUID().toString(); + when(message.messageId()).thenReturn(messageId); - assertThat(parsedMessage.getMessage(), equalTo(message)); - assertThat(parsedMessage.getBucketName(), equalTo(testBucketName)); - assertThat(parsedMessage.getObjectKey(), equalTo(testDecodedObjectKey)); - assertThat(parsedMessage.getObjectSize(), equalTo(testSize)); - assertThat(parsedMessage.getEventName(), equalTo(testEventName)); - assertThat(parsedMessage.getEventTime(), equalTo(testEventTime)); - assertThat(parsedMessage.isFailedParsing(), equalTo(false)); - assertThat(parsedMessage.isEmptyNotification(), equalTo(false)); + final ParsedMessage parsedMessage = new ParsedMessage(message, true); + final String actualString = parsedMessage.toString(); + assertThat(actualString, notNullValue()); + assertThat(actualString, containsString(messageId)); } @Test - void test_parsed_message_with_S3EventBridgeNotification() { - final String testBucketName = UUID.randomUUID().toString(); - final String testDecodedObjectKey = UUID.randomUUID().toString(); - final String testDetailType = UUID.randomUUID().toString(); - final DateTime testEventTime = DateTime.now(); - final int testSize = RANDOM.nextInt(); + void toString_with_failed_parsing_and_no_messageId() { + final ParsedMessage parsedMessage = new ParsedMessage(message, true); + final String actualString = parsedMessage.toString(); + assertThat(actualString, notNullValue()); + } - when(s3EventBridgeNotification.getDetail()).thenReturn(detail); - when(s3EventBridgeNotification.getDetail().getBucket()).thenReturn(bucket); - when(s3EventBridgeNotification.getDetail().getObject()).thenReturn(object); + @Nested + class WithS3EventNotificationRecord { + private S3EventNotification.S3Entity s3Entity; + private S3EventNotification.S3BucketEntity s3BucketEntity; + private S3EventNotification.S3ObjectEntity s3ObjectEntity; + private S3EventNotification.S3EventNotificationRecord s3EventNotificationRecord; + private List s3EventNotificationRecords; + private String testEventName; + private DateTime testEventTime; - when(bucket.getName()).thenReturn(testBucketName); - when(object.getUrlDecodedKey()).thenReturn(testDecodedObjectKey); - when(object.getSize()).thenReturn(testSize); - when(s3EventBridgeNotification.getDetailType()).thenReturn(testDetailType); - when(s3EventBridgeNotification.getTime()).thenReturn(testEventTime); + @BeforeEach + void setUp() { + testEventName = UUID.randomUUID().toString(); + testEventTime = DateTime.now(); - final ParsedMessage parsedMessage = new ParsedMessage(message, s3EventBridgeNotification); + s3Entity = mock(S3EventNotification.S3Entity.class); + s3BucketEntity = mock(S3EventNotification.S3BucketEntity.class); + s3ObjectEntity = mock(S3EventNotification.S3ObjectEntity.class); + s3EventNotificationRecord = mock(S3EventNotification.S3EventNotificationRecord.class); - assertThat(parsedMessage.getMessage(), equalTo(message)); - assertThat(parsedMessage.getBucketName(), equalTo(testBucketName)); - assertThat(parsedMessage.getObjectKey(), equalTo(testDecodedObjectKey)); - assertThat(parsedMessage.getObjectSize(), equalTo((long) testSize)); - assertThat(parsedMessage.getDetailType(), equalTo(testDetailType)); - assertThat(parsedMessage.getEventTime(), equalTo(testEventTime)); - assertThat(parsedMessage.isFailedParsing(), equalTo(false)); - assertThat(parsedMessage.isEmptyNotification(), equalTo(false)); + when(s3EventNotificationRecord.getS3()).thenReturn(s3Entity); + when(s3Entity.getBucket()).thenReturn(s3BucketEntity); + when(s3Entity.getObject()).thenReturn(s3ObjectEntity); + when(s3ObjectEntity.getSizeAsLong()).thenReturn(testSize); + when(s3BucketEntity.getName()).thenReturn(testBucketName); + when(s3ObjectEntity.getUrlDecodedKey()).thenReturn(testDecodedObjectKey); + when(s3EventNotificationRecord.getEventName()).thenReturn(testEventName); + when(s3EventNotificationRecord.getEventTime()).thenReturn(testEventTime); + + s3EventNotificationRecords = List.of(s3EventNotificationRecord); + } + + private ParsedMessage createObjectUnderTest() { + return new ParsedMessage(message, s3EventNotificationRecords); + } + + @Test + void constructor_with_S3EventNotificationRecord_throws_if_Message_is_null() { + message = null; + assertThrows(NullPointerException.class, this::createObjectUnderTest); + } + + @Test + void test_parsed_message_with_S3EventNotificationRecord() { + final ParsedMessage parsedMessage = createObjectUnderTest(); + + assertThat(parsedMessage.getMessage(), equalTo(message)); + assertThat(parsedMessage.getBucketName(), equalTo(testBucketName)); + assertThat(parsedMessage.getObjectKey(), equalTo(testDecodedObjectKey)); + assertThat(parsedMessage.getObjectSize(), equalTo(testSize)); + assertThat(parsedMessage.getEventName(), equalTo(testEventName)); + assertThat(parsedMessage.getEventTime(), equalTo(testEventTime)); + assertThat(parsedMessage.isFailedParsing(), equalTo(false)); + assertThat(parsedMessage.isEmptyNotification(), equalTo(false)); + } + + @Test + void toString_with_messageId() { + final String messageId = UUID.randomUUID().toString(); + when(message.messageId()).thenReturn(messageId); + + final ParsedMessage parsedMessage = createObjectUnderTest(); + final String actualString = parsedMessage.toString(); + assertThat(actualString, notNullValue()); + assertThat(actualString, containsString(messageId)); + assertThat(actualString, containsString(testDecodedObjectKey)); + } + } + + @Nested + class WithS3EventBridgeNotification { + private String testDetailType; + private DateTime testEventTime; + private S3EventBridgeNotification s3EventBridgeNotification; + private S3EventBridgeNotification.Detail detail; + private S3EventBridgeNotification.Bucket bucket; + private S3EventBridgeNotification.Object object; + + @BeforeEach + void setUp() { + s3EventBridgeNotification = mock(S3EventBridgeNotification.class); + detail = mock(S3EventBridgeNotification.Detail.class); + bucket = mock(S3EventBridgeNotification.Bucket.class); + object = mock(S3EventBridgeNotification.Object.class); + + testDetailType = UUID.randomUUID().toString(); + testEventTime = DateTime.now(); + + when(s3EventBridgeNotification.getDetail()).thenReturn(detail); + when(s3EventBridgeNotification.getDetail().getBucket()).thenReturn(bucket); + when(s3EventBridgeNotification.getDetail().getObject()).thenReturn(object); + + when(bucket.getName()).thenReturn(testBucketName); + when(object.getUrlDecodedKey()).thenReturn(testDecodedObjectKey); + when(object.getSize()).thenReturn((int) testSize); + when(s3EventBridgeNotification.getDetailType()).thenReturn(testDetailType); + when(s3EventBridgeNotification.getTime()).thenReturn(testEventTime); + } + + private ParsedMessage createObjectUnderTest() { + return new ParsedMessage(message, s3EventBridgeNotification); + } + + @Test + void constructor_with_S3EventBridgeNotification_throws_if_Message_is_null() { + message = null; + assertThrows(NullPointerException.class, () -> createObjectUnderTest()); + } + + @Test + void test_parsed_message_with_S3EventBridgeNotification() { + final ParsedMessage parsedMessage = createObjectUnderTest(); + + assertThat(parsedMessage.getMessage(), equalTo(message)); + assertThat(parsedMessage.getBucketName(), equalTo(testBucketName)); + assertThat(parsedMessage.getObjectKey(), equalTo(testDecodedObjectKey)); + assertThat(parsedMessage.getObjectSize(), equalTo(testSize)); + assertThat(parsedMessage.getDetailType(), equalTo(testDetailType)); + assertThat(parsedMessage.getEventTime(), equalTo(testEventTime)); + assertThat(parsedMessage.isFailedParsing(), equalTo(false)); + assertThat(parsedMessage.isEmptyNotification(), equalTo(false)); + } + + @Test + void toString_with_messageId() { + final String messageId = UUID.randomUUID().toString(); + when(message.messageId()).thenReturn(messageId); + + final ParsedMessage parsedMessage = createObjectUnderTest(); + final String actualString = parsedMessage.toString(); + assertThat(actualString, notNullValue()); + assertThat(actualString, containsString(messageId)); + assertThat(actualString, containsString(testDecodedObjectKey)); + } } } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/S3EventBridgeNotificationParserTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/S3EventBridgeNotificationParserTest.java index c779ec561f..db361d70e1 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/S3EventBridgeNotificationParserTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/S3EventBridgeNotificationParserTest.java @@ -19,7 +19,7 @@ class S3EventBridgeNotificationParserTest { private final ObjectMapper objectMapper = new ObjectMapper(); - private final String EVENTBRIDGE_MESSAGE = "{\"version\":\"0\",\"id\":\"17793124-05d4-b198-2fde-7ededc63b103\",\"detail-type\":\"Object Created\"," + + static final String EVENTBRIDGE_MESSAGE = "{\"version\":\"0\",\"id\":\"17793124-05d4-b198-2fde-7ededc63b103\",\"detail-type\":\"Object Created\"," + "\"source\":\"aws.s3\",\"account\":\"111122223333\",\"time\":\"2021-11-12T00:00:00Z\"," + "\"region\":\"ca-central-1\",\"resources\":[\"arn:aws:s3:::DOC-EXAMPLE-BUCKET1\"]," + "\"detail\":{\"version\":\"0\",\"bucket\":{\"name\":\"DOC-EXAMPLE-BUCKET1\"}," + diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/S3EventNotificationParserTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/S3EventNotificationParserTest.java index a3d2c91679..c9e3a39da8 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/S3EventNotificationParserTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/S3EventNotificationParserTest.java @@ -16,8 +16,8 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -class S3EventNotificationParserTest { - private static final String DIRECT_SQS_MESSAGE = +public class S3EventNotificationParserTest { + static final String DIRECT_SQS_MESSAGE = "{\"Records\":[{\"eventVersion\":\"2.1\",\"eventSource\":\"aws:s3\",\"awsRegion\":\"us-east-1\",\"eventTime\":\"2023-04-28T16:00:11.324Z\"," + "\"eventName\":\"ObjectCreated:Put\",\"userIdentity\":{\"principalId\":\"AWS:xyz\"},\"requestParameters\":{\"sourceIPAddress\":\"127.0.0.1\"}," + "\"responseElements\":{\"x-amz-request-id\":\"xyz\",\"x-amz-id-2\":\"xyz\"},\"s3\":{\"s3SchemaVersion\":\"1.0\"," + @@ -25,7 +25,7 @@ class S3EventNotificationParserTest { "\"arn\":\"arn:aws:s3:::my-bucket\"},\"object\":{\"key\":\"path/to/myfile.log.gz\",\"size\":3159112,\"eTag\":\"abcd123\"," + "\"sequencer\":\"000\"}}}]}"; - private static final String SNS_BASED_MESSAGE = "{\n" + + public static final String SNS_BASED_MESSAGE = "{\n" + " \"Type\" : \"Notification\",\n" + " \"MessageId\" : \"4e01e115-5b91-5096-8a74-bee95ed1e123\",\n" + " \"TopicArn\" : \"arn:aws:sns:us-east-1:123456789012:notifications\",\n" + diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/SqsMessageParserTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/SqsMessageParserTest.java new file mode 100644 index 0000000000..d0dd711f7e --- /dev/null +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/SqsMessageParserTest.java @@ -0,0 +1,96 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.s3.parser; + +import org.junit.jupiter.api.extension.ExtendWith; +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.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.s3.S3SourceConfig; +import org.opensearch.dataprepper.plugins.source.s3.configuration.NotificationSourceOption; +import software.amazon.awssdk.services.sqs.model.Message; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Set; +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.hamcrest.Matchers.empty; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class SqsMessageParserTest { + @Mock + private S3SourceConfig s3SourceConfig; + + private SqsMessageParser createObjectUnderTest() { + return new SqsMessageParser(s3SourceConfig); + } + + @ParameterizedTest + @ArgumentsSource(SourceArgumentsProvider.class) + void parseSqsMessages_returns_empty_for_empty_messages(final NotificationSourceOption sourceOption) { + when(s3SourceConfig.getNotificationSource()).thenReturn(sourceOption); + final Collection parsedMessages = createObjectUnderTest().parseSqsMessages(Collections.emptyList()); + + assertThat(parsedMessages, notNullValue()); + assertThat(parsedMessages, empty()); + } + + @ParameterizedTest + @ArgumentsSource(SourceArgumentsProvider.class) + void parseSqsMessages_parsed_messages(final NotificationSourceOption sourceOption, + final String messageBody, + final String replacementString) { + when(s3SourceConfig.getNotificationSource()).thenReturn(sourceOption); + final int numberOfMessages = 10; + List messages = IntStream.range(0, numberOfMessages) + .mapToObj(i -> messageBody.replaceAll(replacementString, replacementString + i)) + .map(SqsMessageParserTest::createMockMessage) + .collect(Collectors.toList()); + final Collection parsedMessages = createObjectUnderTest().parseSqsMessages(messages); + + assertThat(parsedMessages, notNullValue()); + assertThat(parsedMessages.size(), equalTo(numberOfMessages)); + + final Set bucketNames = parsedMessages.stream().map(ParsedMessage::getBucketName).collect(Collectors.toSet()); + assertThat("The bucket names are unique, so the bucketNames should match the numberOfMessages.", + bucketNames.size(), equalTo(numberOfMessages)); + } + + static class SourceArgumentsProvider implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext extensionContext) { + return Stream.of( + Arguments.arguments( + NotificationSourceOption.S3, + S3EventNotificationParserTest.DIRECT_SQS_MESSAGE, + "my-bucket"), + Arguments.arguments( + NotificationSourceOption.EVENTBRIDGE, + S3EventBridgeNotificationParserTest.EVENTBRIDGE_MESSAGE, + "DOC-EXAMPLE-BUCKET1") + ); + } + } + + private static Message createMockMessage(final String body) { + final Message message = mock(Message.class); + when(message.body()).thenReturn(body); + return message; + } +} \ No newline at end of file From bbfaef6bc1c59fdbe7519d12d4bd51e9f678085f Mon Sep 17 00:00:00 2001 From: Katherine Shen <40495707+shenkw1@users.noreply.github.com> Date: Mon, 22 Jul 2024 15:06:13 -0500 Subject: [PATCH 081/159] MAINT: Add json property descriptions for csv processor (#4751) * add json property descriptions Signed-off-by: Katherine Shen --- .../processor/csv/CsvProcessorConfig.java | 22 +++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java index ec5d685b7e..8c770b597a 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.csv; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import java.util.List; @@ -20,24 +21,45 @@ public class CsvProcessorConfig { static final Boolean DEFAULT_DELETE_HEADERS = true; @JsonProperty("source") + @JsonPropertyDescription("The field in the event that will be parsed. Default value is `message`.") private String source = DEFAULT_SOURCE; @JsonProperty("delimiter") + @JsonPropertyDescription("The character separating each column. Default value is `,`.") private String delimiter = DEFAULT_DELIMITER; @JsonProperty("delete_header") + @JsonPropertyDescription("If specified, the event header (`column_names_source_key`) is deleted after the event " + + "is parsed. If there is no event header, no action is taken. Default value is true.") private Boolean deleteHeader = DEFAULT_DELETE_HEADERS; @JsonProperty("quote_character") + @JsonPropertyDescription("The character used as a text qualifier for a single column of data. " + + "Default value is `\"`.") private String quoteCharacter = DEFAULT_QUOTE_CHARACTER; @JsonProperty("column_names_source_key") + @JsonPropertyDescription("The field in the event that specifies the CSV column names, which will be " + + "automatically detected. If there need to be extra column names, the column names are automatically " + + "generated according to their index. If `column_names` is also defined, the header in " + + "`column_names_source_key` can also be used to generate the event fields. " + + "If too few columns are specified in this field, the remaining column names are automatically generated. " + + "If too many column names are specified in this field, the CSV processor omits the extra column names.") private String columnNamesSourceKey; @JsonProperty("column_names") + @JsonPropertyDescription("User-specified names for the CSV columns. " + + "Default value is `[column1, column2, ..., columnN]` if there are no columns of data in the CSV " + + "record and `column_names_source_key` is not defined. If `column_names_source_key` is defined, " + + "the header in `column_names_source_key` generates the event fields. If too few columns are specified " + + "in this field, the remaining column names are automatically generated. " + + "If too many column names are specified in this field, the CSV processor omits the extra column names.") private List columnNames; @JsonProperty("csv_when") + @JsonPropertyDescription("Allows you to specify a [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"`, that will be evaluated to determine whether " + + "the processor should be applied to the event.") private String csvWhen; /** From 6a9757f0933dfae36f83e4fb3941072ffd27ac13 Mon Sep 17 00:00:00 2001 From: Santhosh Gandhe <1909520+san81@users.noreply.github.com> Date: Mon, 22 Jul 2024 14:40:18 -0700 Subject: [PATCH 082/159] doc strings for enhanced UI view auto-generation (#4755) Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> --- .../processor/splitevent/SplitEventProcessorConfig.java | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/data-prepper-plugins/split-event-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorConfig.java b/data-prepper-plugins/split-event-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorConfig.java index c4af96a3d4..db70e3c6db 100644 --- a/data-prepper-plugins/split-event-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorConfig.java +++ b/data-prepper-plugins/split-event-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorConfig.java @@ -11,6 +11,7 @@ package org.opensearch.dataprepper.plugins.processor.splitevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.Size; @@ -20,12 +21,15 @@ public class SplitEventProcessorConfig { @NotEmpty @NotNull @JsonProperty("field") + @JsonPropertyDescription("The event field to be split") private String field; @JsonProperty("delimiter_regex") + @JsonPropertyDescription("The regular expression used as the delimiter for splitting the field") private String delimiterRegex; @Size(min = 1, max = 1) + @JsonPropertyDescription("The delimiter used for splitting the field. If not specified, the default delimiter is used") private String delimiter; public String getField() { From 84c4d0b6c04223876218fba30da28648b89285e2 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Tue, 23 Jul 2024 14:34:05 -0500 Subject: [PATCH 083/159] Add json description to AddEntry processor (#4752) Signed-off-by: Hai Yan --- .../mutateevent/AddEntryProcessorConfig.java | 24 +++++++++++++++++++ 1 file changed, 24 insertions(+) diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java index 81f6bbab34..df23740344 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; @@ -16,25 +17,47 @@ public class AddEntryProcessorConfig { public static class Entry { + + @JsonPropertyDescription("The key of the new entry to be added. Some examples of keys include `my_key`, " + + "`myKey`, and `object/sub_Key`. The key can also be a format expression, for example, `${/key1}` to " + + "use the value of field `key1` as the key.") private String key; @JsonProperty("metadata_key") + @JsonPropertyDescription("The key for the new metadata attribute. The argument must be a literal string key " + + "and not a JSON Pointer. Either one string key or `metadata_key` is required.") private String metadataKey; + @JsonPropertyDescription("The value of the new entry to be added, which can be used with any of the " + + "following data types: strings, Booleans, numbers, null, nested objects, and arrays.") private Object value; + @JsonPropertyDescription("A format string to use as the value of the new entry, for example, " + + "`${key1}-${key2}`, where `key1` and `key2` are existing keys in the event. Required if neither " + + "`value` nor `value_expression` is specified.") private String format; @JsonProperty("value_expression") + @JsonPropertyDescription("An expression string to use as the value of the new entry. For example, `/key` " + + "is an existing key in the event with a type of either a number, a string, or a Boolean. " + + "Expressions can also contain functions returning number/string/integer. For example, " + + "`length(/key)` will return the length of the key in the event when the key is a string. For more " + + "information about keys, see [Expression syntax](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/).") private String valueExpression; @JsonProperty("add_when") + @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will be run on the event.") private String addWhen; @JsonProperty("overwrite_if_key_exists") + @JsonPropertyDescription("When set to `true`, the existing value is overwritten if `key` already exists " + + "in the event. The default value is `false`.") private boolean overwriteIfKeyExists = false; @JsonProperty("append_if_key_exists") + @JsonPropertyDescription("When set to `true`, the existing value will be appended if a `key` already " + + "exists in the event. An array will be created if the existing value is not an array. Default is `false`.") private boolean appendIfKeyExists = false; public String getKey() { @@ -110,6 +133,7 @@ public Entry() { @NotEmpty @NotNull @Valid + @JsonPropertyDescription("A list of entries to add to the event.") private List entries; public List getEntries() { From 427bb7806a7e9ac1c7a1bce0d113371e7824ec92 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Tue, 23 Jul 2024 16:30:35 -0500 Subject: [PATCH 084/159] Add stream processing for rds source (#4757) * Add stream processing Signed-off-by: Hai Yan * Address review comments Signed-off-by: Hai Yan * Change s3 partition count default value Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- data-prepper-plugins/rds-source/build.gradle | 2 + .../plugins/source/rds/RdsService.java | 15 ++ .../plugins/source/rds/RdsSource.java | 1 + .../plugins/source/rds/RdsSourceConfig.java | 36 +++ .../rds/configuration/StreamConfig.java | 24 ++ .../rds/converter/MetadataKeyAttributes.java | 4 + .../rds/converter/S3PartitionCreator.java | 31 +++ .../rds/converter/StreamRecordConverter.java | 94 ++++++++ .../rds/coordination/PartitionFactory.java | 22 +- .../coordination/partition/GlobalState.java | 6 + .../partition/StreamPartition.java | 49 ++++ .../state/StreamProgressState.java | 45 ++++ .../source/rds/export/DataFileScheduler.java | 7 +- .../source/rds/leader/LeaderScheduler.java | 13 ++ .../source/rds/model/BinlogCoordinate.java | 33 +++ .../source/rds/model/TableMetadata.java | 59 +++++ .../rds/stream/BinlogClientFactory.java | 46 ++++ .../rds/stream/BinlogEventListener.java | 209 ++++++++++++++++++ .../source/rds/stream/StreamScheduler.java | 85 +++++++ .../source/rds/stream/StreamWorker.java | 95 ++++++++ .../rds/converter/S3PartitionCreatorTest.java | 34 +++ .../converter/StreamRecordConverterTest.java | 60 +++++ .../rds/stream/BinlogClientFactoryTest.java | 66 ++++++ .../rds/stream/BinlogEventListenerTest.java | 92 ++++++++ .../rds/stream/StreamSchedulerTest.java | 116 ++++++++++ .../source/rds/stream/StreamWorkerTest.java | 85 +++++++ .../plugins/source/s3/ScanObjectWorker.java | 4 +- 27 files changed, 1321 insertions(+), 12 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/StreamConfig.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/S3PartitionCreator.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/StreamPartition.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/BinlogCoordinate.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/TableMetadata.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/S3PartitionCreatorTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java diff --git a/data-prepper-plugins/rds-source/build.gradle b/data-prepper-plugins/rds-source/build.gradle index f83b1332eb..6d6a681646 100644 --- a/data-prepper-plugins/rds-source/build.gradle +++ b/data-prepper-plugins/rds-source/build.gradle @@ -21,6 +21,8 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-core' implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'com.zendesk:mysql-binlog-connector-java:0.29.2' + testImplementation project(path: ':data-prepper-test-common') testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation project(path: ':data-prepper-test-event') diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index 77956e6b0e..83e168f41f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds; +import com.github.shyiko.mysql.binlog.BinaryLogClient; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; @@ -14,6 +15,8 @@ import org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; +import org.opensearch.dataprepper.plugins.source.rds.stream.BinlogClientFactory; +import org.opensearch.dataprepper.plugins.source.rds.stream.StreamScheduler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.rds.RdsClient; @@ -42,6 +45,7 @@ public class RdsService { private LeaderScheduler leaderScheduler; private ExportScheduler exportScheduler; private DataFileScheduler dataFileScheduler; + private StreamScheduler streamScheduler; public RdsService(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig, @@ -78,6 +82,12 @@ public void start(Buffer> buffer) { runnableList.add(dataFileScheduler); } + if (sourceConfig.isStreamEnabled()) { + BinaryLogClient binaryLogClient = new BinlogClientFactory(sourceConfig, rdsClient).create(); + streamScheduler = new StreamScheduler(sourceCoordinator, sourceConfig, binaryLogClient, buffer, pluginMetrics); + runnableList.add(streamScheduler); + } + executor = Executors.newFixedThreadPool(runnableList.size()); runnableList.forEach(executor::submit); } @@ -93,6 +103,11 @@ public void shutdown() { exportScheduler.shutdown(); dataFileScheduler.shutdown(); } + + if (sourceConfig.isStreamEnabled()) { + streamScheduler.shutdown(); + } + leaderScheduler.shutdown(); executor.shutdownNow(); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java index 43806c0475..071fc5889b 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java @@ -52,6 +52,7 @@ public RdsSource(final PluginMetrics pluginMetrics, @Override public void start(Buffer> buffer) { + LOG.info("Starting RDS source"); Objects.requireNonNull(sourceCoordinator); sourceCoordinator.createPartition(new LeaderPartition()); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java index cb8e41513a..4d90c475ec 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.plugins.source.rds.configuration.AwsAuthenticationConfig; import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.configuration.ExportConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.StreamConfig; import java.util.List; @@ -70,6 +71,12 @@ public class RdsSourceConfig { @Valid private ExportConfig exportConfig; + @JsonProperty("stream") + private StreamConfig streamConfig; + + @JsonProperty("authentication") + private AuthenticationConfig authenticationConfig; + public String getDbIdentifier() { return dbIdentifier; } @@ -117,4 +124,33 @@ public ExportConfig getExport() { public boolean isExportEnabled() { return exportConfig != null; } + + public StreamConfig getStream() { + return streamConfig; + } + + public boolean isStreamEnabled() { + return streamConfig != null; + } + + public AuthenticationConfig getAuthenticationConfig() { + return this.authenticationConfig; + } + + public static class AuthenticationConfig { + @JsonProperty("username") + private String username; + + @JsonProperty("password") + private String password; + + public String getUsername() { + return username; + } + + public String getPassword() { + return password; + } + + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/StreamConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/StreamConfig.java new file mode 100644 index 0000000000..c246e56b45 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/StreamConfig.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Max; +import jakarta.validation.constraints.Min; + +public class StreamConfig { + + private static final int DEFAULT_S3_FOLDER_PARTITION_COUNT = 100; + + @JsonProperty("partition_count") + @Min(1) + @Max(1000) + private int s3FolderPartitionCount = DEFAULT_S3_FOLDER_PARTITION_COUNT; + + public int getPartitionCount() { + return s3FolderPartitionCount; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java index 91eecdf07b..60e1ba2bd3 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java @@ -14,7 +14,11 @@ public class MetadataKeyAttributes { static final String EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE = "opensearch_action"; + static final String EVENT_DATABASE_NAME_METADATA_ATTRIBUTE = "database_name"; + static final String EVENT_TABLE_NAME_METADATA_ATTRIBUTE = "table_name"; static final String INGESTION_EVENT_TYPE_ATTRIBUTE = "ingestion_type"; + + static final String EVENT_S3_PARTITION_KEY = "s3_partition_key"; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/S3PartitionCreator.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/S3PartitionCreator.java new file mode 100644 index 0000000000..d2096b6ff5 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/S3PartitionCreator.java @@ -0,0 +1,31 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.converter; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +public class S3PartitionCreator { + private static final Logger LOG = LoggerFactory.getLogger(S3PartitionCreator.class); + private final int partitionCount; + + S3PartitionCreator(final int partitionCount) { + this.partitionCount = partitionCount; + } + + List createPartitions() { + final List partitions = new ArrayList<>(); + for (int i = 0; i < partitionCount; i++) { + String partitionName = String.format("%02x", i) + "/"; + partitions.add(partitionName); + } + LOG.info("S3 partitions created successfully."); + return partitions; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java new file mode 100644 index 0000000000..511876d668 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java @@ -0,0 +1,94 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.converter; + +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; + +/** + * Convert binlog row data into JacksonEvent + */ +public class StreamRecordConverter { + + private static final Logger LOG = LoggerFactory.getLogger(StreamRecordConverter.class); + + private final List folderNames; + + static final String S3_PATH_DELIMITER = "/"; + + static final String STREAM_EVENT_TYPE = "STREAM"; + + public StreamRecordConverter(final int partitionCount) { + S3PartitionCreator s3PartitionCreator = new S3PartitionCreator(partitionCount); + folderNames = s3PartitionCreator.createPartitions(); + } + + public Event convert(Map rowData, + String databaseName, + String tableName, + OpenSearchBulkActions bulkAction, + List primaryKeys, + String s3Prefix) { + final Event event = JacksonEvent.builder() + .withEventType("event") + .withData(rowData) + .build(); + + EventMetadata eventMetadata = event.getMetadata(); + + eventMetadata.setAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE, databaseName); + eventMetadata.setAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE, tableName); + eventMetadata.setAttribute(EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE, bulkAction.toString()); + eventMetadata.setAttribute(INGESTION_EVENT_TYPE_ATTRIBUTE, STREAM_EVENT_TYPE); + + final String primaryKeyValue = primaryKeys.stream() + .map(rowData::get) + .map(String::valueOf) + .collect(Collectors.joining("|")); + eventMetadata.setAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE, primaryKeyValue); + eventMetadata.setAttribute(MetadataKeyAttributes.EVENT_S3_PARTITION_KEY, s3Prefix + S3_PATH_DELIMITER + hashKeyToPartition(primaryKeyValue)); + + return event; + } + + private String hashKeyToPartition(final String key) { + return folderNames.get(hashKeyToIndex(key)); + } + private int hashKeyToIndex(final String key) { + try { + // Create a SHA-256 hash instance + final MessageDigest digest = MessageDigest.getInstance("SHA-256"); + // Hash the key + byte[] hashBytes = digest.digest(key.getBytes()); + // Convert the hash to an integer + int hashValue = bytesToInt(hashBytes); + // Map the hash value to an index in the list + return Math.abs(hashValue) % folderNames.size(); + } catch (final NoSuchAlgorithmException e) { + return -1; + } + } + private int bytesToInt(byte[] bytes) { + return ByteBuffer.wrap(bytes).getInt(); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java index 6213263b09..419f1bf805 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/PartitionFactory.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import java.util.function.Function; @@ -24,15 +25,18 @@ public EnhancedSourcePartition apply(SourcePartitionStoreItem partitionStoreItem String sourceIdentifier = partitionStoreItem.getSourceIdentifier(); String partitionType = sourceIdentifier.substring(sourceIdentifier.lastIndexOf('|') + 1); - if (LeaderPartition.PARTITION_TYPE.equals(partitionType)) { - return new LeaderPartition(partitionStoreItem); - } else if (ExportPartition.PARTITION_TYPE.equals(partitionType)) { - return new ExportPartition(partitionStoreItem); - } else if (DataFilePartition.PARTITION_TYPE.equals(partitionType)) { - return new DataFilePartition(partitionStoreItem); - } else { - // Unable to acquire other partitions. - return new GlobalState(partitionStoreItem); + switch (partitionType) { + case LeaderPartition.PARTITION_TYPE: + return new LeaderPartition(partitionStoreItem); + case ExportPartition.PARTITION_TYPE: + return new ExportPartition(partitionStoreItem); + case DataFilePartition.PARTITION_TYPE: + return new DataFilePartition(partitionStoreItem); + case StreamPartition.PARTITION_TYPE: + return new StreamPartition(partitionStoreItem); + default: + // Unable to acquire other partitions. + return new GlobalState(partitionStoreItem); } } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/GlobalState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/GlobalState.java index c6f1d394a2..4110c1e286 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/GlobalState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/GlobalState.java @@ -11,6 +11,12 @@ import java.util.Map; import java.util.Optional; +/** + * Global State is a special type of partition. The partition type is null. + * You can't acquire (own) a Global State. + * However, you can read and update Global State whenever required. + * The progress state is a Map object. + */ public class GlobalState extends EnhancedSourcePartition> { private final String stateName; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/StreamPartition.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/StreamPartition.java new file mode 100644 index 0000000000..2a3e971ac5 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/partition/StreamPartition.java @@ -0,0 +1,49 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; + +import java.util.Optional; + +public class StreamPartition extends EnhancedSourcePartition { + + public static final String PARTITION_TYPE = "STREAM"; + + private final String dbIdentifier; + private final StreamProgressState state; + + public StreamPartition(String dbIdentifier, StreamProgressState state) { + this.dbIdentifier = dbIdentifier; + this.state = state; + } + + public StreamPartition(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + dbIdentifier = sourcePartitionStoreItem.getSourcePartitionKey(); + state = convertStringToPartitionProgressState(StreamProgressState.class, sourcePartitionStoreItem.getPartitionProgressState()); + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return dbIdentifier; + } + + @Override + public Optional getProgressState() { + if (state != null) { + return Optional.of(state); + } + return Optional.empty(); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java new file mode 100644 index 0000000000..21873179da --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java @@ -0,0 +1,45 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; + +public class StreamProgressState { + + @JsonProperty("startPosition") + private BinlogCoordinate startPosition; + + @JsonProperty("currentPosition") + private BinlogCoordinate currentPosition; + + @JsonProperty("waitForExport") + private boolean waitForExport = false; + + public BinlogCoordinate getStartPosition() { + return startPosition; + } + + public void setStartPosition(BinlogCoordinate startPosition) { + this.startPosition = startPosition; + } + + public BinlogCoordinate getCurrentPosition() { + return currentPosition; + } + + public void setCurrentPosition(BinlogCoordinate currentPosition) { + this.currentPosition = currentPosition; + } + + public boolean shouldWaitForExport() { + return waitForExport; + } + + public void setWaitForExport(boolean waitForExport) { + this.waitForExport = waitForExport; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java index d465d55076..0a2b2fb638 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java @@ -125,7 +125,7 @@ private void processDataFilePartition(DataFilePartition dataFilePartition) { updateLoadStatus(dataFilePartition.getExportTaskId(), DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT); sourceCoordinator.completePartition(dataFilePartition); } else { - LOG.error("There was an exception while processing an S3 data file", (Throwable) ex); + LOG.error("There was an exception while processing an S3 data file", ex); sourceCoordinator.giveUpPartition(dataFilePartition); } numOfWorkers.decrementAndGet(); @@ -153,7 +153,10 @@ private void updateLoadStatus(String exportTaskId, Duration timeout) { try { sourceCoordinator.saveProgressStateForPartition(globalState, null); - // TODO: Stream is enabled and loadStatus.getLoadedFiles() == loadStatus.getTotalFiles(), create global state to indicate that stream can start + if (sourceConfig.isStreamEnabled() && loadStatus.getLoadedFiles() == loadStatus.getTotalFiles()) { + LOG.info("All exports are done, streaming can continue..."); + sourceCoordinator.createPartition(new GlobalState("stream-for-" + sourceConfig.getDbIdentifier(), null)); + } break; } catch (Exception e) { LOG.error("Failed to update the global status, looks like the status was out of date, will retry.."); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index 4831f1e91a..f35975f5b2 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -11,8 +11,10 @@ import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.LeaderProgressState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -101,6 +103,11 @@ private void init() { createExportPartition(sourceConfig, startTime); } + if (sourceConfig.isStreamEnabled()) { + LOG.debug("Stream is enabled. Creating stream partition in the source coordination store."); + createStreamPartition(sourceConfig); + } + LOG.debug("Update initialization state"); LeaderProgressState leaderProgressState = leaderPartition.getProgressState().get(); leaderProgressState.setInitialized(true); @@ -118,4 +125,10 @@ private void createExportPartition(RdsSourceConfig sourceConfig, Instant exportT sourceCoordinator.createPartition(exportPartition); } + private void createStreamPartition(RdsSourceConfig sourceConfig) { + final StreamProgressState progressState = new StreamProgressState(); + progressState.setWaitForExport(sourceConfig.isExportEnabled()); + StreamPartition streamPartition = new StreamPartition(sourceConfig.getDbIdentifier(), progressState); + sourceCoordinator.createPartition(streamPartition); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/BinlogCoordinate.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/BinlogCoordinate.java new file mode 100644 index 0000000000..6818dabe9b --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/BinlogCoordinate.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +public class BinlogCoordinate { + + @JsonProperty("binlogFilename") + private final String binlogFilename; + + @JsonProperty("binlogPosition") + private final long binlogPosition; + + @JsonCreator + public BinlogCoordinate(@JsonProperty("binlogFilename") String binlogFilename, + @JsonProperty("binlogPosition") long binlogPosition) { + this.binlogFilename = binlogFilename; + this.binlogPosition = binlogPosition; + } + + public String getBinlogFilename() { + return binlogFilename; + } + + public long getBinlogPosition() { + return binlogPosition; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/TableMetadata.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/TableMetadata.java new file mode 100644 index 0000000000..f9ce48a3cc --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/TableMetadata.java @@ -0,0 +1,59 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +import java.util.List; + +public class TableMetadata { + private String databaseName; + private String tableName; + private List columnNames; + private List primaryKeys; + + public TableMetadata(String tableName, String databaseName, List columnNames, List primaryKeys) { + this.tableName = tableName; + this.databaseName = databaseName; + this.columnNames = columnNames; + this.primaryKeys = primaryKeys; + } + + public String getDatabaseName() { + return databaseName; + } + + public String getTableName() { + return tableName; + } + + public String getFullTableName() { + return databaseName + "." + tableName; + } + + public List getColumnNames() { + return columnNames; + } + + public List getPrimaryKeys() { + return primaryKeys; + } + + public void setDatabaseName(String databaseName) { + this.databaseName = databaseName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public void setColumnNames(List columnNames) { + this.columnNames = columnNames; + } + + public void setPrimaryKeys(List primaryKeys) { + this.primaryKeys = primaryKeys; + } + +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java new file mode 100644 index 0000000000..8802d7de46 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.DBInstance; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; + +public class BinlogClientFactory { + + private final RdsSourceConfig sourceConfig; + + private final RdsClient rdsClient; + + public BinlogClientFactory(final RdsSourceConfig sourceConfig, final RdsClient rdsClient) { + this.sourceConfig = sourceConfig; + this.rdsClient = rdsClient; + } + + public BinaryLogClient create() { + DBInstance dbInstance = describeDbInstance(sourceConfig.getDbIdentifier()); + return new BinaryLogClient( + dbInstance.endpoint().address(), + dbInstance.endpoint().port(), + // For test + // "127.0.0.1", + // 3306, + sourceConfig.getAuthenticationConfig().getUsername(), + sourceConfig.getAuthenticationConfig().getPassword()); + } + + private DBInstance describeDbInstance(final String dbInstanceIdentifier) { + DescribeDbInstancesRequest request = DescribeDbInstancesRequest.builder() + .dbInstanceIdentifier(dbInstanceIdentifier) + .build(); + + DescribeDbInstancesResponse response = rdsClient.describeDBInstances(request); + return response.dbInstances().get(0); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java new file mode 100644 index 0000000000..7ac0dcbe2b --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java @@ -0,0 +1,209 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.event.DeleteRowsEventData; +import com.github.shyiko.mysql.binlog.event.EventType; +import com.github.shyiko.mysql.binlog.event.TableMapEventData; +import com.github.shyiko.mysql.binlog.event.TableMapEventMetadata; +import com.github.shyiko.mysql.binlog.event.UpdateRowsEventData; +import com.github.shyiko.mysql.binlog.event.WriteRowsEventData; +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.converter.StreamRecordConverter; +import org.opensearch.dataprepper.plugins.source.rds.model.TableMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; +import java.time.Duration; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class BinlogEventListener implements BinaryLogClient.EventListener { + + private static final Logger LOG = LoggerFactory.getLogger(BinlogEventListener.class); + + static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); + static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; + + /** + * TableId to TableMetadata mapping + */ + private final Map tableMetadataMap; + + private final StreamRecordConverter recordConverter; + private final BufferAccumulator> bufferAccumulator; + private final List tableNames; + private final String s3Prefix; + + public BinlogEventListener(final Buffer> buffer, final RdsSourceConfig sourceConfig) { + tableMetadataMap = new HashMap<>(); + recordConverter = new StreamRecordConverter(sourceConfig.getStream().getPartitionCount()); + bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); + s3Prefix = sourceConfig.getS3Prefix(); + tableNames = sourceConfig.getTableNames(); + } + + @Override + public void onEvent(com.github.shyiko.mysql.binlog.event.Event event) { + EventType eventType = event.getHeader().getEventType(); + + switch (eventType) { + case TABLE_MAP: + handleTableMapEvent(event); + break; + case WRITE_ROWS: + case EXT_WRITE_ROWS: + handleInsertEvent(event); + break; + case UPDATE_ROWS: + case EXT_UPDATE_ROWS: + handleUpdateEvent(event); + break; + case DELETE_ROWS: + case EXT_DELETE_ROWS: + handleDeleteEvent(event); + break; + } + } + + void handleTableMapEvent(com.github.shyiko.mysql.binlog.event.Event event) { + final TableMapEventData data = event.getData(); + final TableMapEventMetadata tableMapEventMetadata = data.getEventMetadata(); + final List columnNames = tableMapEventMetadata.getColumnNames(); + final List primaryKeys = tableMapEventMetadata.getSimplePrimaryKeys().stream() + .map(columnNames::get) + .collect(Collectors.toList()); + final TableMetadata tableMetadata = new TableMetadata( + data.getTable(), data.getDatabase(), columnNames, primaryKeys); + if (isTableOfInterest(tableMetadata.getFullTableName())) { + tableMetadataMap.put(data.getTableId(), tableMetadata); + } + } + + void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { + // get new row data from the event + LOG.debug("Handling insert event"); + final WriteRowsEventData data = event.getData(); + if (!tableMetadataMap.containsKey(data.getTableId())) { + LOG.debug("Cannot find table metadata, the event is likely not from a table of interest or the table metadata was not read"); + return; + } + final TableMetadata tableMetadata = tableMetadataMap.get(data.getTableId()); + final String fullTableName = tableMetadata.getFullTableName(); + if (!isTableOfInterest(fullTableName)) { + LOG.debug("The event is not from a table of interest"); + return; + } + final List columnNames = tableMetadata.getColumnNames(); + final List primaryKeys = tableMetadata.getPrimaryKeys(); + + // Construct data prepper JacksonEvent + for (final Object[] rowDataArray : data.getRows()) { + final Map rowDataMap = new HashMap<>(); + for (int i = 0; i < rowDataArray.length; i++) { + rowDataMap.put(columnNames.get(i), rowDataArray[i]); + } + + Event pipelineEvent = recordConverter.convert( + rowDataMap, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), OpenSearchBulkActions.INDEX, primaryKeys, s3Prefix); + addToBuffer(new Record<>(pipelineEvent)); + } + + flushBuffer(); + } + + void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { + LOG.debug("Handling update event"); + final UpdateRowsEventData data = event.getData(); + if (!tableMetadataMap.containsKey(data.getTableId())) { + return; + } + final TableMetadata tableMetadata = tableMetadataMap.get(data.getTableId()); + final String fullTableName = tableMetadata.getFullTableName(); + if (!isTableOfInterest(fullTableName)) { + LOG.debug("The event is not from a table of interest"); + return; + } + final List columnNames = tableMetadata.getColumnNames(); + final List primaryKeys = tableMetadata.getPrimaryKeys(); + + for (Map.Entry updatedRow : data.getRows()) { + // updatedRow contains data before update as key and data after update as value + final Object[] rowData = updatedRow.getValue(); + + final Map dataMap = new HashMap<>(); + for (int i = 0; i < rowData.length; i++) { + dataMap.put(columnNames.get(i), rowData[i]); + } + + final Event pipelineEvent = recordConverter.convert( + dataMap, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), OpenSearchBulkActions.INDEX, primaryKeys, s3Prefix); + addToBuffer(new Record<>(pipelineEvent)); + } + + flushBuffer(); + } + + void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { + LOG.debug("Handling delete event"); + final DeleteRowsEventData data = event.getData(); + if (!tableMetadataMap.containsKey(data.getTableId())) { + LOG.debug("Cannot find table metadata, the event is likely not from a table of interest or the table metadata was not read"); + return; + } + final TableMetadata tableMetadata = tableMetadataMap.get(data.getTableId()); + final String fullTableName = tableMetadata.getFullTableName(); + if (!isTableOfInterest(fullTableName)) { + LOG.debug("The event is not from a table of interest"); + return; + } + final List columnNames = tableMetadata.getColumnNames(); + final List primaryKeys = tableMetadata.getPrimaryKeys(); + + for (Object[] rowDataArray : data.getRows()) { + final Map rowDataMap = new HashMap<>(); + for (int i = 0; i < rowDataArray.length; i++) { + rowDataMap.put(columnNames.get(i), rowDataArray[i]); + } + + final Event pipelineEvent = recordConverter.convert( + rowDataMap, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), OpenSearchBulkActions.DELETE, primaryKeys, s3Prefix); + addToBuffer(new Record<>(pipelineEvent)); + } + + flushBuffer(); + } + + private boolean isTableOfInterest(String tableName) { + return new HashSet<>(tableNames).contains(tableName); + } + + private void addToBuffer(final Record record) { + try { + bufferAccumulator.add(record); + } catch (Exception e) { + LOG.error("Failed to add event to buffer", e); + } + } + + private void flushBuffer() { + try { + bufferAccumulator.flush(); + } catch (Exception e) { + LOG.error("Failed to flush buffer", e); + } + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java new file mode 100644 index 0000000000..0b42c95c38 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java @@ -0,0 +1,85 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + + +public class StreamScheduler implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(StreamScheduler.class); + + private static final int DEFAULT_TAKE_LEASE_INTERVAL_MILLIS = 60_000; + + private final EnhancedSourceCoordinator sourceCoordinator; + private final RdsSourceConfig sourceConfig; + private final BinaryLogClient binaryLogClient; + private final PluginMetrics pluginMetrics; + + private volatile boolean shutdownRequested = false; + + public StreamScheduler(final EnhancedSourceCoordinator sourceCoordinator, + final RdsSourceConfig sourceConfig, + final BinaryLogClient binaryLogClient, + final Buffer> buffer, + final PluginMetrics pluginMetrics) { + this.sourceCoordinator = sourceCoordinator; + this.sourceConfig = sourceConfig; + this.binaryLogClient = binaryLogClient; + this.binaryLogClient.registerEventListener(new BinlogEventListener(buffer, sourceConfig)); + this.pluginMetrics = pluginMetrics; + } + + @Override + public void run() { + LOG.debug("Start running Stream Scheduler"); + while (!shutdownRequested && !Thread.currentThread().isInterrupted()) { + try { + final Optional sourcePartition = sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE); + if (sourcePartition.isPresent()) { + LOG.info("Acquired partition to read from stream"); + + final StreamPartition streamPartition = (StreamPartition) sourcePartition.get(); + final StreamWorker streamWorker = StreamWorker.create(sourceCoordinator, binaryLogClient, pluginMetrics); + streamWorker.processStream(streamPartition); + } + + try { + LOG.debug("Waiting to acquire stream partition."); + Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException e) { + LOG.info("The StreamScheduler was interrupted while waiting to retry, stopping processing"); + break; + } + + } catch (Exception e) { + LOG.error("Received an exception during stream processing, backing off and retrying", e); + try { + Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException ex) { + LOG.info("The StreamScheduler was interrupted while waiting to retry, stopping processing"); + break; + } + } + } + } + + public void shutdown() { + shutdownRequested = true; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java new file mode 100644 index 0000000000..ce130d0d12 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java @@ -0,0 +1,95 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Optional; + +public class StreamWorker { + private static final Logger LOG = LoggerFactory.getLogger(StreamWorker.class); + + private static final int DEFAULT_EXPORT_COMPLETE_WAIT_INTERVAL_MILLIS = 60_000; + + private final EnhancedSourceCoordinator sourceCoordinator; + private final BinaryLogClient binaryLogClient; + private final PluginMetrics pluginMetrics; + + StreamWorker(final EnhancedSourceCoordinator sourceCoordinator, + final BinaryLogClient binaryLogClient, + final PluginMetrics pluginMetrics) { + this.sourceCoordinator = sourceCoordinator; + this.binaryLogClient = binaryLogClient; + this.pluginMetrics = pluginMetrics; + } + + public static StreamWorker create(final EnhancedSourceCoordinator sourceCoordinator, + final BinaryLogClient binaryLogClient, + final PluginMetrics pluginMetrics) { + return new StreamWorker(sourceCoordinator, binaryLogClient, pluginMetrics); + } + + public void processStream(final StreamPartition streamPartition) { + // get current binlog position + BinlogCoordinate currentBinlogCoords = streamPartition.getProgressState().get().getCurrentPosition(); + + // set start of binlog stream to current position if exists + if (currentBinlogCoords != null) { + final String binlogFilename = currentBinlogCoords.getBinlogFilename(); + final long binlogPosition = currentBinlogCoords.getBinlogPosition(); + LOG.debug("Will start binlog stream from binlog file {} and position {}.", binlogFilename, binlogPosition); + binaryLogClient.setBinlogFilename(binlogFilename); + binaryLogClient.setBinlogPosition(binlogPosition); + } + + while (shouldWaitForExport(streamPartition) && !Thread.currentThread().isInterrupted()) { + LOG.info("Initial load not completed yet for {}, waiting...", streamPartition.getPartitionKey()); + try { + Thread.sleep(DEFAULT_EXPORT_COMPLETE_WAIT_INTERVAL_MILLIS); + } catch (final InterruptedException ex) { + LOG.info("The StreamScheduler was interrupted while waiting to retry, stopping processing"); + Thread.currentThread().interrupt(); + break; + } + } + + try { + LOG.info("Connecting to binary log stream."); + binaryLogClient.connect(); + } catch (IOException e) { + throw new RuntimeException(e); + } finally { + try { + binaryLogClient.disconnect(); + } catch (IOException e) { + LOG.error("Binary log client failed to disconnect.", e); + } + } + } + + private boolean shouldWaitForExport(final StreamPartition streamPartition) { + if (!streamPartition.getProgressState().get().shouldWaitForExport()) { + LOG.debug("Export is not enabled. Proceed with streaming."); + return false; + } + + return !isExportDone(streamPartition); + } + + private boolean isExportDone(StreamPartition streamPartition) { + final String dbIdentifier = streamPartition.getPartitionKey(); + Optional globalStatePartition = sourceCoordinator.getPartition("stream-for-" + dbIdentifier); + return globalStatePartition.isPresent(); + } +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/S3PartitionCreatorTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/S3PartitionCreatorTest.java new file mode 100644 index 0000000000..1018d90865 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/S3PartitionCreatorTest.java @@ -0,0 +1,34 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.converter; + +import org.junit.jupiter.api.Test; + +import java.util.HashSet; +import java.util.List; +import java.util.Random; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; + + +class S3PartitionCreatorTest { + + @Test + void test_createPartition_create_correct_number_of_distinct_partition_strings() { + final int partitionCount = new Random().nextInt(10) + 1; + final S3PartitionCreator s3PartitionCreator = createObjectUnderTest(partitionCount); + + final List partitions = s3PartitionCreator.createPartitions(); + + assertThat(partitions.size(), is(partitionCount)); + assertThat(new HashSet<>(partitions).size(), is(partitionCount)); + } + + private S3PartitionCreator createObjectUnderTest(final int partitionCount) { + return new S3PartitionCreator(partitionCount); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java new file mode 100644 index 0000000000..8ccecc3ff1 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java @@ -0,0 +1,60 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.converter; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; + +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.startsWith; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_S3_PARTITION_KEY; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.StreamRecordConverter.S3_PATH_DELIMITER; + + +class StreamRecordConverterTest { + + private StreamRecordConverter streamRecordConverter; + + @BeforeEach + void setUp() { + streamRecordConverter = createObjectUnderTest(); + } + + @Test + void test_convert_returns_expected_event() { + Map rowData = Map.of("key1", "value1", "key2", "value2"); + final String databaseName = UUID.randomUUID().toString(); + final String tableName = UUID.randomUUID().toString(); + final OpenSearchBulkActions bulkAction = OpenSearchBulkActions.INDEX; + final List primaryKeys = List.of("key1"); + final String s3Prefix = UUID.randomUUID().toString(); + + Event event = streamRecordConverter.convert(rowData, databaseName, tableName, bulkAction, primaryKeys, s3Prefix); + + assertThat(event.toMap(), is(rowData)); + assertThat(event.getMetadata().getAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE), is(databaseName)); + assertThat(event.getMetadata().getAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE), is(tableName)); + assertThat(event.getMetadata().getAttribute(EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE), is(bulkAction.toString())); + assertThat(event.getMetadata().getAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE), is("value1")); + assertThat(event.getMetadata().getAttribute(EVENT_S3_PARTITION_KEY).toString(), startsWith(s3Prefix + S3_PATH_DELIMITER)); + } + + private StreamRecordConverter createObjectUnderTest() { + return new StreamRecordConverter(new Random().nextInt(1000) + 1); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java new file mode 100644 index 0000000000..52ba312b5a --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java @@ -0,0 +1,66 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +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.source.rds.RdsSourceConfig; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.DBInstance; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; + +import java.util.List; +import java.util.Random; +import java.util.UUID; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class BinlogClientFactoryTest { + + @Mock + private RdsSourceConfig sourceConfig; + + @Mock + private RdsClient rdsClient; + + private BinlogClientFactory binlogClientFactory; + private Random random; + + @BeforeEach + void setUp() { + binlogClientFactory = createBinlogClientFactory(); + random = new Random(); + } + + @Test + void test_create() { + DescribeDbInstancesResponse describeDbInstancesResponse = mock(DescribeDbInstancesResponse.class); + DBInstance dbInstance = mock(DBInstance.class, RETURNS_DEEP_STUBS); + final String address = UUID.randomUUID().toString(); + final Integer port = random.nextInt(); + when(dbInstance.endpoint().address()).thenReturn(address); + when(dbInstance.endpoint().port()).thenReturn(port); + when(describeDbInstancesResponse.dbInstances()).thenReturn(List.of(dbInstance)); + when(sourceConfig.getDbIdentifier()).thenReturn(UUID.randomUUID().toString()); + when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); + RdsSourceConfig.AuthenticationConfig authenticationConfig = mock(RdsSourceConfig.AuthenticationConfig.class); + when(sourceConfig.getAuthenticationConfig()).thenReturn(authenticationConfig); + + binlogClientFactory.create(); + } + + private BinlogClientFactory createBinlogClientFactory() { + return new BinlogClientFactory(sourceConfig, rdsClient); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java new file mode 100644 index 0000000000..406a89cec9 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java @@ -0,0 +1,92 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import com.github.shyiko.mysql.binlog.event.EventType; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; +import org.mockito.Answers; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; + +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class BinlogEventListenerTest { + + @Mock + private Buffer> buffer; + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private RdsSourceConfig sourceConfig; + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private com.github.shyiko.mysql.binlog.event.Event binlogEvent; + + private static BinlogEventListener objectUnderTest; + + @BeforeEach + void setUp() { + objectUnderTest = spy(createObjectUnderTest()); + } + + @Test + void test_given_TableMap_event_then_calls_correct_handler() { + when(binlogEvent.getHeader().getEventType()).thenReturn(EventType.TABLE_MAP); + doNothing().when(objectUnderTest).handleTableMapEvent(binlogEvent); + + objectUnderTest.onEvent(binlogEvent); + + verify(objectUnderTest).handleTableMapEvent(binlogEvent); + } + + @ParameterizedTest + @EnumSource(names = {"WRITE_ROWS", "EXT_WRITE_ROWS"}) + void test_given_WriteRows_event_then_calls_correct_handler(EventType eventType) { + when(binlogEvent.getHeader().getEventType()).thenReturn(eventType); + doNothing().when(objectUnderTest).handleInsertEvent(binlogEvent); + + objectUnderTest.onEvent(binlogEvent); + + verify(objectUnderTest).handleInsertEvent(binlogEvent); + } + + @ParameterizedTest + @EnumSource(names = {"UPDATE_ROWS", "EXT_UPDATE_ROWS"}) + void test_given_UpdateRows_event_then_calls_correct_handler(EventType eventType) { + when(binlogEvent.getHeader().getEventType()).thenReturn(eventType); + doNothing().when(objectUnderTest).handleUpdateEvent(binlogEvent); + + objectUnderTest.onEvent(binlogEvent); + + verify(objectUnderTest).handleUpdateEvent(binlogEvent); + } + + @ParameterizedTest + @EnumSource(names = {"DELETE_ROWS", "EXT_DELETE_ROWS"}) + void test_given_DeleteRows_event_then_calls_correct_handler(EventType eventType) { + when(binlogEvent.getHeader().getEventType()).thenReturn(eventType); + doNothing().when(objectUnderTest).handleDeleteEvent(binlogEvent); + + objectUnderTest.onEvent(binlogEvent); + + verify(objectUnderTest).handleDeleteEvent(binlogEvent); + } + + private BinlogEventListener createObjectUnderTest() { + return new BinlogEventListener(buffer, sourceConfig); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java new file mode 100644 index 0000000000..1a152137ee --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java @@ -0,0 +1,116 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; + +import java.time.Duration; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class StreamSchedulerTest { + + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private RdsSourceConfig sourceConfig; + + @Mock + private BinaryLogClient binaryLogClient; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private Buffer> buffer; + + private StreamScheduler objectUnderTest; + + @BeforeEach + void setUp() { + objectUnderTest = createObjectUnderTest(); + } + + @Test + void test_given_no_stream_partition_then_no_stream_actions() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE)).thenReturn(Optional.empty()); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(objectUnderTest); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(StreamPartition.PARTITION_TYPE)); + Thread.sleep(100); + executorService.shutdownNow(); + + verify(binaryLogClient).registerEventListener(any(BinlogEventListener.class)); + verifyNoMoreInteractions(binaryLogClient); + } + + @Test + void test_given_stream_partition_then_start_stream() throws InterruptedException { + final StreamPartition streamPartition = mock(StreamPartition.class); + when(sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE)).thenReturn(Optional.of(streamPartition)); + + StreamWorker streamWorker = mock(StreamWorker.class); + doNothing().when(streamWorker).processStream(streamPartition); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> { + try (MockedStatic streamWorkerMockedStatic = mockStatic(StreamWorker.class)) { + streamWorkerMockedStatic.when(() -> StreamWorker.create(sourceCoordinator, binaryLogClient, pluginMetrics)) + .thenReturn(streamWorker); + objectUnderTest.run(); + } + + }); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(StreamPartition.PARTITION_TYPE)); + Thread.sleep(100); + executorService.shutdownNow(); + + verify(streamWorker).processStream(streamPartition); + } + + @Test + void test_shutdown() { + lenient().when(sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE)).thenReturn(Optional.empty()); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(objectUnderTest); + objectUnderTest.shutdown(); + verifyNoMoreInteractions(sourceCoordinator); + executorService.shutdownNow(); + } + + private StreamScheduler createObjectUnderTest() { + return new StreamScheduler(sourceCoordinator, sourceConfig, binaryLogClient, buffer, pluginMetrics); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java new file mode 100644 index 0000000000..4dd3930466 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java @@ -0,0 +1,85 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +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.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; + +import java.io.IOException; +import java.util.Optional; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class StreamWorkerTest { + + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + + @Mock + private BinaryLogClient binaryLogClient; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private StreamPartition streamPartition; + + private StreamWorker streamWorker; + + @BeforeEach + void setUp() { + streamWorker = createObjectUnderTest(); + } + + @Test + void test_processStream_with_given_binlog_coordinates() throws IOException { + StreamProgressState streamProgressState = mock(StreamProgressState.class); + when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); + final String binlogFilename = "binlog-001"; + final Long binlogPosition = 100L; + when(streamProgressState.getCurrentPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); + when(streamProgressState.shouldWaitForExport()).thenReturn(false); + + streamWorker.processStream(streamPartition); + + verify(binaryLogClient).setBinlogFilename(binlogFilename); + verify(binaryLogClient).setBinlogPosition(binlogPosition); + verify(binaryLogClient).connect(); + } + + @Test + void test_processStream_without_current_binlog_coordinates() throws IOException { + StreamProgressState streamProgressState = mock(StreamProgressState.class); + when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); + final String binlogFilename = "binlog-001"; + final Long binlogPosition = 100L; + when(streamProgressState.getCurrentPosition()).thenReturn(null); + when(streamProgressState.shouldWaitForExport()).thenReturn(false); + + streamWorker.processStream(streamPartition); + + verify(binaryLogClient, never()).setBinlogFilename(binlogFilename); + verify(binaryLogClient, never()).setBinlogPosition(binlogPosition); + verify(binaryLogClient).connect(); + } + + private StreamWorker createObjectUnderTest() { + return new StreamWorker(sourceCoordinator, binaryLogClient, pluginMetrics); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java index 2b1d6b3e49..3cddb1a2e8 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java @@ -152,6 +152,7 @@ public void run() { } for (String partitionKey: partitionKeys) { + LOG.debug("Scan object worker is stopped, giving up partitions."); sourceCoordinator.giveUpPartition(partitionKey); } } @@ -203,6 +204,7 @@ private void startProcessingObject(final long waitTimeMillis) { deleteObjectsForPartition.forEach(s3ObjectDeleteWorker::deleteS3Object); objectsToDeleteForAcknowledgmentSets.remove(objectToProcess.get().getPartitionKey()); } else { + LOG.debug("Did not receive positive acknowledgement, giving up partition."); sourceCoordinator.giveUpPartition(objectToProcess.get().getPartitionKey()); } partitionKeys.remove(objectToProcess.get().getPartitionKey()); @@ -268,7 +270,7 @@ private void processFolderPartition(final SourcePartition sourceCoordinator.deletePartition(folderPartition.getPartitionKey()); return; } - + LOG.debug("No objects to process, giving up partition"); sourceCoordinator.giveUpPartition(folderPartition.getPartitionKey(), Instant.now()); return; } From 5c7d58c03059c7a753d882f5b74fa6ed32f45641 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Tue, 23 Jul 2024 17:03:57 -0500 Subject: [PATCH 085/159] Revert "Run tests on the current JVM for Java 17 & 21 / Gradle 8.8 (#4730)" (#4762) This reverts commit 67f3595805f07442d8f05823c9959b50358aa4d9. Signed-off-by: Hai Yan --- build.gradle | 3 -- .../dataprepper/model/sink/AbstractSink.java | 6 ++-- .../dataprepper/model/sink/SinkThread.java | 8 +---- .../model/sink/AbstractSinkTest.java | 22 +++++------- data-prepper-core/build.gradle | 3 ++ .../avro/AvroAutoSchemaGeneratorTest.java | 4 +-- .../blockingbuffer/BlockingBufferTests.java | 2 +- .../event_json/EventJsonInputCodecTest.java | 34 ++++++++----------- .../EventJsonInputOutputCodecTest.java | 26 ++++++-------- .../event_json/EventJsonOutputCodecTest.java | 10 +++--- gradle/wrapper/gradle-wrapper.properties | 2 +- gradlew | 2 +- 12 files changed, 50 insertions(+), 72 deletions(-) diff --git a/build.gradle b/build.gradle index 3dccd497cf..f77ecc442b 100644 --- a/build.gradle +++ b/build.gradle @@ -226,9 +226,6 @@ subprojects { test { useJUnitPlatform() - javaLauncher = javaToolchains.launcherFor { - languageVersion = JavaLanguageVersion.current() - } reports { junitXml.required html.required diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java index 26dd7e98a6..1c3e596265 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java @@ -28,7 +28,6 @@ public abstract class AbstractSink> implements Sink { private Thread retryThread; private int maxRetries; private int waitTimeMs; - private SinkThread sinkThread; public AbstractSink(final PluginSetting pluginSetting, int numRetries, int waitTimeMs) { this.pluginMetrics = PluginMetrics.fromPluginSetting(pluginSetting); @@ -52,8 +51,7 @@ public void initialize() { // the exceptions which are not retryable. doInitialize(); if (!isReady() && retryThread == null) { - sinkThread = new SinkThread(this, maxRetries, waitTimeMs); - retryThread = new Thread(sinkThread); + retryThread = new Thread(new SinkThread(this, maxRetries, waitTimeMs)); retryThread.start(); } } @@ -78,7 +76,7 @@ public void output(Collection records) { @Override public void shutdown() { if (retryThread != null) { - sinkThread.stop(); + retryThread.stop(); } } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java index 451cef7dff..c304de37af 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java @@ -10,8 +10,6 @@ class SinkThread implements Runnable { private int maxRetries; private int waitTimeMs; - private volatile boolean isStopped = false; - public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) { this.sink = sink; this.maxRetries = maxRetries; @@ -21,15 +19,11 @@ public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) { @Override public void run() { int numRetries = 0; - while (!sink.isReady() && numRetries++ < maxRetries && !isStopped) { + while (!sink.isReady() && numRetries++ < maxRetries) { try { Thread.sleep(waitTimeMs); sink.doInitialize(); } catch (InterruptedException e){} } } - - public void stop() { - isStopped = true; - } } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java index 8d1af7ea44..3b9fe7c007 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java @@ -11,10 +11,15 @@ import org.opensearch.dataprepper.metrics.MetricNames; import org.opensearch.dataprepper.metrics.MetricsTestUtil; import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.EventHandle; import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.event.EventHandle; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.mock; import java.time.Duration; import java.util.Arrays; @@ -25,12 +30,6 @@ import java.util.UUID; import static org.awaitility.Awaitility.await; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; class AbstractSinkTest { private int count; @@ -72,13 +71,13 @@ void testMetrics() { } @Test - void testSinkNotReady() throws InterruptedException { + void testSinkNotReady() { final String sinkName = "testSink"; final String pipelineName = "pipelineName"; MetricsTestUtil.initMetrics(); PluginSetting pluginSetting = new PluginSetting(sinkName, Collections.emptyMap()); pluginSetting.setPipelineName(pipelineName); - AbstractSinkNotReadyImpl abstractSink = new AbstractSinkNotReadyImpl(pluginSetting); + AbstractSink> abstractSink = new AbstractSinkNotReadyImpl(pluginSetting); abstractSink.initialize(); assertEquals(abstractSink.isReady(), false); assertEquals(abstractSink.getRetryThreadState(), Thread.State.RUNNABLE); @@ -88,10 +87,7 @@ void testSinkNotReady() throws InterruptedException { await().atMost(Duration.ofSeconds(5)) .until(abstractSink::isReady); assertEquals(abstractSink.getRetryThreadState(), Thread.State.TERMINATED); - int initCountBeforeShutdown = abstractSink.initCount; abstractSink.shutdown(); - Thread.sleep(200); - assertThat(abstractSink.initCount, equalTo(initCountBeforeShutdown)); } @Test diff --git a/data-prepper-core/build.gradle b/data-prepper-core/build.gradle index c939129a1c..080538c5e4 100644 --- a/data-prepper-core/build.gradle +++ b/data-prepper-core/build.gradle @@ -48,6 +48,7 @@ dependencies { exclude group: 'commons-logging', module: 'commons-logging' } implementation 'software.amazon.cloudwatchlogs:aws-embedded-metrics:2.0.0-beta-1' + testImplementation 'org.apache.logging.log4j:log4j-jpl:2.23.0' testImplementation testLibs.spring.test implementation libs.armeria.core implementation libs.armeria.grpc @@ -88,6 +89,8 @@ task integrationTest(type: Test) { classpath = sourceSets.integrationTest.runtimeClasspath + systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties' + filter { includeTestsMatching '*IT' } diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java index 1b66b62c37..622eb56a1b 100644 --- a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java @@ -17,7 +17,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Timer; +import java.util.Random; import java.util.UUID; import java.util.stream.Stream; @@ -218,7 +218,7 @@ static class SomeUnknownTypesArgumentsProvider implements ArgumentsProvider { @Override public Stream provideArguments(ExtensionContext context) { return Stream.of( - arguments(Timer.class), + arguments(Random.class), arguments(InputStream.class), arguments(File.class) ); diff --git a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java index f3f28db174..194c810ec4 100644 --- a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java +++ b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java @@ -328,7 +328,7 @@ public Stream provideArguments(final ExtensionContext conte return Stream.of( Arguments.of(0, randomInt + 1, 0.0), Arguments.of(1, 100, 1.0), - Arguments.of(randomInt + 1, randomInt + 1, 100.0), + Arguments.of(randomInt, randomInt, 100.0), Arguments.of(randomInt, randomInt + 250, ((double) randomInt / (randomInt + 250)) * 100), Arguments.of(6, 9, 66.66666666666666), Arguments.of(531, 1000, 53.1), diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java index a4b0377963..f85d1c6605 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java @@ -11,12 +11,9 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; - import static org.mockito.Mockito.when; import static org.mockito.Mockito.mock; - import org.mockito.Mock; - import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.CoreMatchers.not; @@ -31,7 +28,6 @@ import java.io.ByteArrayInputStream; import java.time.Instant; -import java.time.temporal.ChronoUnit; import java.util.List; import java.util.LinkedList; import java.util.Map; @@ -60,7 +56,7 @@ public EventJsonInputCodec createInputCodec() { @ParameterizedTest @ValueSource(strings = {"", "{}"}) public void emptyTest(String input) throws Exception { - input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":[" + input + "]}"; + input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["+input+"]}"; ByteArrayInputStream inputStream = new ByteArrayInputStream(input.getBytes()); inputCodec = createInputCodec(); Consumer> consumer = mock(Consumer.class); @@ -74,15 +70,15 @@ public void inCompatibleVersionTest() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); + Instant startTime = Instant.now(); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\"" + EventJsonDefines.VERSION + "\":\"3.0\", \"" + EventJsonDefines.EVENTS + "\":["; + String input = "{\""+EventJsonDefines.VERSION+"\":\"3.0\", \""+EventJsonDefines.EVENTS+"\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; + input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; comma = ","; } input += "]}"; @@ -99,15 +95,15 @@ public void basicTest() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); + Instant startTime = Instant.now(); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":["; + String input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; + input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; comma = ","; } input += "]}"; @@ -115,8 +111,8 @@ public void basicTest() throws Exception { List> records = new LinkedList<>(); inputCodec.parse(inputStream, records::add); assertThat(records.size(), equalTo(2)); - for (Record record : records) { - Event e = (Event) record.getData(); + for(Record record : records) { + Event e = (Event)record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -130,15 +126,15 @@ public void test_with_timeReceivedOverridden() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS).minusSeconds(5); + Instant startTime = Instant.now().minusSeconds(5); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":["; + String input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; + input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; comma = ","; } input += "]}"; @@ -146,8 +142,8 @@ public void test_with_timeReceivedOverridden() throws Exception { List> records = new LinkedList<>(); inputCodec.parse(inputStream, records::add); assertThat(records.size(), equalTo(2)); - for (Record record : records) { - Event e = (Event) record.getData(); + for(Record record : records) { + Event e = (Event)record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), not(equalTo(startTime))); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -163,7 +159,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent) logBuilder.build(); + final JacksonEvent event = (JacksonEvent)logBuilder.build(); return event; } diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java index 7ea8c49cd0..85e91e5a55 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java @@ -6,12 +6,9 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; - import static org.mockito.Mockito.when; import static org.mockito.Mockito.mock; - import org.mockito.Mock; - import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -25,7 +22,6 @@ import org.opensearch.dataprepper.model.log.JacksonLog; import java.time.Instant; -import java.time.temporal.ChronoUnit; import java.util.List; import java.util.LinkedList; import java.util.Map; @@ -68,7 +64,7 @@ public void basicTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); + Instant startTime = Instant.now(); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); inputCodec = createInputCodec(); @@ -79,8 +75,8 @@ public void basicTest() throws Exception { inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(1)); - for (Record record : records) { - Event e = (Event) record.getData(); + for(Record record : records) { + Event e = (Event)record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -94,7 +90,7 @@ public void multipleEventsTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); + Instant startTime = Instant.now(); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); inputCodec = createInputCodec(); @@ -107,8 +103,8 @@ public void multipleEventsTest() throws Exception { inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(3)); - for (Record record : records) { - Event e = (Event) record.getData(); + for(Record record : records) { + Event e = (Event)record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -126,7 +122,7 @@ public void extendedTest() throws Exception { Set tags = Set.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); List tagsList = tags.stream().collect(Collectors.toList()); - Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); + Instant startTime = Instant.now(); Event event = createEvent(data, startTime); Instant origTime = startTime.minusSeconds(5); event.getMetadata().setExternalOriginationTime(origTime); @@ -139,11 +135,11 @@ public void extendedTest() throws Exception { outputCodec.complete(outputStream); assertThat(outputCodec.getExtension(), equalTo(EventJsonOutputCodec.EVENT_JSON)); List> records = new LinkedList<>(); - inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); +inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(1)); - for (Record record : records) { - Event e = (Event) record.getData(); + for(Record record : records) { + Event e = (Event)record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags(), equalTo(tags)); @@ -161,7 +157,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent) logBuilder.build(); + final JacksonEvent event = (JacksonEvent)logBuilder.build(); return event; } diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java index b32d2b62e9..51dda545cb 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java @@ -11,7 +11,6 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mock; - import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -23,7 +22,6 @@ import org.opensearch.dataprepper.model.log.JacksonLog; import java.time.Instant; -import java.time.temporal.ChronoUnit; import java.util.Map; import java.util.UUID; @@ -51,7 +49,7 @@ public void basicTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); + Instant startTime = Instant.now(); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); outputCodec.start(outputStream, null, null); @@ -61,10 +59,10 @@ public void basicTest() throws Exception { Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); //String expectedOutput = "{\"version\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\",\""+EventJsonDefines.EVENTS+"\":["; - String expectedOutput = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\",\"" + EventJsonDefines.EVENTS + "\":["; + String expectedOutput = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\",\""+EventJsonDefines.EVENTS+"\":["; String comma = ""; for (int i = 0; i < 2; i++) { - expectedOutput += comma + "{\"" + EventJsonDefines.DATA + "\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"" + EventJsonDefines.METADATA + "\":" + objectMapper.writeValueAsString(metadataMap) + "}"; + expectedOutput += comma+"{\""+EventJsonDefines.DATA+"\":"+objectMapper.writeValueAsString(dataMap)+","+"\""+EventJsonDefines.METADATA+"\":"+objectMapper.writeValueAsString(metadataMap)+"}"; comma = ","; } expectedOutput += "]}"; @@ -80,7 +78,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent) logBuilder.build(); + final JacksonEvent event = (JacksonEvent)logBuilder.build(); return event; } diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index a4413138c9..b82aa23a4f 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-8.8-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.7-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index b740cf1339..1aa94a4269 100755 --- a/gradlew +++ b/gradlew @@ -55,7 +55,7 @@ # Darwin, MinGW, and NonStop. # # (3) This script is generated from the Groovy template -# https://github.com/gradle/gradle/blob/HEAD/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt +# https://github.com/gradle/gradle/blob/HEAD/subprojects/plugins/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt # within the Gradle project. # # You can find Gradle at https://github.com/gradle/gradle/. From fe16d8f8a640b43d08686b15ce2a3e79d685ec5d Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Wed, 24 Jul 2024 22:52:38 -0500 Subject: [PATCH 086/159] Add support for taking snapshots on RDS/Aurora Clusters (#4761) Signed-off-by: Hai Yan --- .../plugins/source/rds/RdsService.java | 12 +- .../plugins/source/rds/RdsSourceConfig.java | 2 +- .../rds/export/ClusterSnapshotStrategy.java | 64 ++++++++++ .../source/rds/export/ExportScheduler.java | 10 +- .../rds/export/InstanceSnapshotStrategy.java | 64 ++++++++++ .../source/rds/export/SnapshotManager.java | 53 ++------ .../source/rds/export/SnapshotStrategy.java | 30 +++++ .../export/ClusterSnapshotStrategyTest.java | 118 ++++++++++++++++++ .../rds/export/ExportSchedulerTest.java | 94 +++++++------- .../export/InstanceSnapshotStrategyTest.java | 118 ++++++++++++++++++ .../rds/export/SnapshotManagerTest.java | 83 ++---------- 11 files changed, 475 insertions(+), 173 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategy.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategy.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index 83e168f41f..c7f7274763 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -12,8 +12,13 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.export.ClusterSnapshotStrategy; import org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; +import org.opensearch.dataprepper.plugins.source.rds.export.ExportTaskManager; +import org.opensearch.dataprepper.plugins.source.rds.export.InstanceSnapshotStrategy; +import org.opensearch.dataprepper.plugins.source.rds.export.SnapshotManager; +import org.opensearch.dataprepper.plugins.source.rds.export.SnapshotStrategy; import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; import org.opensearch.dataprepper.plugins.source.rds.stream.BinlogClientFactory; import org.opensearch.dataprepper.plugins.source.rds.stream.StreamScheduler; @@ -75,7 +80,12 @@ public void start(Buffer> buffer) { runnableList.add(leaderScheduler); if (sourceConfig.isExportEnabled()) { - exportScheduler = new ExportScheduler(sourceCoordinator, rdsClient, s3Client, pluginMetrics); + final SnapshotStrategy snapshotStrategy = sourceConfig.isCluster() ? + new ClusterSnapshotStrategy(rdsClient) : new InstanceSnapshotStrategy(rdsClient); + final SnapshotManager snapshotManager = new SnapshotManager(snapshotStrategy); + final ExportTaskManager exportTaskManager = new ExportTaskManager(rdsClient); + exportScheduler = new ExportScheduler( + sourceCoordinator, snapshotManager, exportTaskManager, s3Client, pluginMetrics); dataFileScheduler = new DataFileScheduler( sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer); runnableList.add(exportScheduler); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java index 4d90c475ec..c58c9f5974 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java @@ -82,7 +82,7 @@ public String getDbIdentifier() { } public boolean isCluster() { - return isCluster; + return isCluster || isAurora; } public EngineType getEngine() { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategy.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategy.java new file mode 100644 index 0000000000..b7ea008133 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategy.java @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.CreateDbClusterSnapshotRequest; +import software.amazon.awssdk.services.rds.model.CreateDbClusterSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DescribeDbClusterSnapshotsRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbClusterSnapshotsResponse; + +import java.time.Instant; + +/** + * This snapshot strategy works with RDS/Aurora Clusters + */ +public class ClusterSnapshotStrategy implements SnapshotStrategy { + private static final Logger LOG = LoggerFactory.getLogger(ClusterSnapshotStrategy.class); + private final RdsClient rdsClient; + + public ClusterSnapshotStrategy(final RdsClient rdsClient) { + this.rdsClient = rdsClient; + } + + @Override + public SnapshotInfo createSnapshot(String dbIdentifier, String snapshotId) { + CreateDbClusterSnapshotRequest request = CreateDbClusterSnapshotRequest.builder() + .dbClusterIdentifier(dbIdentifier) + .dbClusterSnapshotIdentifier(snapshotId) + .build(); + + try { + CreateDbClusterSnapshotResponse response = rdsClient.createDBClusterSnapshot(request); + String snapshotArn = response.dbClusterSnapshot().dbClusterSnapshotArn(); + String status = response.dbClusterSnapshot().status(); + Instant createTime = response.dbClusterSnapshot().snapshotCreateTime(); + LOG.info("Creating snapshot with id {} for {} and the current status is {}", snapshotId, dbIdentifier, status); + + return new SnapshotInfo(snapshotId, snapshotArn, createTime, status); + } catch (Exception e) { + LOG.error("Failed to create snapshot for {}", dbIdentifier, e); + return null; + } + } + + @Override + public SnapshotInfo describeSnapshot(String snapshotId) { + DescribeDbClusterSnapshotsRequest request = DescribeDbClusterSnapshotsRequest.builder() + .dbClusterSnapshotIdentifier(snapshotId) + .build(); + + DescribeDbClusterSnapshotsResponse response = rdsClient.describeDBClusterSnapshots(request); + String snapshotArn = response.dbClusterSnapshots().get(0).dbClusterSnapshotArn(); + String status = response.dbClusterSnapshots().get(0).status(); + Instant createTime = response.dbClusterSnapshots().get(0).snapshotCreateTime(); + + return new SnapshotInfo(snapshotId, snapshotArn, createTime, status); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index abcbd2c1f4..c56a70482c 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -20,7 +20,6 @@ import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotStatus; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import software.amazon.awssdk.services.rds.RdsClient; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; @@ -50,7 +49,6 @@ public class ExportScheduler implements Runnable { private static final Duration DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT = Duration.ofMinutes(60); static final String PARQUET_SUFFIX = ".parquet"; - private final RdsClient rdsClient; private final S3Client s3Client; private final PluginMetrics pluginMetrics; private final EnhancedSourceCoordinator sourceCoordinator; @@ -61,16 +59,16 @@ public class ExportScheduler implements Runnable { private volatile boolean shutdownRequested = false; public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, - final RdsClient rdsClient, + final SnapshotManager snapshotManager, + final ExportTaskManager exportTaskManager, final S3Client s3Client, final PluginMetrics pluginMetrics) { this.pluginMetrics = pluginMetrics; this.sourceCoordinator = sourceCoordinator; - this.rdsClient = rdsClient; this.s3Client = s3Client; this.executor = Executors.newCachedThreadPool(); - this.exportTaskManager = new ExportTaskManager(rdsClient); - this.snapshotManager = new SnapshotManager(rdsClient); + this.snapshotManager = snapshotManager; + this.exportTaskManager = exportTaskManager; } @Override diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategy.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategy.java new file mode 100644 index 0000000000..0bcfbfadf4 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategy.java @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; + +import java.time.Instant; + +/** + * This snapshot strategy works with RDS Instances + */ +public class InstanceSnapshotStrategy implements SnapshotStrategy { + private static final Logger LOG = LoggerFactory.getLogger(InstanceSnapshotStrategy.class); + private final RdsClient rdsClient; + + public InstanceSnapshotStrategy(final RdsClient rdsClient) { + this.rdsClient = rdsClient; + } + + @Override + public SnapshotInfo createSnapshot(String dbIdentifier, String snapshotId) { + CreateDbSnapshotRequest request = CreateDbSnapshotRequest.builder() + .dbInstanceIdentifier(dbIdentifier) + .dbSnapshotIdentifier(snapshotId) + .build(); + + try { + CreateDbSnapshotResponse response = rdsClient.createDBSnapshot(request); + String snapshotArn = response.dbSnapshot().dbSnapshotArn(); + String status = response.dbSnapshot().status(); + Instant createTime = response.dbSnapshot().snapshotCreateTime(); + LOG.info("Creating snapshot with id {} for {} and the current status is {}", snapshotId, dbIdentifier, status); + + return new SnapshotInfo(snapshotId, snapshotArn, createTime, status); + } catch (Exception e) { + LOG.error("Failed to create snapshot for {}", dbIdentifier, e); + return null; + } + } + + @Override + public SnapshotInfo describeSnapshot(String snapshotId) { + DescribeDbSnapshotsRequest request = DescribeDbSnapshotsRequest.builder() + .dbSnapshotIdentifier(snapshotId) + .build(); + + DescribeDbSnapshotsResponse response = rdsClient.describeDBSnapshots(request); + String snapshotArn = response.dbSnapshots().get(0).dbSnapshotArn(); + String status = response.dbSnapshots().get(0).status(); + Instant createTime = response.dbSnapshots().get(0).snapshotCreateTime(); + + return new SnapshotInfo(snapshotId, snapshotArn, createTime, status); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManager.java index 7b8da8717c..5113b2185c 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManager.java @@ -6,61 +6,26 @@ package org.opensearch.dataprepper.plugins.source.rds.export; import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import software.amazon.awssdk.services.rds.RdsClient; -import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; -import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; -import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; -import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; -import java.time.Instant; import java.util.UUID; public class SnapshotManager { - private static final Logger LOG = LoggerFactory.getLogger(SnapshotManager.class); + private final SnapshotStrategy snapshotStrategy; - private final RdsClient rdsClient; - - public SnapshotManager(final RdsClient rdsClient) { - this.rdsClient = rdsClient; + public SnapshotManager(final SnapshotStrategy snapshotStrategy) { + this.snapshotStrategy = snapshotStrategy; } - public SnapshotInfo createSnapshot(String dbInstanceId) { - final String snapshotId = generateSnapshotId(dbInstanceId); - CreateDbSnapshotRequest request = CreateDbSnapshotRequest.builder() - .dbInstanceIdentifier(dbInstanceId) - .dbSnapshotIdentifier(snapshotId) - .build(); - - try { - CreateDbSnapshotResponse response = rdsClient.createDBSnapshot(request); - String snapshotArn = response.dbSnapshot().dbSnapshotArn(); - String status = response.dbSnapshot().status(); - Instant createTime = response.dbSnapshot().snapshotCreateTime(); - LOG.info("Creating snapshot with id {} and status {}", snapshotId, status); - - return new SnapshotInfo(snapshotId, snapshotArn, createTime, status); - } catch (Exception e) { - LOG.error("Failed to create snapshot for {}", dbInstanceId, e); - return null; - } + public SnapshotInfo createSnapshot(String dbIdentifier) { + final String snapshotId = generateSnapshotId(dbIdentifier); + return snapshotStrategy.createSnapshot(dbIdentifier, snapshotId); } public SnapshotInfo checkSnapshotStatus(String snapshotId) { - DescribeDbSnapshotsRequest request = DescribeDbSnapshotsRequest.builder() - .dbSnapshotIdentifier(snapshotId) - .build(); - - DescribeDbSnapshotsResponse response = rdsClient.describeDBSnapshots(request); - String snapshotArn = response.dbSnapshots().get(0).dbSnapshotArn(); - String status = response.dbSnapshots().get(0).status(); - Instant createTime = response.dbSnapshots().get(0).snapshotCreateTime(); - - return new SnapshotInfo(snapshotId, snapshotArn, createTime, status); + return snapshotStrategy.describeSnapshot(snapshotId); } - private String generateSnapshotId(String dbClusterId) { - return dbClusterId + "-snapshot-" + UUID.randomUUID().toString().substring(0, 8); + private String generateSnapshotId(String dbIdentifier) { + return dbIdentifier + "-snapshot-" + UUID.randomUUID().toString().substring(0, 8); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java new file mode 100644 index 0000000000..af2f0507f7 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; + +/** + * Provides a strategy for creating and describing RDS snapshots. + */ +public interface SnapshotStrategy { + /** + * Creates a snapshot of an RDS instance or cluster. + * + * @param dbIdentifier The identifier of the RDS instance or cluster to snapshot. + * @param snapshotId The identifier of the snapshot. + * @return An {@link SnapshotInfo} object describing the snapshot. + */ + SnapshotInfo createSnapshot(String dbIdentifier, String snapshotId); + + /** + * Checks the status of a snapshot. + * + * @param snapshotId The identifier of the snapshot. + * @return An {@link SnapshotInfo} object describing the snapshot. + */ + SnapshotInfo describeSnapshot(String snapshotId); +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java new file mode 100644 index 0000000000..a0965efad2 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java @@ -0,0 +1,118 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.CreateDbClusterSnapshotRequest; +import software.amazon.awssdk.services.rds.model.CreateDbClusterSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DBClusterSnapshot; +import software.amazon.awssdk.services.rds.model.DescribeDbClusterSnapshotsRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbClusterSnapshotsResponse; + +import java.time.Instant; +import java.util.List; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ClusterSnapshotStrategyTest { + + @Mock + private RdsClient rdsClient; + + private ClusterSnapshotStrategy objectUnderTest; + + @BeforeEach + void setUp() { + objectUnderTest = createObjectUnderTest(); + } + + @Test + void test_create_snapshot_with_success() { + final String dbInstanceId = UUID.randomUUID().toString(); + final String snapshotId = UUID.randomUUID().toString(); + CreateDbClusterSnapshotResponse createDbClusterSnapshotResponse = mock(CreateDbClusterSnapshotResponse.class); + DBClusterSnapshot dbClusterSnapshot = mock(DBClusterSnapshot.class); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:" + snapshotId; + final String status = "creating"; + final Instant createTime = Instant.now(); + when(dbClusterSnapshot.dbClusterSnapshotArn()).thenReturn(snapshotArn); + when(dbClusterSnapshot.status()).thenReturn(status); + when(dbClusterSnapshot.snapshotCreateTime()).thenReturn(createTime); + when(createDbClusterSnapshotResponse.dbClusterSnapshot()).thenReturn(dbClusterSnapshot); + when(rdsClient.createDBClusterSnapshot(any(CreateDbClusterSnapshotRequest.class))).thenReturn(createDbClusterSnapshotResponse); + + SnapshotInfo snapshotInfo = objectUnderTest.createSnapshot(dbInstanceId, snapshotId); + + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(CreateDbClusterSnapshotRequest.class); + verify(rdsClient).createDBClusterSnapshot(argumentCaptor.capture()); + + CreateDbClusterSnapshotRequest request = argumentCaptor.getValue(); + assertThat(request.dbClusterIdentifier(), equalTo(dbInstanceId)); + assertThat(request.dbClusterSnapshotIdentifier(), equalTo(snapshotId)); + + assertThat(snapshotInfo, notNullValue()); + assertThat(snapshotInfo.getSnapshotArn(), equalTo(snapshotArn)); + assertThat(snapshotInfo.getStatus(), equalTo(status)); + assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); + } + + @Test + void test_create_snapshot_throws_exception_then_returns_null() { + final String dbInstanceId = UUID.randomUUID().toString(); + final String snapshotId = UUID.randomUUID().toString(); + when(rdsClient.createDBClusterSnapshot(any(CreateDbClusterSnapshotRequest.class))).thenThrow(new RuntimeException("Error")); + + SnapshotInfo snapshotInfo = objectUnderTest.createSnapshot(dbInstanceId, snapshotId); + + assertThat(snapshotInfo, equalTo(null)); + } + + @Test + void test_check_snapshot_status_returns_correct_result() { + DBClusterSnapshot dbClusterSnapshot = mock(DBClusterSnapshot.class); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:" + UUID.randomUUID(); + final String status = "creating"; + final Instant createTime = Instant.now(); + when(dbClusterSnapshot.dbClusterSnapshotArn()).thenReturn(snapshotArn); + when(dbClusterSnapshot.status()).thenReturn(status); + when(dbClusterSnapshot.snapshotCreateTime()).thenReturn(createTime); + DescribeDbClusterSnapshotsResponse describeDbClusterSnapshotsResponse = mock(DescribeDbClusterSnapshotsResponse.class); + when(describeDbClusterSnapshotsResponse.dbClusterSnapshots()).thenReturn(List.of(dbClusterSnapshot)); + + final String snapshotId = UUID.randomUUID().toString(); + DescribeDbClusterSnapshotsRequest describeDbClusterSnapshotsRequest = DescribeDbClusterSnapshotsRequest.builder() + .dbClusterSnapshotIdentifier(snapshotId) + .build(); + when(rdsClient.describeDBClusterSnapshots(describeDbClusterSnapshotsRequest)).thenReturn(describeDbClusterSnapshotsResponse); + + SnapshotInfo snapshotInfo = objectUnderTest.describeSnapshot(snapshotId); + + assertThat(snapshotInfo, notNullValue()); + assertThat(snapshotInfo.getSnapshotId(), equalTo(snapshotId)); + assertThat(snapshotInfo.getSnapshotArn(), equalTo(snapshotArn)); + assertThat(snapshotInfo.getStatus(), equalTo(status)); + assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); + } + + private ClusterSnapshotStrategy createObjectUnderTest() { + return new ClusterSnapshotStrategy(rdsClient); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java index 32aff02a57..a1a520a47a 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java @@ -11,23 +11,15 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Answers; import org.mockito.Mock; -import org.mockito.Mockito; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; -import software.amazon.awssdk.services.rds.RdsClient; -import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; -import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; -import software.amazon.awssdk.services.rds.model.DBSnapshot; -import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; -import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; -import software.amazon.awssdk.services.rds.model.DescribeExportTasksRequest; -import software.amazon.awssdk.services.rds.model.DescribeExportTasksResponse; -import software.amazon.awssdk.services.rds.model.StartExportTaskRequest; -import software.amazon.awssdk.services.rds.model.StartExportTaskResponse; +import org.opensearch.dataprepper.plugins.source.rds.model.ExportStatus; +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotStatus; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; @@ -60,7 +52,10 @@ class ExportSchedulerTest { private EnhancedSourceCoordinator sourceCoordinator; @Mock - private RdsClient rdsClient; + private SnapshotManager snapshotManager; + + @Mock + private ExportTaskManager exportTaskManager; @Mock private S3Client s3Client; @@ -92,23 +87,20 @@ void test_given_no_export_partition_then_not_export() throws InterruptedExceptio Thread.sleep(100); executorService.shutdownNow(); - verifyNoInteractions(rdsClient); + verifyNoInteractions(snapshotManager, exportTaskManager, s3Client); } @Test - void test_given_export_partition_and_task_id_then_complete_export() throws InterruptedException { + void test_given_export_partition_and_export_task_id_then_complete_export() throws InterruptedException { when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.of(exportPartition)); when(exportPartition.getPartitionKey()).thenReturn(UUID.randomUUID().toString()); - when(exportProgressState.getExportTaskId()).thenReturn(UUID.randomUUID().toString()); + final String exportTaskId = UUID.randomUUID().toString(); + when(exportProgressState.getExportTaskId()).thenReturn(exportTaskId); when(exportPartition.getProgressState()).thenReturn(Optional.of(exportProgressState)); - - DescribeExportTasksResponse describeExportTasksResponse = mock(DescribeExportTasksResponse.class, Mockito.RETURNS_DEEP_STUBS); - when(describeExportTasksResponse.exportTasks().get(0).status()).thenReturn("COMPLETE"); - when(rdsClient.describeExportTasks(any(DescribeExportTasksRequest.class))).thenReturn(describeExportTasksResponse); + when(exportTaskManager.checkExportStatus(exportTaskId)).thenReturn(ExportStatus.COMPLETE.name()); // Mock list s3 objects response ListObjectsV2Response listObjectsV2Response = mock(ListObjectsV2Response.class); - String exportTaskId = UUID.randomUUID().toString(); String tableName = UUID.randomUUID().toString(); // objectKey needs to have this structure: "{prefix}/{export task ID}/{database name}/{table name}/{numbered folder}/{file name}" S3Object s3Object = S3Object.builder() @@ -125,49 +117,49 @@ void test_given_export_partition_and_task_id_then_complete_export() throws Inter Thread.sleep(100); executorService.shutdownNow(); + verify(snapshotManager, never()).createSnapshot(any(String.class)); + verify(exportTaskManager, never()).startExportTask( + any(String.class), any(String.class), any(String.class), + any(String.class), any(String.class), any(List.class)); verify(sourceCoordinator).createPartition(any(DataFilePartition.class)); verify(sourceCoordinator).completePartition(exportPartition); - verify(rdsClient, never()).startExportTask(any(StartExportTaskRequest.class)); - verify(rdsClient, never()).createDBSnapshot(any(CreateDbSnapshotRequest.class)); } @Test - void test_given_export_partition_without_task_id_then_start_and_complete_export() throws InterruptedException { + void test_given_export_partition_without_export_task_id_then_start_and_complete_export() throws InterruptedException { when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.of(exportPartition)); when(exportPartition.getPartitionKey()).thenReturn(UUID.randomUUID().toString()); - when(exportProgressState.getExportTaskId()).thenReturn(null).thenReturn(UUID.randomUUID().toString()); + final String exportTaskId = UUID.randomUUID().toString(); + when(exportProgressState.getExportTaskId()) + .thenReturn(null) + .thenReturn(exportTaskId); when(exportPartition.getProgressState()).thenReturn(Optional.of(exportProgressState)); final String dbIdentifier = UUID.randomUUID().toString(); when(exportPartition.getDbIdentifier()).thenReturn(dbIdentifier); // Mock snapshot response - CreateDbSnapshotResponse createDbSnapshotResponse = mock(CreateDbSnapshotResponse.class); - DBSnapshot dbSnapshot = mock(DBSnapshot.class); - final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:snapshot-0b5ae174"; - when(dbSnapshot.dbSnapshotArn()).thenReturn(snapshotArn); - when(dbSnapshot.status()).thenReturn("creating").thenReturn("available"); - when(dbSnapshot.snapshotCreateTime()).thenReturn(Instant.now()); - when(createDbSnapshotResponse.dbSnapshot()).thenReturn(dbSnapshot); - when(rdsClient.createDBSnapshot(any(CreateDbSnapshotRequest.class))).thenReturn(createDbSnapshotResponse); - - DescribeDbSnapshotsResponse describeDbSnapshotsResponse = DescribeDbSnapshotsResponse.builder() - .dbSnapshots(dbSnapshot) - .build(); - when(rdsClient.describeDBSnapshots(any(DescribeDbSnapshotsRequest.class))).thenReturn(describeDbSnapshotsResponse); + final String snapshotId = UUID.randomUUID().toString(); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:" + snapshotId; + final Instant createTime = Instant.now(); + final SnapshotInfo snapshotInfoWhenCreate = new SnapshotInfo( + snapshotId, snapshotArn, createTime, SnapshotStatus.CREATING.getStatusName()); + final SnapshotInfo snapshotInfoWhenComplete = new SnapshotInfo( + snapshotId, snapshotArn, createTime, SnapshotStatus.AVAILABLE.getStatusName()); + when(snapshotManager.createSnapshot(dbIdentifier)).thenReturn(snapshotInfoWhenCreate); + when(snapshotManager.checkSnapshotStatus(snapshotId)).thenReturn(snapshotInfoWhenComplete); // Mock export response - StartExportTaskResponse startExportTaskResponse = mock(StartExportTaskResponse.class); - when(startExportTaskResponse.status()).thenReturn("STARTING"); - when(rdsClient.startExportTask(any(StartExportTaskRequest.class))).thenReturn(startExportTaskResponse); - - DescribeExportTasksResponse describeExportTasksResponse = mock(DescribeExportTasksResponse.class, Mockito.RETURNS_DEEP_STUBS); - when(describeExportTasksResponse.exportTasks().get(0).status()).thenReturn("COMPLETE"); - when(rdsClient.describeExportTasks(any(DescribeExportTasksRequest.class))).thenReturn(describeExportTasksResponse); + when(exportProgressState.getIamRoleArn()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getBucket()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getPrefix()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getKmsKeyId()).thenReturn(UUID.randomUUID().toString()); + when(exportTaskManager.startExportTask(any(String.class), any(String.class), any(String.class), + any(String.class), any(String.class), any(List.class))).thenReturn(exportTaskId); + when(exportTaskManager.checkExportStatus(exportTaskId)).thenReturn(ExportStatus.COMPLETE.name()); // Mock list s3 objects response ListObjectsV2Response listObjectsV2Response = mock(ListObjectsV2Response.class); - String exportTaskId = UUID.randomUUID().toString(); String tableName = UUID.randomUUID().toString(); // objectKey needs to have this structure: "{prefix}/{export task ID}/{database name}/{table name}/{numbered folder}/{file name}" S3Object s3Object = S3Object.builder() @@ -177,6 +169,7 @@ void test_given_export_partition_without_task_id_then_start_and_complete_export( when(listObjectsV2Response.isTruncated()).thenReturn(false); when(s3Client.listObjectsV2(any(ListObjectsV2Request.class))).thenReturn(listObjectsV2Response); + // Act final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(exportScheduler); await().atMost(Duration.ofSeconds(1)) @@ -184,8 +177,11 @@ void test_given_export_partition_without_task_id_then_start_and_complete_export( Thread.sleep(200); executorService.shutdownNow(); - verify(rdsClient).createDBSnapshot(any(CreateDbSnapshotRequest.class)); - verify(rdsClient).startExportTask(any(StartExportTaskRequest.class)); + // Assert + verify(snapshotManager).createSnapshot(dbIdentifier); + verify(exportTaskManager).startExportTask( + any(String.class), any(String.class), any(String.class), + any(String.class), any(String.class), any(List.class)); verify(sourceCoordinator).createPartition(any(DataFilePartition.class)); verify(sourceCoordinator).completePartition(exportPartition); } @@ -197,11 +193,11 @@ void test_shutDown() { final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(exportScheduler); exportScheduler.shutdown(); - verifyNoMoreInteractions(sourceCoordinator, rdsClient); + verifyNoMoreInteractions(sourceCoordinator, snapshotManager, exportTaskManager, s3Client); executorService.shutdownNow(); } private ExportScheduler createObjectUnderTest() { - return new ExportScheduler(sourceCoordinator, rdsClient, s3Client, pluginMetrics); + return new ExportScheduler(sourceCoordinator, snapshotManager, exportTaskManager, s3Client, pluginMetrics); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java new file mode 100644 index 0000000000..c9a87ea3a2 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java @@ -0,0 +1,118 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.export; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; +import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; +import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DBSnapshot; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; + +import java.time.Instant; +import java.util.List; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class InstanceSnapshotStrategyTest { + + @Mock + private RdsClient rdsClient; + + private InstanceSnapshotStrategy objectUnderTest; + + @BeforeEach + void setUp() { + objectUnderTest = createObjectUnderTest(); + } + + @Test + void test_create_snapshot_with_success() { + final String dbInstanceId = UUID.randomUUID().toString(); + final String snapshotId = UUID.randomUUID().toString(); + CreateDbSnapshotResponse createDbSnapshotResponse = mock(CreateDbSnapshotResponse.class); + DBSnapshot dbSnapshot = mock(DBSnapshot.class); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:" + snapshotId; + final String status = "creating"; + final Instant createTime = Instant.now(); + when(dbSnapshot.dbSnapshotArn()).thenReturn(snapshotArn); + when(dbSnapshot.status()).thenReturn(status); + when(dbSnapshot.snapshotCreateTime()).thenReturn(createTime); + when(createDbSnapshotResponse.dbSnapshot()).thenReturn(dbSnapshot); + when(rdsClient.createDBSnapshot(any(CreateDbSnapshotRequest.class))).thenReturn(createDbSnapshotResponse); + + SnapshotInfo snapshotInfo = objectUnderTest.createSnapshot(dbInstanceId, snapshotId); + + ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(CreateDbSnapshotRequest.class); + verify(rdsClient).createDBSnapshot(argumentCaptor.capture()); + + CreateDbSnapshotRequest request = argumentCaptor.getValue(); + assertThat(request.dbInstanceIdentifier(), equalTo(dbInstanceId)); + assertThat(request.dbSnapshotIdentifier(), equalTo(snapshotId)); + + assertThat(snapshotInfo, notNullValue()); + assertThat(snapshotInfo.getSnapshotArn(), equalTo(snapshotArn)); + assertThat(snapshotInfo.getStatus(), equalTo(status)); + assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); + } + + @Test + void test_create_snapshot_throws_exception_then_returns_null() { + final String dbInstanceId = UUID.randomUUID().toString(); + final String snapshotId = UUID.randomUUID().toString(); + when(rdsClient.createDBSnapshot(any(CreateDbSnapshotRequest.class))).thenThrow(new RuntimeException("Error")); + + SnapshotInfo snapshotInfo = objectUnderTest.createSnapshot(dbInstanceId, snapshotId); + + assertThat(snapshotInfo, equalTo(null)); + } + + @Test + void test_check_snapshot_status_returns_correct_result() { + DBSnapshot dbSnapshot = mock(DBSnapshot.class); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:" + UUID.randomUUID(); + final String status = "creating"; + final Instant createTime = Instant.now(); + when(dbSnapshot.dbSnapshotArn()).thenReturn(snapshotArn); + when(dbSnapshot.status()).thenReturn(status); + when(dbSnapshot.snapshotCreateTime()).thenReturn(createTime); + DescribeDbSnapshotsResponse describeDbSnapshotsResponse = mock(DescribeDbSnapshotsResponse.class); + when(describeDbSnapshotsResponse.dbSnapshots()).thenReturn(List.of(dbSnapshot)); + + final String snapshotId = UUID.randomUUID().toString(); + DescribeDbSnapshotsRequest describeDbSnapshotsRequest = DescribeDbSnapshotsRequest.builder() + .dbSnapshotIdentifier(snapshotId) + .build(); + when(rdsClient.describeDBSnapshots(describeDbSnapshotsRequest)).thenReturn(describeDbSnapshotsResponse); + + SnapshotInfo snapshotInfo = objectUnderTest.describeSnapshot(snapshotId); + + assertThat(snapshotInfo, notNullValue()); + assertThat(snapshotInfo.getSnapshotId(), equalTo(snapshotId)); + assertThat(snapshotInfo.getSnapshotArn(), equalTo(snapshotArn)); + assertThat(snapshotInfo.getStatus(), equalTo(status)); + assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); + } + + private InstanceSnapshotStrategy createObjectUnderTest() { + return new InstanceSnapshotStrategy(rdsClient); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java index bca52a5fdd..d88531bb65 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java @@ -8,34 +8,20 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; -import software.amazon.awssdk.services.rds.RdsClient; -import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; -import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; -import software.amazon.awssdk.services.rds.model.DBSnapshot; -import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; -import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; -import java.time.Instant; -import java.util.List; import java.util.UUID; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.notNullValue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.mock; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.ArgumentMatchers.startsWith; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) class SnapshotManagerTest { @Mock - private RdsClient rdsClient; + private SnapshotStrategy snapshotStrategy; private SnapshotManager snapshotManager; @@ -45,71 +31,24 @@ void setUp() { } @Test - void test_create_snapshot_with_success() { - String dbInstanceId = UUID.randomUUID().toString(); - CreateDbSnapshotResponse createDbSnapshotResponse = mock(CreateDbSnapshotResponse.class); - DBSnapshot dbSnapshot = mock(DBSnapshot.class); - final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:snapshot-0b5ae174"; - final String status = "creating"; - final Instant createTime = Instant.now(); - when(dbSnapshot.dbSnapshotArn()).thenReturn(snapshotArn); - when(dbSnapshot.status()).thenReturn(status); - when(dbSnapshot.snapshotCreateTime()).thenReturn(createTime); - when(createDbSnapshotResponse.dbSnapshot()).thenReturn(dbSnapshot); - when(rdsClient.createDBSnapshot(any(CreateDbSnapshotRequest.class))).thenReturn(createDbSnapshotResponse); + void test_create_snapshot() { + final String dbIdentifier = UUID.randomUUID().toString(); - SnapshotInfo snapshotInfo = snapshotManager.createSnapshot(dbInstanceId); + snapshotManager.createSnapshot(dbIdentifier); - ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(CreateDbSnapshotRequest.class); - verify(rdsClient).createDBSnapshot(argumentCaptor.capture()); - - CreateDbSnapshotRequest request = argumentCaptor.getValue(); - assertThat(request.dbInstanceIdentifier(), equalTo(dbInstanceId)); - - assertThat(snapshotInfo, notNullValue()); - assertThat(snapshotInfo.getSnapshotArn(), equalTo(snapshotArn)); - assertThat(snapshotInfo.getStatus(), equalTo(status)); - assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); - } - - @Test - void test_create_snapshot_throws_exception_then_returns_null() { - String dbInstanceId = UUID.randomUUID().toString(); - when(rdsClient.createDBSnapshot(any(CreateDbSnapshotRequest.class))).thenThrow(new RuntimeException("Error")); - - SnapshotInfo snapshotInfo = snapshotManager.createSnapshot(dbInstanceId); - - assertThat(snapshotInfo, equalTo(null)); + verify(snapshotStrategy).createSnapshot(eq(dbIdentifier), startsWith(dbIdentifier + "-snapshot-")); } @Test - void test_check_snapshot_status_returns_correct_result() { - DBSnapshot dbSnapshot = mock(DBSnapshot.class); - final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:snapshot-0b5ae174"; - final String status = "creating"; - final Instant createTime = Instant.now(); - when(dbSnapshot.dbSnapshotArn()).thenReturn(snapshotArn); - when(dbSnapshot.status()).thenReturn(status); - when(dbSnapshot.snapshotCreateTime()).thenReturn(createTime); - DescribeDbSnapshotsResponse describeDbSnapshotsResponse = mock(DescribeDbSnapshotsResponse.class); - when(describeDbSnapshotsResponse.dbSnapshots()).thenReturn(List.of(dbSnapshot)); - + void test_check_snapshot_status() { final String snapshotId = UUID.randomUUID().toString(); - DescribeDbSnapshotsRequest describeDbSnapshotsRequest = DescribeDbSnapshotsRequest.builder() - .dbSnapshotIdentifier(snapshotId) - .build(); - when(rdsClient.describeDBSnapshots(describeDbSnapshotsRequest)).thenReturn(describeDbSnapshotsResponse); - SnapshotInfo snapshotInfo = snapshotManager.checkSnapshotStatus(snapshotId); + snapshotManager.checkSnapshotStatus(snapshotId); - assertThat(snapshotInfo, notNullValue()); - assertThat(snapshotInfo.getSnapshotId(), equalTo(snapshotId)); - assertThat(snapshotInfo.getSnapshotArn(), equalTo(snapshotArn)); - assertThat(snapshotInfo.getStatus(), equalTo(status)); - assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); + verify(snapshotStrategy).describeSnapshot(snapshotId); } private SnapshotManager createObjectUnderTest() { - return new SnapshotManager(rdsClient); + return new SnapshotManager(snapshotStrategy); } } \ No newline at end of file From 4501a1e61bbaa9d4b71b0864ece1c4b0a0cd084d Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 25 Jul 2024 01:01:21 -0500 Subject: [PATCH 087/159] Fixes a bug with HistogramAggregateAction where the startTime may be incorrect. This was discovered by a flaky test. (#4749) The HistogramAggregateAction was incorrectly using the current time as the start time for the aggregation when creating the group. The very first event's time was overridden by the current system time. If this event had the earliest time, then the overall histogram would never get the correct start time. This is fixed by removing an errant line. I also added a unit test to directly test this failure scenario. Signed-off-by: David Venable --- .../actions/HistogramAggregateAction.java | 19 +- .../HistogramAggregateActionTests.java | 171 +++++++++++++----- 2 files changed, 134 insertions(+), 56 deletions(-) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java index bdb9a3fad6..0bded67d75 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateAction.java @@ -63,7 +63,6 @@ public class HistogramAggregateAction implements AggregateAction { private double maxValue; private final String metricName; - private long startTimeNanos; private double[] buckets; @DataPrepperPluginConstructor @@ -147,8 +146,8 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct Instant eventTime = Instant.now(); Instant eventStartTime = eventTime; Instant eventEndTime = eventTime; - Object startTime = event.get(startTimeKey, Object.class); - Object endTime = event.get(endTimeKey, Object.class); + final Object startTime = event.get(startTimeKey, Object.class); + final Object endTime = event.get(endTimeKey, Object.class); if (startTime != null) { eventStartTime = AggregateProcessor.convertObjectToInstant(startTime); } @@ -161,7 +160,6 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct Long[] bucketCountsList = new Long[buckets.length-1]; Arrays.fill(bucketCountsList, (long)0); bucketCountsList[idx]++; - groupState.put(startTimeKey, eventTime); groupState.putAll(aggregateActionInput.getIdentificationKeys()); groupState.put(sumKey, doubleValue); groupState.put(countKey, 1); @@ -195,12 +193,14 @@ public AggregateActionResponse handleEvent(final Event event, final AggregateAct maxValue = doubleValue; } } - Instant groupStartTime = (Instant)groupState.get(startTimeKey); - Instant groupEndTime = (Instant)groupState.get(endTimeKey); - if (eventStartTime.isBefore(groupStartTime)) + final Instant groupStartTime = (Instant)groupState.get(startTimeKey); + final Instant groupEndTime = (Instant)groupState.get(endTimeKey); + if (eventStartTime.isBefore(groupStartTime)) { groupState.put(startTimeKey, eventStartTime); - if (eventEndTime.isAfter(groupEndTime)) + } + if (eventEndTime.isAfter(groupEndTime)) { groupState.put(endTimeKey, eventEndTime); + } } return AggregateActionResponse.nullEventResponse(); } @@ -233,8 +233,7 @@ public AggregateActionOutput concludeGroup(final AggregateActionInput aggregateA explicitBoundsList.add(this.buckets[i]); } List buckets = createBuckets(bucketCounts, explicitBoundsList); - Integer countValue = (Integer)groupState.get(countKey); - Map attr = new HashMap(); + Map attr = new HashMap<>(); aggregateActionInput.getIdentificationKeys().forEach((k, v) -> { attr.put((String)k, v); }); diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java index 155acee918..27e8f8d801 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionTests.java @@ -5,56 +5,62 @@ package org.opensearch.dataprepper.plugins.processor.aggregate.actions; -import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.metric.JacksonMetric; import org.opensearch.dataprepper.model.metric.Exemplar; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.model.metric.JacksonMetric; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateAction; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionInput; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionOutput; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionResponse; import org.opensearch.dataprepper.plugins.processor.aggregate.AggregateActionTestUtils; -import org.mockito.junit.jupiter.MockitoExtension; -import org.apache.commons.lang3.RandomStringUtils; - -import java.util.Arrays; +import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.Map; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.UUID; -import java.time.Instant; import java.util.concurrent.ThreadLocalRandom; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.mock; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.hasKey; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; @ExtendWith(MockitoExtension.class) public class HistogramAggregateActionTests { - AggregateActionInput aggregateActionInput; - private AggregateAction histogramAggregateAction; + private HistogramAggregateActionConfig histogramAggregateActionConfig; - private AggregateAction createObjectUnderTest(HistogramAggregateActionConfig config) { - return new HistogramAggregateAction(config); + @BeforeEach + void setUp() { + histogramAggregateActionConfig = new HistogramAggregateActionConfig(); + } + + private AggregateAction createObjectUnderTest() { + return new HistogramAggregateAction(histogramAggregateActionConfig); } @ParameterizedTest @ValueSource(ints = {10, 20, 50, 100}) - void testHistogramAggregate(int testCount) throws NoSuchFieldException, IllegalAccessException { - HistogramAggregateActionConfig histogramAggregateActionConfig = new HistogramAggregateActionConfig(); + void testHistogramAggregate(final int testCount) throws NoSuchFieldException, IllegalAccessException { setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "outputFormat", OutputFormat.RAW.toString()); final String testKeyPrefix = RandomStringUtils.randomAlphabetic(5)+"_"; setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "generatedKeyPrefix", testKeyPrefix); @@ -73,7 +79,7 @@ void testHistogramAggregate(int testCount) throws NoSuchFieldException, IllegalA setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "buckets", buckets); final String testKey = RandomStringUtils.randomAlphabetic(10); setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "key", testKey); - histogramAggregateAction = createObjectUnderTest(histogramAggregateActionConfig); + histogramAggregateAction = createObjectUnderTest(); final String dataKey = RandomStringUtils.randomAlphabetic(10); final AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(Collections.emptyMap()); Long[] expectedBucketCounts = new Long[buckets.size()+1]; @@ -135,8 +141,7 @@ void testHistogramAggregate(int testCount) throws NoSuchFieldException, IllegalA @ParameterizedTest @ValueSource(ints = {10, 20, 50, 100}) - void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException, IllegalAccessException { - HistogramAggregateActionConfig histogramAggregateActionConfig = new HistogramAggregateActionConfig(); + void testHistogramAggregateOTelFormat(final int testCount) throws NoSuchFieldException, IllegalAccessException { final String testKeyPrefix = RandomStringUtils.randomAlphabetic(5)+"_"; setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "generatedKeyPrefix", testKeyPrefix); final String testUnits = "ms"; @@ -155,7 +160,7 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "buckets", buckets); final String testKey = RandomStringUtils.randomAlphabetic(10); setField(HistogramAggregateActionConfig.class, histogramAggregateActionConfig, "key", testKey); - histogramAggregateAction = createObjectUnderTest(histogramAggregateActionConfig); + histogramAggregateAction = createObjectUnderTest(); final String dataKey = RandomStringUtils.randomAlphabetic(10); final String dataValue = RandomStringUtils.randomAlphabetic(15); final AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(Map.of(dataKey, dataValue)); @@ -194,8 +199,6 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException final AggregateActionOutput actionOutput = histogramAggregateAction.concludeGroup(aggregateActionInput); final List result = actionOutput.getEvents(); assertThat(result.size(), equalTo(1)); - final String expectedCountKey = histogramAggregateActionConfig.getCountKey(); - final String expectedStartTimeKey = histogramAggregateActionConfig.getStartTimeKey(); Map expectedEventMap = new HashMap<>(Collections.singletonMap("count", (long)testCount)); expectedEventMap.put("unit", testUnits); expectedEventMap.put("name", HistogramAggregateActionConfig.HISTOGRAM_METRIC_NAME); @@ -243,20 +246,20 @@ void testHistogramAggregateOTelFormat(int testCount) throws NoSuchFieldException @ParameterizedTest @ValueSource(ints = {10, 20, 50, 100}) - void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCount) throws NoSuchFieldException, IllegalAccessException { - HistogramAggregateActionConfig mockConfig = mock(HistogramAggregateActionConfig.class); + void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(final int testCount) { + histogramAggregateActionConfig = mock(HistogramAggregateActionConfig.class); String startTimeKey = UUID.randomUUID().toString(); String endTimeKey = UUID.randomUUID().toString(); final String testKeyPrefix = RandomStringUtils.randomAlphabetic(5)+"_"; - when(mockConfig.getStartTimeKey()).thenReturn(startTimeKey); - when(mockConfig.getEndTimeKey()).thenReturn(endTimeKey); + when(histogramAggregateActionConfig.getStartTimeKey()).thenReturn(startTimeKey); + when(histogramAggregateActionConfig.getEndTimeKey()).thenReturn(endTimeKey); final String testName = UUID.randomUUID().toString(); - when(mockConfig.getMetricName()).thenReturn(testName); - when(mockConfig.getOutputFormat()).thenReturn(OutputFormat.OTEL_METRICS.toString()); + when(histogramAggregateActionConfig.getMetricName()).thenReturn(testName); + when(histogramAggregateActionConfig.getOutputFormat()).thenReturn(OutputFormat.OTEL_METRICS.toString()); String keyPrefix = UUID.randomUUID().toString(); final String testUnits = "ms"; - when(mockConfig.getUnits()).thenReturn(testUnits); - when(mockConfig.getRecordMinMax()).thenReturn(true); + when(histogramAggregateActionConfig.getUnits()).thenReturn(testUnits); + when(histogramAggregateActionConfig.getRecordMinMax()).thenReturn(true); final double TEST_VALUE_RANGE_MIN = 0.0; final double TEST_VALUE_RANGE_MAX = 6.0; final double TEST_VALUE_RANGE_STEP = 2.0; @@ -267,18 +270,18 @@ void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCoun buckets.add(bucket1); buckets.add(bucket2); buckets.add(bucket3); - when(mockConfig.getBuckets()).thenReturn(buckets); + when(histogramAggregateActionConfig.getBuckets()).thenReturn(buckets); final String testKey = RandomStringUtils.randomAlphabetic(10); - when(mockConfig.getKey()).thenReturn(testKey); + when(histogramAggregateActionConfig.getKey()).thenReturn(testKey); final String testPrefix = RandomStringUtils.randomAlphabetic(7); - when(mockConfig.getSumKey()).thenReturn(testPrefix+"sum"); - when(mockConfig.getMinKey()).thenReturn(testPrefix+"min"); - when(mockConfig.getMaxKey()).thenReturn(testPrefix+"max"); - when(mockConfig.getCountKey()).thenReturn(testPrefix+"count"); - when(mockConfig.getBucketsKey()).thenReturn(testPrefix+"buckets"); - when(mockConfig.getBucketCountsKey()).thenReturn(testPrefix+"bucketcounts"); - when(mockConfig.getDurationKey()).thenReturn(testPrefix+"duration"); - histogramAggregateAction = createObjectUnderTest(mockConfig); + when(histogramAggregateActionConfig.getSumKey()).thenReturn(testPrefix+"sum"); + when(histogramAggregateActionConfig.getMinKey()).thenReturn(testPrefix+"min"); + when(histogramAggregateActionConfig.getMaxKey()).thenReturn(testPrefix+"max"); + when(histogramAggregateActionConfig.getCountKey()).thenReturn(testPrefix+"count"); + when(histogramAggregateActionConfig.getBucketsKey()).thenReturn(testPrefix+"buckets"); + when(histogramAggregateActionConfig.getBucketCountsKey()).thenReturn(testPrefix+"bucketcounts"); + when(histogramAggregateActionConfig.getDurationKey()).thenReturn(testPrefix+"duration"); + histogramAggregateAction = createObjectUnderTest(); final String dataKey = RandomStringUtils.randomAlphabetic(10); final String dataValue = RandomStringUtils.randomAlphabetic(15); final AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(Map.of(dataKey, dataValue)); @@ -321,8 +324,6 @@ void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCoun final AggregateActionOutput actionOutput = histogramAggregateAction.concludeGroup(aggregateActionInput); final List result = actionOutput.getEvents(); assertThat(result.size(), equalTo(1)); - final String expectedCountKey = mockConfig.getCountKey(); - final String expectedStartTimeKey = mockConfig.getStartTimeKey(); Map expectedEventMap = new HashMap<>(Collections.singletonMap("count", (long)testCount)); expectedEventMap.put("unit", testUnits); expectedEventMap.put("name", testName); @@ -343,7 +344,7 @@ void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCoun List exemplars = (List )result.get(0).toMap().get("exemplars"); assertThat(exemplars.size(), equalTo(2)); assertThat(((Map)result.get(0).toMap().get("attributes")), hasEntry(dataKey, dataValue)); - final String expectedDurationKey = mockConfig.getDurationKey(); + final String expectedDurationKey = histogramAggregateActionConfig.getDurationKey(); assertThat(((Map)result.get(0).toMap().get("attributes")), hasKey(expectedDurationKey)); JacksonMetric metric = (JacksonMetric) result.get(0); assertThat(metric.toJsonString().indexOf("attributes"), not(-1)); @@ -370,4 +371,82 @@ void testHistogramAggregateOTelFormatWithStartAndEndTimesInTheEvent(int testCoun assertThat(result.get(0).get("startTime", String.class), equalTo(testTime.toString())); assertThat(result.get(0).get("time", String.class), equalTo(testTime.plusSeconds(100).toString())); } + + @Test + void testHistogramAggregateOTelFormat_with_startTime_before_currentTime_and_all_other_times_after_that_has_the_correct_startTime() { + histogramAggregateActionConfig = mock(HistogramAggregateActionConfig.class); + String startTimeKey = UUID.randomUUID().toString(); + String endTimeKey = UUID.randomUUID().toString(); + when(histogramAggregateActionConfig.getStartTimeKey()).thenReturn(startTimeKey); + when(histogramAggregateActionConfig.getEndTimeKey()).thenReturn(endTimeKey); + final String testName = UUID.randomUUID().toString(); + when(histogramAggregateActionConfig.getMetricName()).thenReturn(testName); + when(histogramAggregateActionConfig.getOutputFormat()).thenReturn(OutputFormat.OTEL_METRICS.toString()); + final String testUnits = "ms"; + when(histogramAggregateActionConfig.getUnits()).thenReturn(testUnits); + when(histogramAggregateActionConfig.getRecordMinMax()).thenReturn(true); + final double TEST_VALUE_RANGE_MIN = 0.0; + final double TEST_VALUE_RANGE_MAX = 6.0; + final double TEST_VALUE_RANGE_STEP = 2.0; + final double bucket1 = TEST_VALUE_RANGE_MIN; + final double bucket2 = bucket1 + TEST_VALUE_RANGE_STEP; + final double bucket3 = bucket2 + TEST_VALUE_RANGE_STEP; + List buckets = new ArrayList(); + buckets.add(bucket1); + buckets.add(bucket2); + buckets.add(bucket3); + when(histogramAggregateActionConfig.getBuckets()).thenReturn(buckets); + final String testKey = RandomStringUtils.randomAlphabetic(10); + when(histogramAggregateActionConfig.getKey()).thenReturn(testKey); + final String testPrefix = RandomStringUtils.randomAlphabetic(7); + when(histogramAggregateActionConfig.getSumKey()).thenReturn(testPrefix+"sum"); + when(histogramAggregateActionConfig.getMinKey()).thenReturn(testPrefix+"min"); + when(histogramAggregateActionConfig.getMaxKey()).thenReturn(testPrefix+"max"); + when(histogramAggregateActionConfig.getCountKey()).thenReturn(testPrefix+"count"); + when(histogramAggregateActionConfig.getBucketsKey()).thenReturn(testPrefix+"buckets"); + when(histogramAggregateActionConfig.getBucketCountsKey()).thenReturn(testPrefix+"bucketcounts"); + when(histogramAggregateActionConfig.getDurationKey()).thenReturn(testPrefix+"duration"); + histogramAggregateAction = createObjectUnderTest(); + final String dataKey = RandomStringUtils.randomAlphabetic(10); + final String dataValue = RandomStringUtils.randomAlphabetic(15); + final AggregateActionInput aggregateActionInput = new AggregateActionTestUtils.TestAggregateActionInput(Map.of(dataKey, dataValue)); + + final Instant expectedFirstStartTime = Instant.now().truncatedTo(ChronoUnit.SECONDS).minus(2, ChronoUnit.SECONDS); + final double value1 = ThreadLocalRandom.current().nextDouble(TEST_VALUE_RANGE_MIN-TEST_VALUE_RANGE_STEP, TEST_VALUE_RANGE_MAX+TEST_VALUE_RANGE_STEP); + Map eventMapEarliest = Collections.synchronizedMap(Map.of(testKey, value1, startTimeKey, expectedFirstStartTime, endTimeKey, expectedFirstStartTime)); + Event testEventEarliest = JacksonEvent.builder() + .withEventType("event") + .withData(eventMapEarliest) + .build(); + final AggregateActionResponse aggregateActionResponse = histogramAggregateAction.handleEvent(testEventEarliest, aggregateActionInput); + assertThat(aggregateActionResponse.getEvent(), equalTo(null)); + + final Instant laterStartTime = Instant.now().truncatedTo(ChronoUnit.SECONDS).plus(5, ChronoUnit.SECONDS); + final double value2 = ThreadLocalRandom.current().nextDouble(TEST_VALUE_RANGE_MIN-TEST_VALUE_RANGE_STEP, TEST_VALUE_RANGE_MAX+TEST_VALUE_RANGE_STEP); + Map eventMapLater = Collections.synchronizedMap(Map.of(testKey, value2, startTimeKey, laterStartTime, endTimeKey, laterStartTime)); + Event testEventLater = JacksonEvent.builder() + .withEventType("event") + .withData(eventMapLater) + .build(); + + final AggregateActionResponse aggregateActionResponseLater = histogramAggregateAction.handleEvent(testEventLater, aggregateActionInput); + assertThat(aggregateActionResponseLater.getEvent(), equalTo(null)); + + final AggregateActionOutput actionOutput = histogramAggregateAction.concludeGroup(aggregateActionInput); + final List result = actionOutput.getEvents(); + assertThat(result.size(), equalTo(1)); + + assertThat(result.get(0).toMap(), hasKey("startTime")); + assertThat(result.get(0).toMap(), hasKey("time")); + + final String actualStartTime = result.get(0).get("startTime", String.class); + assertThat(actualStartTime, notNullValue()); + final Instant startTimeInstant = Instant.parse(actualStartTime).truncatedTo(ChronoUnit.MILLIS); + assertThat(startTimeInstant, equalTo(expectedFirstStartTime)); + + final String actualTime = result.get(0).get("time", String.class); + assertThat(actualTime, notNullValue()); + final Instant actualTimeInstant = Instant.parse(actualTime).truncatedTo(ChronoUnit.MILLIS); + assertThat(actualTimeInstant, equalTo(laterStartTime)); + } } From 7bec86540dbaefc59925988c9dfc41d94207a070 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Fri, 26 Jul 2024 13:31:54 -0500 Subject: [PATCH 088/159] A few improvements to rds source (#4765) * Add error logging to event handlers Signed-off-by: Hai Yan * Add tls config and enable tls by default Signed-off-by: Hai Yan * Add original event name to metadata Signed-off-by: Hai Yan * Update metadata for export and stream events Signed-off-by: Hai Yan * Add some fixes Signed-off-by: Hai Yan * Remove config alias ssl Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- .../plugins/source/rds/RdsService.java | 6 +++ .../plugins/source/rds/RdsSourceConfig.java | 12 +++-- .../source/rds/configuration/TlsConfig.java | 20 ++++++++ .../rds/converter/ExportRecordConverter.java | 24 ++++++++- .../rds/converter/MetadataKeyAttributes.java | 4 +- .../rds/converter/StreamRecordConverter.java | 27 +++++++--- .../state/DataFileProgressState.java | 22 ++++++++ .../state/ExportProgressState.java | 12 ++--- .../source/rds/export/DataFileLoader.java | 20 +++++++- .../source/rds/export/ExportScheduler.java | 28 ++++++---- .../source/rds/leader/LeaderScheduler.java | 7 +-- .../rds/stream/BinlogClientFactory.java | 32 +++++++++--- .../rds/stream/BinlogEventListener.java | 51 ++++++++++++++++--- .../source/rds/stream/StreamWorker.java | 2 +- .../plugins/source/rds/RdsServiceTest.java | 49 +++++++++++++++++- .../converter/ExportRecordConverterTest.java | 34 ++++++++++++- .../converter/StreamRecordConverterTest.java | 22 ++++++-- 17 files changed, 312 insertions(+), 60 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/TlsConfig.java diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index c7f7274763..982751a3db 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.source.rds; import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.network.SSLMode; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; @@ -94,6 +95,11 @@ public void start(Buffer> buffer) { if (sourceConfig.isStreamEnabled()) { BinaryLogClient binaryLogClient = new BinlogClientFactory(sourceConfig, rdsClient).create(); + if (sourceConfig.getTlsConfig() == null || !sourceConfig.getTlsConfig().isInsecure()) { + binaryLogClient.setSSLMode(SSLMode.REQUIRED); + } else { + binaryLogClient.setSSLMode(SSLMode.DISABLED); + } streamScheduler = new StreamScheduler(sourceCoordinator, sourceConfig, binaryLogClient, buffer, pluginMetrics); runnableList.add(streamScheduler); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java index c58c9f5974..548dc4a2fb 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.configuration.ExportConfig; import org.opensearch.dataprepper.plugins.source.rds.configuration.StreamConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.TlsConfig; import java.util.List; @@ -55,9 +56,6 @@ public class RdsSourceConfig { @JsonProperty("acknowledgments") private boolean acknowledgments = false; - /** - * S3 bucket for holding both export and stream data - */ @JsonProperty("s3_bucket") private String s3Bucket; @@ -77,6 +75,9 @@ public class RdsSourceConfig { @JsonProperty("authentication") private AuthenticationConfig authenticationConfig; + @JsonProperty("tls") + private TlsConfig tlsConfig; + public String getDbIdentifier() { return dbIdentifier; } @@ -133,6 +134,10 @@ public boolean isStreamEnabled() { return streamConfig != null; } + public TlsConfig getTlsConfig() { + return tlsConfig; + } + public AuthenticationConfig getAuthenticationConfig() { return this.authenticationConfig; } @@ -151,6 +156,5 @@ public String getUsername() { public String getPassword() { return password; } - } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/TlsConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/TlsConfig.java new file mode 100644 index 0000000000..8a8d1c1929 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/TlsConfig.java @@ -0,0 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class TlsConfig { + + @JsonProperty("insecure") + private boolean insecure = false; + + // TODO: add options for server identity verification + + public boolean isInsecure() { + return insecure; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java index 11932cd512..428f121788 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverter.java @@ -7,11 +7,19 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; import org.opensearch.dataprepper.model.record.Record; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.util.List; +import java.util.stream.Collectors; + +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TIMESTAMP_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; @@ -21,16 +29,28 @@ public class ExportRecordConverter { static final String EXPORT_EVENT_TYPE = "EXPORT"; - public Event convert(Record record, String tableName, String primaryKeyName) { + public Event convert(final Record record, + final String databaseName, + final String tableName, + final List primaryKeys, + final long eventCreateTimeEpochMillis, + final long eventVersionNumber) { Event event = record.getData(); EventMetadata eventMetadata = event.getMetadata(); + eventMetadata.setAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE, databaseName); eventMetadata.setAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE, tableName); + eventMetadata.setAttribute(BULK_ACTION_METADATA_ATTRIBUTE, OpenSearchBulkActions.INDEX.toString()); eventMetadata.setAttribute(INGESTION_EVENT_TYPE_ATTRIBUTE, EXPORT_EVENT_TYPE); - final Object primaryKeyValue = record.getData().get(primaryKeyName, Object.class); + final String primaryKeyValue = primaryKeys.stream() + .map(key -> event.get(key, String.class)) + .collect(Collectors.joining("|")); eventMetadata.setAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE, primaryKeyValue); + eventMetadata.setAttribute(EVENT_TIMESTAMP_METADATA_ATTRIBUTE, eventCreateTimeEpochMillis); + eventMetadata.setAttribute(EVENT_VERSION_FROM_TIMESTAMP, eventVersionNumber); + return event; } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java index 60e1ba2bd3..0b7d808deb 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/MetadataKeyAttributes.java @@ -12,7 +12,9 @@ public class MetadataKeyAttributes { static final String EVENT_TIMESTAMP_METADATA_ATTRIBUTE = "event_timestamp"; - static final String EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE = "opensearch_action"; + static final String BULK_ACTION_METADATA_ATTRIBUTE = "opensearch_action"; + + static final String CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE = "change_event_type"; static final String EVENT_DATABASE_NAME_METADATA_ATTRIBUTE = "database_name"; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java index 511876d668..51cce6541e 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds.converter; +import com.github.shyiko.mysql.binlog.event.EventType; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventMetadata; import org.opensearch.dataprepper.model.event.JacksonEvent; @@ -19,9 +20,12 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; -import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TIMESTAMP_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; @@ -43,12 +47,15 @@ public StreamRecordConverter(final int partitionCount) { folderNames = s3PartitionCreator.createPartitions(); } - public Event convert(Map rowData, - String databaseName, - String tableName, - OpenSearchBulkActions bulkAction, - List primaryKeys, - String s3Prefix) { + public Event convert(final Map rowData, + final String databaseName, + final String tableName, + final EventType eventType, + final OpenSearchBulkActions bulkAction, + final List primaryKeys, + final String s3Prefix, + final long eventCreateTimeEpochMillis, + final long eventVersionNumber) { final Event event = JacksonEvent.builder() .withEventType("event") .withData(rowData) @@ -58,8 +65,9 @@ public Event convert(Map rowData, eventMetadata.setAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE, databaseName); eventMetadata.setAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE, tableName); - eventMetadata.setAttribute(EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE, bulkAction.toString()); + eventMetadata.setAttribute(BULK_ACTION_METADATA_ATTRIBUTE, bulkAction.toString()); eventMetadata.setAttribute(INGESTION_EVENT_TYPE_ATTRIBUTE, STREAM_EVENT_TYPE); + eventMetadata.setAttribute(CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE, eventType.toString()); final String primaryKeyValue = primaryKeys.stream() .map(rowData::get) @@ -68,6 +76,9 @@ public Event convert(Map rowData, eventMetadata.setAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE, primaryKeyValue); eventMetadata.setAttribute(MetadataKeyAttributes.EVENT_S3_PARTITION_KEY, s3Prefix + S3_PATH_DELIMITER + hashKeyToPartition(primaryKeyValue)); + eventMetadata.setAttribute(EVENT_TIMESTAMP_METADATA_ATTRIBUTE, eventCreateTimeEpochMillis); + eventMetadata.setAttribute(EVENT_VERSION_FROM_TIMESTAMP, eventVersionNumber); + return event; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java index c65c0bbe01..4f94f57d15 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java @@ -15,9 +15,15 @@ public class DataFileProgressState { @JsonProperty("totalRecords") private int totalRecords; + @JsonProperty("sourceDatabase") + private String sourceDatabase; + @JsonProperty("sourceTable") private String sourceTable; + @JsonProperty("snapshotTime") + private long snapshotTime; + public int getTotalRecords() { return totalRecords; } @@ -34,6 +40,14 @@ public void setLoaded(boolean loaded) { this.isLoaded = loaded; } + public String getSourceDatabase() { + return sourceDatabase; + } + + public void setSourceDatabase(String sourceDatabase) { + this.sourceDatabase = sourceDatabase; + } + public String getSourceTable() { return sourceTable; } @@ -41,4 +55,12 @@ public String getSourceTable() { public void setSourceTable(String sourceTable) { this.sourceTable = sourceTable; } + + public long getSnapshotTime() { + return snapshotTime; + } + + public void setSnapshotTime(long snapshotTime) { + this.snapshotTime = snapshotTime; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java index cde2be6dd8..b224f63a97 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java @@ -35,8 +35,8 @@ public class ExportProgressState { @JsonProperty("kmsKeyId") private String kmsKeyId; - @JsonProperty("exportTime") - private String exportTime; + @JsonProperty("snapshotTime") + private long snapshotTime; @JsonProperty("status") private String status; @@ -97,12 +97,12 @@ public void setKmsKeyId(String kmsKeyId) { this.kmsKeyId = kmsKeyId; } - public String getExportTime() { - return exportTime; + public long getSnapshotTime() { + return snapshotTime; } - public void setExportTime(String exportTime) { - this.exportTime = exportTime; + public void setSnapshotTime(long snapshotTime) { + this.snapshotTime = snapshotTime; } public String getStatus() { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java index e76a04e99d..f12d44a75f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -11,15 +11,20 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.DataFileProgressState; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.InputStream; +import java.time.Duration; +import java.util.List; public class DataFileLoader implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(DataFileLoader.class); + static final Duration VERSION_OVERLAP_TIME_FOR_EXPORT = Duration.ofMinutes(5); + private final DataFilePartition dataFilePartition; private final String bucket; private final String objectKey; @@ -58,10 +63,21 @@ public void run() { codec.parse(inputStream, record -> { try { - final String tableName = dataFilePartition.getProgressState().get().getSourceTable(); + DataFileProgressState progressState = dataFilePartition.getProgressState().get(); + // TODO: primary key to be obtained by querying database schema final String primaryKeyName = "id"; - Record transformedRecord = new Record<>(recordConverter.convert(record, tableName, primaryKeyName)); + + final long snapshotTime = progressState.getSnapshotTime(); + final long eventVersionNumber = snapshotTime - VERSION_OVERLAP_TIME_FOR_EXPORT.toMillis(); + Record transformedRecord = new Record<>( + recordConverter.convert( + record, + progressState.getSourceDatabase(), + progressState.getSourceTable(), + List.of(primaryKeyName), + snapshotTime, + eventVersionNumber)); bufferAccumulator.add(transformedRecord); } catch (Exception e) { throw new RuntimeException(e); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index c56a70482c..79ef3d5a61 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -140,12 +140,13 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { final String snapshotId = snapshotInfo.getSnapshotId(); try { - checkSnapshotStatus(snapshotId, DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT); + snapshotInfo = checkSnapshotStatus(snapshotId, DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT); } catch (Exception e) { LOG.warn("Check snapshot status for {} failed", snapshotId, e); sourceCoordinator.giveUpPartition(exportPartition); return null; } + progressState.setSnapshotTime(snapshotInfo.getCreateTime().toEpochMilli()); LOG.info("Creating an export task for db {} from snapshot {}", exportPartition.getDbIdentifier(), snapshotId); String exportTaskId = exportTaskManager.startExportTask( @@ -172,7 +173,7 @@ private void closeExportPartitionWithError(ExportPartition exportPartition) { sourceCoordinator.closePartition(exportPartition, DEFAULT_CLOSE_DURATION, DEFAULT_MAX_CLOSE_COUNT); } - private String checkSnapshotStatus(String snapshotId, Duration timeout) { + private SnapshotInfo checkSnapshotStatus(String snapshotId, Duration timeout) { final Instant endTime = Instant.now().plus(timeout); LOG.debug("Start checking status of snapshot {}", snapshotId); @@ -183,7 +184,7 @@ private String checkSnapshotStatus(String snapshotId, Duration timeout) { // The status should never be "copying" here if (SnapshotStatus.AVAILABLE.getStatusName().equals(status)) { LOG.info("Snapshot {} is available.", snapshotId); - return status; + return snapshotInfo; } LOG.debug("Snapshot {} is still creating. Wait and check later", snapshotId); @@ -254,13 +255,14 @@ private BiConsumer completeExport(ExportPartition exportParti LOG.info("Export for {} completed successfully", exportPartition.getPartitionKey()); ExportProgressState state = exportPartition.getProgressState().get(); - String bucket = state.getBucket(); - String prefix = state.getPrefix(); - String exportTaskId = state.getExportTaskId(); + final String bucket = state.getBucket(); + final String prefix = state.getPrefix(); + final String exportTaskId = state.getExportTaskId(); + final long snapshotTime = state.getSnapshotTime(); // Create data file partitions for processing S3 files List dataFileObjectKeys = getDataFileObjectKeys(bucket, prefix, exportTaskId); - createDataFilePartitions(bucket, exportTaskId, dataFileObjectKeys); + createDataFilePartitions(bucket, exportTaskId, dataFileObjectKeys, snapshotTime); completeExportPartition(exportPartition); } @@ -289,14 +291,18 @@ private List getDataFileObjectKeys(String bucket, String prefix, String return objectKeys; } - private void createDataFilePartitions(String bucket, String exportTaskId, List dataFileObjectKeys) { + private void createDataFilePartitions(String bucket, String exportTaskId, List dataFileObjectKeys, long snapshotTime) { LOG.info("Total of {} data files generated for export {}", dataFileObjectKeys.size(), exportTaskId); AtomicInteger totalFiles = new AtomicInteger(); for (final String objectKey : dataFileObjectKeys) { - DataFileProgressState progressState = new DataFileProgressState(); - ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKey); - String table = exportObjectKey.getTableName(); + final DataFileProgressState progressState = new DataFileProgressState(); + final ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKey); + final String database = exportObjectKey.getDatabaseName(); + final String table = exportObjectKey.getTableName(); + + progressState.setSourceDatabase(database); progressState.setSourceTable(table); + progressState.setSnapshotTime(snapshotTime); DataFilePartition dataFilePartition = new DataFilePartition(exportTaskId, bucket, objectKey, Optional.of(progressState)); sourceCoordinator.createPartition(dataFilePartition); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index f35975f5b2..83b86c096d 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -19,7 +19,6 @@ import org.slf4j.LoggerFactory; import java.time.Duration; -import java.time.Instant; import java.util.Optional; public class LeaderScheduler implements Runnable { @@ -98,9 +97,8 @@ private void init() { sourceCoordinator.createPartition(new GlobalState(sourceConfig.getDbIdentifier(), null)); if (sourceConfig.isExportEnabled()) { - Instant startTime = Instant.now(); LOG.debug("Export is enabled. Creating export partition in the source coordination store."); - createExportPartition(sourceConfig, startTime); + createExportPartition(sourceConfig); } if (sourceConfig.isStreamEnabled()) { @@ -113,14 +111,13 @@ private void init() { leaderProgressState.setInitialized(true); } - private void createExportPartition(RdsSourceConfig sourceConfig, Instant exportTime) { + private void createExportPartition(RdsSourceConfig sourceConfig) { ExportProgressState progressState = new ExportProgressState(); progressState.setIamRoleArn(sourceConfig.getAwsAuthenticationConfig().getAwsStsRoleArn()); progressState.setBucket(sourceConfig.getS3Bucket()); progressState.setPrefix(sourceConfig.getS3Prefix()); progressState.setTables(sourceConfig.getTableNames()); progressState.setKmsKeyId(sourceConfig.getExport().getKmsKeyId()); - progressState.setExportTime(exportTime.toString()); ExportPartition exportPartition = new ExportPartition(sourceConfig.getDbIdentifier(), sourceConfig.isCluster(), progressState); sourceCoordinator.createPartition(exportPartition); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java index 8802d7de46..7c28eea8db 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java @@ -8,7 +8,10 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.DBCluster; import software.amazon.awssdk.services.rds.model.DBInstance; +import software.amazon.awssdk.services.rds.model.DescribeDbClustersRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbClustersResponse; import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; @@ -24,13 +27,21 @@ public BinlogClientFactory(final RdsSourceConfig sourceConfig, final RdsClient r } public BinaryLogClient create() { - DBInstance dbInstance = describeDbInstance(sourceConfig.getDbIdentifier()); + // TODO: refactor SnapshotStrategy to RdsApiStrategy to accommodate more APIs for clusters and instances + String hostName; + int port; + if (sourceConfig.isCluster()) { + DBCluster dbCluster = describeDbCluster(sourceConfig.getDbIdentifier()); + hostName = dbCluster.endpoint(); + port = dbCluster.port(); + } else { + DBInstance dbInstance = describeDbInstance(sourceConfig.getDbIdentifier()); + hostName = dbInstance.endpoint().address(); + port = dbInstance.endpoint().port(); + } return new BinaryLogClient( - dbInstance.endpoint().address(), - dbInstance.endpoint().port(), - // For test - // "127.0.0.1", - // 3306, + hostName, + port, sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword()); } @@ -43,4 +54,13 @@ private DBInstance describeDbInstance(final String dbInstanceIdentifier) { DescribeDbInstancesResponse response = rdsClient.describeDBInstances(request); return response.dbInstances().get(0); } + + private DBCluster describeDbCluster(final String dbClusterIdentifier) { + DescribeDbClustersRequest request = DescribeDbClustersRequest.builder() + .dbClusterIdentifier(dbClusterIdentifier) + .build(); + + DescribeDbClustersResponse response = rdsClient.describeDBClusters(request); + return response.dbClusters().get(0); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java index 7ac0dcbe2b..39beff3f32 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java @@ -29,6 +29,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.function.Consumer; import java.util.stream.Collectors; public class BinlogEventListener implements BinaryLogClient.EventListener { @@ -62,19 +63,19 @@ public void onEvent(com.github.shyiko.mysql.binlog.event.Event event) { switch (eventType) { case TABLE_MAP: - handleTableMapEvent(event); + handleEventAndErrors(event, this::handleTableMapEvent); break; case WRITE_ROWS: case EXT_WRITE_ROWS: - handleInsertEvent(event); + handleEventAndErrors(event, this::handleInsertEvent); break; case UPDATE_ROWS: case EXT_UPDATE_ROWS: - handleUpdateEvent(event); + handleEventAndErrors(event, this::handleUpdateEvent); break; case DELETE_ROWS: case EXT_DELETE_ROWS: - handleDeleteEvent(event); + handleEventAndErrors(event, this::handleDeleteEvent); break; } } @@ -109,6 +110,7 @@ void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); + final long eventTimestampMillis = event.getHeader().getTimestamp(); // Construct data prepper JacksonEvent for (final Object[] rowDataArray : data.getRows()) { @@ -118,7 +120,15 @@ void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { } Event pipelineEvent = recordConverter.convert( - rowDataMap, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), OpenSearchBulkActions.INDEX, primaryKeys, s3Prefix); + rowDataMap, + tableMetadata.getDatabaseName(), + tableMetadata.getTableName(), + event.getHeader().getEventType(), + OpenSearchBulkActions.INDEX, + primaryKeys, + s3Prefix, + eventTimestampMillis, + eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); } @@ -139,6 +149,7 @@ void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); + final long eventTimestampMillis = event.getHeader().getTimestamp(); for (Map.Entry updatedRow : data.getRows()) { // updatedRow contains data before update as key and data after update as value @@ -150,7 +161,15 @@ void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final Event pipelineEvent = recordConverter.convert( - dataMap, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), OpenSearchBulkActions.INDEX, primaryKeys, s3Prefix); + dataMap, + tableMetadata.getDatabaseName(), + tableMetadata.getTableName(), + event.getHeader().getEventType(), + OpenSearchBulkActions.INDEX, + primaryKeys, + s3Prefix, + eventTimestampMillis, + eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); } @@ -172,6 +191,7 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); + final long eventTimestampMillis = event.getHeader().getTimestamp(); for (Object[] rowDataArray : data.getRows()) { final Map rowDataMap = new HashMap<>(); @@ -180,7 +200,15 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { } final Event pipelineEvent = recordConverter.convert( - rowDataMap, tableMetadata.getDatabaseName(), tableMetadata.getTableName(), OpenSearchBulkActions.DELETE, primaryKeys, s3Prefix); + rowDataMap, + tableMetadata.getDatabaseName(), + tableMetadata.getTableName(), + event.getHeader().getEventType(), + OpenSearchBulkActions.DELETE, + primaryKeys, + s3Prefix, + eventTimestampMillis, + eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); } @@ -206,4 +234,13 @@ private void flushBuffer() { LOG.error("Failed to flush buffer", e); } } + + private void handleEventAndErrors(com.github.shyiko.mysql.binlog.event.Event event, + Consumer function) { + try { + function.accept(event); + } catch (Exception e) { + LOG.error("Failed to process change event", e); + } + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java index ce130d0d12..e1e169dea5 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java @@ -65,7 +65,7 @@ public void processStream(final StreamPartition streamPartition) { } try { - LOG.info("Connecting to binary log stream."); + LOG.info("Connect to database to read change events."); binaryLogClient.connect(); } catch (IOException e) { throw new RuntimeException(e); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java index 7a18dd6159..275d5c0c38 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java @@ -8,6 +8,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; import org.mockito.Mock; import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; @@ -17,17 +18,27 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.configuration.TlsConfig; import org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; +import org.opensearch.dataprepper.plugins.source.rds.stream.StreamScheduler; import software.amazon.awssdk.services.rds.RdsClient; +import software.amazon.awssdk.services.rds.model.DBInstance; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; +import software.amazon.awssdk.services.rds.model.Endpoint; +import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -43,7 +54,7 @@ class RdsServiceTest { @Mock private PluginMetrics pluginMetrics; - @Mock + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private RdsSourceConfig sourceConfig; @Mock @@ -67,6 +78,7 @@ void setUp() { void test_normal_service_start_when_export_is_enabled() { RdsService rdsService = createObjectUnderTest(); when(sourceConfig.isExportEnabled()).thenReturn(true); + when(sourceConfig.isStreamEnabled()).thenReturn(false); try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); rdsService.start(buffer); @@ -75,6 +87,41 @@ void test_normal_service_start_when_export_is_enabled() { verify(executor).submit(any(LeaderScheduler.class)); verify(executor).submit(any(ExportScheduler.class)); verify(executor).submit(any(DataFileScheduler.class)); + verify(executor, never()).submit(any(StreamScheduler.class)); + } + + @Test + void test_normal_service_start_when_stream_is_enabled() { + RdsService rdsService = createObjectUnderTest(); + when(sourceConfig.isStreamEnabled()).thenReturn(true); + when(sourceConfig.isExportEnabled()).thenReturn(false); + final String dbIdentifier = UUID.randomUUID().toString(); + when(sourceConfig.getDbIdentifier()).thenReturn(dbIdentifier); + final DescribeDbInstancesResponse describeDbInstancesResponse = mock(DescribeDbInstancesResponse.class, RETURNS_DEEP_STUBS); + final Endpoint hostEndpoint = Endpoint.builder() + .address(UUID.randomUUID().toString()) + .port(3306) + .build(); + when(describeDbInstancesResponse.dbInstances().get(0)).thenReturn( + DBInstance.builder() + .endpoint(hostEndpoint) + .build()); + when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); + final RdsSourceConfig.AuthenticationConfig authConfig = mock(RdsSourceConfig.AuthenticationConfig.class); + when(authConfig.getUsername()).thenReturn(UUID.randomUUID().toString()); + when(authConfig.getPassword()).thenReturn(UUID.randomUUID().toString()); + when(sourceConfig.getAuthenticationConfig()).thenReturn(authConfig); + when(sourceConfig.getTlsConfig()).thenReturn(mock(TlsConfig.class)); + + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { + executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); + rdsService.start(buffer); + } + + verify(executor).submit(any(LeaderScheduler.class)); + verify(executor).submit(any(StreamScheduler.class)); + verify(executor, never()).submit(any(ExportScheduler.class)); + verify(executor, never()).submit(any(DataFileScheduler.class)); } @Test diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java index 79c5597c3b..564fde166b 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/ExportRecordConverterTest.java @@ -5,47 +5,77 @@ package org.opensearch.dataprepper.plugins.source.rds.converter; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.event.TestEventFactory; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; import org.opensearch.dataprepper.model.record.Record; +import java.util.List; import java.util.Map; +import java.util.Random; import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.sameInstance; import static org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter.EXPORT_EVENT_TYPE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TIMESTAMP_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; @ExtendWith(MockitoExtension.class) class ExportRecordConverterTest { + private Random random; + private ExportRecordConverter exportRecordConverter; + + @BeforeEach + void setUp() { + random = new Random(); + exportRecordConverter = createObjectUnderTest(); + } + @Test void test_convert() { + final String databaseName = UUID.randomUUID().toString(); final String tableName = UUID.randomUUID().toString(); final String primaryKeyName = UUID.randomUUID().toString(); final String primaryKeyValue = UUID.randomUUID().toString(); + final long eventCreateTimeEpochMillis = random.nextLong(); + final long eventVersionNumber = random.nextLong(); + final Event testEvent = TestEventFactory.getTestEventFactory().eventBuilder(EventBuilder.class) - .withEventType("EVENT") + .withEventType("event") .withData(Map.of(primaryKeyName, primaryKeyValue)) .build(); Record testRecord = new Record<>(testEvent); ExportRecordConverter exportRecordConverter = new ExportRecordConverter(); - Event actualEvent = exportRecordConverter.convert(testRecord, tableName, primaryKeyName); + Event actualEvent = exportRecordConverter.convert( + testRecord, databaseName, tableName, List.of(primaryKeyName), eventCreateTimeEpochMillis, eventVersionNumber); // Assert + assertThat(actualEvent.getMetadata().getAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE), equalTo(databaseName)); assertThat(actualEvent.getMetadata().getAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE), equalTo(tableName)); + assertThat(actualEvent.getMetadata().getAttribute(BULK_ACTION_METADATA_ATTRIBUTE), equalTo(OpenSearchBulkActions.INDEX.toString())); assertThat(actualEvent.getMetadata().getAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE), equalTo(primaryKeyValue)); assertThat(actualEvent.getMetadata().getAttribute(INGESTION_EVENT_TYPE_ATTRIBUTE), equalTo(EXPORT_EVENT_TYPE)); + assertThat(actualEvent.getMetadata().getAttribute(EVENT_TIMESTAMP_METADATA_ATTRIBUTE), equalTo(eventCreateTimeEpochMillis)); + assertThat(actualEvent.getMetadata().getAttribute(EVENT_VERSION_FROM_TIMESTAMP), equalTo(eventVersionNumber)); assertThat(actualEvent, sameInstance(testRecord.getData())); } + + private ExportRecordConverter createObjectUnderTest() { + return new ExportRecordConverter(); + } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java index 8ccecc3ff1..8b857bcf4f 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverterTest.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds.converter; +import com.github.shyiko.mysql.binlog.event.EventType; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.model.event.Event; @@ -18,10 +19,13 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.startsWith; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.BULK_ACTION_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_DATABASE_NAME_METADATA_ATTRIBUTE; -import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_S3_PARTITION_KEY; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TABLE_NAME_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_TIMESTAMP_METADATA_ATTRIBUTE; +import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP; import static org.opensearch.dataprepper.plugins.source.rds.converter.MetadataKeyAttributes.PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE; import static org.opensearch.dataprepper.plugins.source.rds.converter.StreamRecordConverter.S3_PATH_DELIMITER; @@ -29,29 +33,39 @@ class StreamRecordConverterTest { private StreamRecordConverter streamRecordConverter; + private Random random; @BeforeEach void setUp() { streamRecordConverter = createObjectUnderTest(); + random = new Random(); } @Test void test_convert_returns_expected_event() { - Map rowData = Map.of("key1", "value1", "key2", "value2"); + final Map rowData = Map.of("key1", "value1", "key2", "value2"); final String databaseName = UUID.randomUUID().toString(); final String tableName = UUID.randomUUID().toString(); + final EventType eventType = EventType.EXT_WRITE_ROWS; final OpenSearchBulkActions bulkAction = OpenSearchBulkActions.INDEX; final List primaryKeys = List.of("key1"); final String s3Prefix = UUID.randomUUID().toString(); + final long eventCreateTimeEpochMillis = random.nextLong(); + final long eventVersionNumber = random.nextLong(); - Event event = streamRecordConverter.convert(rowData, databaseName, tableName, bulkAction, primaryKeys, s3Prefix); + Event event = streamRecordConverter.convert( + rowData, databaseName, tableName, eventType, bulkAction, + primaryKeys, s3Prefix, eventCreateTimeEpochMillis, eventVersionNumber); assertThat(event.toMap(), is(rowData)); assertThat(event.getMetadata().getAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE), is(databaseName)); assertThat(event.getMetadata().getAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE), is(tableName)); - assertThat(event.getMetadata().getAttribute(EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE), is(bulkAction.toString())); + assertThat(event.getMetadata().getAttribute(CHANGE_EVENT_TYPE_METADATA_ATTRIBUTE), is(eventType.toString())); + assertThat(event.getMetadata().getAttribute(BULK_ACTION_METADATA_ATTRIBUTE), is(bulkAction.toString())); assertThat(event.getMetadata().getAttribute(PRIMARY_KEY_DOCUMENT_ID_METADATA_ATTRIBUTE), is("value1")); assertThat(event.getMetadata().getAttribute(EVENT_S3_PARTITION_KEY).toString(), startsWith(s3Prefix + S3_PATH_DELIMITER)); + assertThat(event.getMetadata().getAttribute(EVENT_TIMESTAMP_METADATA_ATTRIBUTE), is(eventCreateTimeEpochMillis)); + assertThat(event.getMetadata().getAttribute(EVENT_VERSION_FROM_TIMESTAMP), is(eventVersionNumber)); } private StreamRecordConverter createObjectUnderTest() { From b90380ab57c73aeda0f0bb7d321382dc7d0b2e5d Mon Sep 17 00:00:00 2001 From: Srikanth Govindarajan Date: Fri, 26 Jul 2024 20:56:36 -0700 Subject: [PATCH 089/159] Lambda sink refactor (#4766) * Lambda sink refactor Signed-off-by: Srikanth Govindarajan * Address comments Signed-off-by: Srikanth Govindarajan --------- Signed-off-by: Srikanth Govindarajan --- .../{lambda => aws-lambda}/README.md | 5 +- .../{lambda => aws-lambda}/build.gradle | 10 ++-- .../lambda/sink}/LambdaSinkServiceIT.java | 4 +- .../lambda/common/accumlator/Buffer.java | 14 +++++ .../common/accumlator/BufferFactory.java | 0 .../common/accumlator/InMemoryBuffer.java | 60 +++++++++++++++++-- .../accumlator/InMemoryBufferFactory.java | 0 .../lambda/common/codec/LambdaJsonCodec.java | 0 .../config/AwsAuthenticationOptions.java | 0 .../lambda/common/config/BatchOptions.java | 0 .../common/config/ThresholdOptions.java | 2 +- .../lambda/common/util/ThresholdCheck.java | 0 .../lambda/sink/LambdaClientFactory.java | 0 .../plugins/lambda/sink/LambdaSink.java | 2 +- .../plugins/lambda/sink/LambdaSinkConfig.java | 0 .../lambda/sink/LambdaSinkService.java | 0 .../lambda/sink/dlq/DlqPushHandler.java | 0 .../sink/dlq/LambdaSinkFailedDlqData.java | 0 .../lambda/common/ThresholdCheckTest.java | 0 .../InMemoryBufferFactoryTest.java | 0 .../accumulator/InMemoryBufferTest.java | 0 .../common/codec/LambdaJsonCodecTest.java | 0 .../common/config/ThresholdOptionsTest.java | 2 +- .../lambda/sink/LambdaClientFactoryTest.java | 0 .../lambda/sink/LambdaSinkConfigTest.java | 0 .../lambda/sink/LambdaSinkServiceTest.java | 0 .../plugins/lambda/sink/LambdaSinkTest.java | 2 +- .../lambda/sink/dlq/DlqPushHandlerTest.java | 0 .../org.mockito.plugins.MockMaker | 3 + .../test/resources/simplelogger.properties | 0 settings.gradle | 2 +- 31 files changed, 87 insertions(+), 19 deletions(-) rename data-prepper-plugins/{lambda => aws-lambda}/README.md (75%) rename data-prepper-plugins/{lambda => aws-lambda}/build.gradle (82%) rename data-prepper-plugins/{lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda => aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/sink}/LambdaSinkServiceIT.java (97%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java (67%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java (64%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java (95%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java (97%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java (93%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java (100%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java (98%) rename data-prepper-plugins/{lambda => aws-lambda}/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java (100%) create mode 100644 data-prepper-plugins/aws-lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker rename data-prepper-plugins/{lambda => aws-lambda}/src/test/resources/simplelogger.properties (100%) diff --git a/data-prepper-plugins/lambda/README.md b/data-prepper-plugins/aws-lambda/README.md similarity index 75% rename from data-prepper-plugins/lambda/README.md rename to data-prepper-plugins/aws-lambda/README.md index c0b2c29211..4c49873350 100644 --- a/data-prepper-plugins/lambda/README.md +++ b/data-prepper-plugins/aws-lambda/README.md @@ -1,3 +1,4 @@ + # Lambda Sink This plugin enables you to send data from your Data Prepper pipeline directly to AWS Lambda functions for further processing. @@ -7,7 +8,7 @@ This plugin enables you to send data from your Data Prepper pipeline directly to lambda-pipeline: ... sink: - - lambda: + - aws_lambda: aws: region: "us-east-1" sts_role_arn: "" @@ -31,6 +32,6 @@ The integration tests for this plugin do not run as part of the Data Prepper bui The following command runs the integration tests: ``` -./gradlew :data-prepper-plugins:lambda-sink:integrationTest -Dtests.sink.lambda.region="us-east-1" -Dtests.sink.lambda.functionName="lambda_test_function" -Dtests.sink.lambda.sts_role_arn="arn:aws:iam::123456789012:role/dataprepper-role +./gradlew :data-prepper-plugins:aws-lambda:integrationTest -Dtests.sink.lambda.region="us-east-1" -Dtests.sink.lambda.functionName="lambda_test_function" -Dtests.sink.lambda.sts_role_arn="arn:aws:iam::123456789012:role/dataprepper-role ``` diff --git a/data-prepper-plugins/lambda/build.gradle b/data-prepper-plugins/aws-lambda/build.gradle similarity index 82% rename from data-prepper-plugins/lambda/build.gradle rename to data-prepper-plugins/aws-lambda/build.gradle index 8447c3abdf..be9280e8c8 100644 --- a/data-prepper-plugins/lambda/build.gradle +++ b/data-prepper-plugins/aws-lambda/build.gradle @@ -26,8 +26,10 @@ dependencies { testAnnotationProcessor 'org.projectlombok:lombok:1.18.20' testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' testImplementation project(':data-prepper-test-common') - testImplementation project(':data-prepper-plugins:parse-json-processor') testImplementation testLibs.slf4j.simple + testImplementation 'org.mockito:mockito-core:4.6.1' + testImplementation 'org.junit.jupiter:junit-jupiter-api:5.8.2' + testRuntimeOnly 'org.junit.jupiter:junit-jupiter-engine:5.8.2' } test { @@ -59,9 +61,9 @@ task integrationTest(type: Test) { classpath = sourceSets.integrationTest.runtimeClasspath systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties' - systemProperty 'tests.sink.lambda.region', System.getProperty('tests.sink.lambda.region') - systemProperty 'tests.sink.lambda.functionName', System.getProperty('tests.sink.lambda.functionName') - systemProperty 'tests.sink.lambda.sts_role_arn', System.getProperty('tests.sink.lambda.sts_role_arn') + systemProperty 'tests.lambda.sink.region', System.getProperty('tests.lambda.sink.region') + systemProperty 'tests.lambda.sink.functionName', System.getProperty('tests.lambda.sink.functionName') + systemProperty 'tests.lambda.sink.sts_role_arn', System.getProperty('tests.lambda.sink.sts_role_arn') filter { includeTestsMatching '*IT' diff --git a/data-prepper-plugins/lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java b/data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceIT.java similarity index 97% rename from data-prepper-plugins/lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java rename to data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceIT.java index 76fb4831ce..1a7e169a47 100644 --- a/data-prepper-plugins/lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/lambda/LambdaSinkServiceIT.java +++ b/data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceIT.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.sink.lambda; +package org.opensearch.dataprepper.plugins.lambda.sink; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; @@ -35,8 +35,6 @@ import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.lambda.common.config.BatchOptions; import org.opensearch.dataprepper.plugins.lambda.common.config.ThresholdOptions; -import org.opensearch.dataprepper.plugins.lambda.sink.LambdaSinkConfig; -import org.opensearch.dataprepper.plugins.lambda.sink.LambdaSinkService; import org.opensearch.dataprepper.plugins.lambda.sink.dlq.DlqPushHandler; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.lambda.LambdaClient; diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java similarity index 67% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java index f52a8e5de0..a2c5dde4a9 100644 --- a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/Buffer.java @@ -31,4 +31,18 @@ public interface Buffer { SdkBytes getPayload(); void setEventCount(int eventCount); + + //Metrics + public Duration getFlushLambdaSyncLatencyMetric(); + + public Long getPayloadRequestSyncSize(); + + public Duration getFlushLambdaAsyncLatencyMetric(); + + public Long getPayloadResponseSyncSize(); + + public Long getPayloadRequestAsyncSize(); + + public Long getPayloadResponseAsyncSize(); + } diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/BufferFactory.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java similarity index 64% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java index 5d9d5a5134..095e6f47b2 100644 --- a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java @@ -12,6 +12,7 @@ import software.amazon.awssdk.services.lambda.LambdaClient; import software.amazon.awssdk.services.lambda.model.InvokeRequest; import software.amazon.awssdk.services.lambda.model.InvokeResponse; +import software.amazon.awssdk.services.lambda.model.LambdaException; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -31,7 +32,13 @@ public class InMemoryBuffer implements Buffer { private final String invocationType; private int eventCount; private final StopWatch watch; + private final StopWatch lambdaSyncLatencyWatch; + private final StopWatch lambdaAsyncLatencyWatch; private boolean isCodecStarted; + private long payloadRequestSyncSize; + private long payloadResponseSyncSize; + private long payloadRequestAsyncSize; + private long payloadResponseAsyncSize; public InMemoryBuffer(LambdaClient lambdaClient, String functionName, String invocationType) { @@ -44,6 +51,12 @@ public InMemoryBuffer(LambdaClient lambdaClient, String functionName, String inv watch = new StopWatch(); watch.start(); isCodecStarted = false; + lambdaSyncLatencyWatch = new StopWatch(); + lambdaAsyncLatencyWatch = new StopWatch(); + payloadRequestSyncSize = 0; + payloadResponseSyncSize = 0; + payloadRequestAsyncSize = 0; + payloadResponseAsyncSize =0; } @Override @@ -65,6 +78,7 @@ public Duration getDuration() { public void flushToLambdaAsync() { InvokeResponse resp; SdkBytes payload = getPayload(); + payloadRequestAsyncSize = payload.asByteArray().length; // Setup an InvokeRequest. InvokeRequest request = InvokeRequest.builder() @@ -73,13 +87,17 @@ public void flushToLambdaAsync() { .invocationType(invocationType) .build(); - lambdaClient.invoke(request); + lambdaAsyncLatencyWatch.start(); + resp = lambdaClient.invoke(request); + lambdaAsyncLatencyWatch.stop(); + payloadResponseAsyncSize = resp.payload().asByteArray().length; } @Override public InvokeResponse flushToLambdaSync() { - InvokeResponse resp; + InvokeResponse resp = null; SdkBytes payload = getPayload(); + payloadRequestSyncSize = payload.asByteArray().length; // Setup an InvokeRequest. InvokeRequest request = InvokeRequest.builder() @@ -88,8 +106,16 @@ public InvokeResponse flushToLambdaSync() { .invocationType(invocationType) .build(); - resp = lambdaClient.invoke(request); - return resp; + lambdaSyncLatencyWatch.start(); + try { + resp = lambdaClient.invoke(request); + payloadResponseSyncSize = resp.payload().asByteArray().length; + lambdaSyncLatencyWatch.stop(); + return resp; + } catch (LambdaException e){ + lambdaSyncLatencyWatch.stop(); + throw new RuntimeException(e); + } } private SdkBytes validatePayload(String payload_string) { @@ -121,6 +147,30 @@ public SdkBytes getPayload() { byte[] bytes = byteArrayOutputStream.toByteArray(); SdkBytes sdkBytes = SdkBytes.fromByteArray(bytes); return sdkBytes; - } + } + + public Duration getFlushLambdaSyncLatencyMetric (){ + return Duration.ofMillis(lambdaSyncLatencyWatch.getTime(TimeUnit.MILLISECONDS)); + } + + public Duration getFlushLambdaAsyncLatencyMetric (){ + return Duration.ofMillis(lambdaAsyncLatencyWatch.getTime(TimeUnit.MILLISECONDS)); + } + + public Long getPayloadRequestSyncSize() { + return payloadRequestSyncSize; + } + + public Long getPayloadResponseSyncSize() { + return payloadResponseSyncSize; + } + + public Long getPayloadRequestAsyncSize() { + return payloadRequestAsyncSize; + } + + public Long getPayloadResponseAsyncSize() { + return payloadResponseAsyncSize; + } } diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBufferFactory.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodec.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/AwsAuthenticationOptions.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/BatchOptions.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java similarity index 95% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java index 1f92b90b48..ca8ed6e574 100644 --- a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptions.java @@ -17,7 +17,7 @@ public class ThresholdOptions { - private static final String DEFAULT_BYTE_CAPACITY = "6mb"; + private static final String DEFAULT_BYTE_CAPACITY = "3mb"; @JsonProperty("event_count") @Size(min = 0, max = 10000000, message = "event_count size should be between 0 and 10000000") diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/ThresholdCheck.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactory.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java similarity index 97% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java index 54e484fd13..715ef3295d 100644 --- a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSink.java @@ -26,7 +26,7 @@ import java.util.Collection; -@DataPrepperPlugin(name = "lambda", pluginType = Sink.class, pluginConfigurationType = LambdaSinkConfig.class) +@DataPrepperPlugin(name = "aws_lambda", pluginType = Sink.class, pluginConfigurationType = LambdaSinkConfig.class) public class LambdaSink extends AbstractSink> { private static final Logger LOG = LoggerFactory.getLogger(LambdaSink.class); diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfig.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkService.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandler.java diff --git a/data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java similarity index 100% rename from data-prepper-plugins/lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java rename to data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/LambdaSinkFailedDlqData.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/ThresholdCheckTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferFactoryTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/accumulator/InMemoryBufferTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/codec/LambdaJsonCodecTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java similarity index 93% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java index 5d12aca3da..98437b49fe 100644 --- a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java +++ b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/config/ThresholdOptionsTest.java @@ -11,7 +11,7 @@ import org.opensearch.dataprepper.model.types.ByteCount; class ThresholdOptionsTest { - private static final String DEFAULT_BYTE_CAPACITY = "6mb"; + private static final String DEFAULT_BYTE_CAPACITY = "3mb"; private static final int DEFAULT_EVENT_COUNT = 0; @Test diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaClientFactoryTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkConfigTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkServiceTest.java diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java similarity index 98% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java index 9a042014f0..1842795e7c 100644 --- a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java +++ b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/LambdaSinkTest.java @@ -28,7 +28,7 @@ class LambdaSinkTest { public static final String S3_REGION = "us-east-1"; public static final String CODEC_PLUGIN_NAME = "json"; - public static final String SINK_PLUGIN_NAME = "lambda"; + public static final String SINK_PLUGIN_NAME = "aws_lambda"; public static final String SINK_PIPELINE_NAME = "lambda-sink-pipeline"; private LambdaSinkConfig lambdaSinkConfig; private LambdaSink lambdaSink; diff --git a/data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java similarity index 100% rename from data-prepper-plugins/lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java rename to data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/sink/dlq/DlqPushHandlerTest.java diff --git a/data-prepper-plugins/aws-lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker b/data-prepper-plugins/aws-lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker new file mode 100644 index 0000000000..23c33feb6d --- /dev/null +++ b/data-prepper-plugins/aws-lambda/src/test/resources/mockito-extensions/org.mockito.plugins.MockMaker @@ -0,0 +1,3 @@ +# To enable mocking of final classes with vanilla Mockito +# https://github.com/mockito/mockito/wiki/What%27s-new-in-Mockito-2#mock-the-unmockable-opt-in-mocking-of-final-classesmethods +mock-maker-inline diff --git a/data-prepper-plugins/lambda/src/test/resources/simplelogger.properties b/data-prepper-plugins/aws-lambda/src/test/resources/simplelogger.properties similarity index 100% rename from data-prepper-plugins/lambda/src/test/resources/simplelogger.properties rename to data-prepper-plugins/aws-lambda/src/test/resources/simplelogger.properties diff --git a/settings.gradle b/settings.gradle index 9d84b2ccf0..cb7e888c53 100644 --- a/settings.gradle +++ b/settings.gradle @@ -178,4 +178,4 @@ include 'data-prepper-plugins:mongodb' include 'data-prepper-plugins:rds-source' include 'data-prepper-plugins:http-source-common' include 'data-prepper-plugins:http-common' -include 'data-prepper-plugins:lambda' \ No newline at end of file +include 'data-prepper-plugins:aws-lambda' \ No newline at end of file From 101be0400db89e24e35dfaf16c732d8663620895 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Mon, 29 Jul 2024 11:32:25 -0500 Subject: [PATCH 090/159] Add json property descriptions to dissect, flatten, copy_value and translate processor (#4760) Signed-off-by: Hai Yan --- .../dissect/DissectProcessorConfig.java | 8 ++++++++ .../flatten/FlattenProcessorConfig.java | 16 ++++++++++++++++ .../mutateevent/CopyValueProcessorConfig.java | 11 +++++++++++ .../processor/translate/FileParameterConfig.java | 3 +++ .../translate/MappingsParameterConfig.java | 3 +++ .../processor/translate/S3ObjectConfig.java | 4 ++++ .../translate/TargetsParameterConfig.java | 11 +++++++++++ .../translate/TranslateProcessorConfig.java | 3 +++ 8 files changed, 59 insertions(+) diff --git a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java index bd1ca0d910..bc8ef4705a 100644 --- a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java +++ b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java @@ -1,6 +1,7 @@ package org.opensearch.dataprepper.plugins.processor.dissect; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.plugins.processor.mutateevent.TargetType; @@ -9,10 +10,17 @@ public class DissectProcessorConfig { @NotNull @JsonProperty("map") + @JsonPropertyDescription("Defines the `dissect` patterns for specific keys. For details on how to define fields " + + "in the `dissect` pattern, see [Field notations](#field-notations).") private Map map; @JsonProperty("target_types") + @JsonPropertyDescription("Specifies the data types for extract fields. Valid options are `integer`, " + + "`double`, `string`, and `boolean`. By default, all fields are of the `string` type.") private Map targetTypes; @JsonProperty("dissect_when") + @JsonPropertyDescription("Specifies a condition for performing the `dissect` operation using a " + + "[Data Prepper expression]({{site.url}}{{site.baseurl}}/data-prepper/pipelines/expression-syntax/). " + + "If specified, the `dissect` operation will only run when the expression evaluates to true.") private String dissectWhen; public String getDissectWhen(){ diff --git a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java index c1208f5f40..783f73a9da 100644 --- a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java +++ b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotNull; @@ -19,28 +20,43 @@ public class FlattenProcessorConfig { @NotNull @JsonProperty("source") + @JsonPropertyDescription("The source key on which to perform the operation. If set to an empty string (`\"\"`), " + + "then the processor uses the root of the event as the source.") private String source; @NotNull @JsonProperty("target") + @JsonPropertyDescription("The target key to put into the flattened fields. If set to an empty string (`\"\"`), " + + "then the processor uses the root of the event as the target.") private String target; @JsonProperty("remove_processed_fields") + @JsonPropertyDescription("When `true`, the processor removes all processed fields from the source. Default is `false`.") private boolean removeProcessedFields = false; @JsonProperty("remove_list_indices") + @JsonPropertyDescription("When `true`, the processor converts the fields from the source map into lists and " + + "puts the lists into the target field. Default is `false`.") private boolean removeListIndices = false; @JsonProperty("remove_brackets") + @JsonPropertyDescription("When `true`, the processor also removes brackets around the indices. Can only be " + + "set to `true` when `remove_list_indices` is `true`.") private boolean removeBrackets = false; @JsonProperty("exclude_keys") + @JsonPropertyDescription("The keys from the source field that should be excluded from processing. " + + "Default is an empty list (`[]`).") private List excludeKeys = DEFAULT_EXCLUDE_KEYS; @JsonProperty("flatten_when") + @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that determines whether the `flatten` processor will be run on the " + + "event. Default is `null`, which means that all events will be processed unless otherwise stated.") private String flattenWhen; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of tags to add to the event metadata when the event fails to process.") private List tagsOnFailure; public String getSource() { diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java index b6aee5c7ba..1d24198d84 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; @@ -18,17 +19,23 @@ public static class Entry { @NotEmpty @NotNull @JsonProperty("from_key") + @JsonPropertyDescription("The key of the entry to be copied.") private String fromKey; @NotEmpty @NotNull @JsonProperty("to_key") + @JsonPropertyDescription("The key of the new entry to be added.") private String toKey; @JsonProperty("copy_when") + @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will be run on the event.") private String copyWhen; @JsonProperty("overwrite_if_to_key_exists") + @JsonPropertyDescription("When set to `true`, the existing value is overwritten if `key` already exists in " + + "the event. The default value is `false`.") private boolean overwriteIfToKeyExists = false; public String getFromKey() { @@ -60,15 +67,19 @@ public Entry() { @NotEmpty @NotNull @Valid + @JsonPropertyDescription("A list of entries to be copied in an event.") private List entries; @JsonProperty("from_list") + @JsonPropertyDescription("The source list to copy values from.") private String fromList; @JsonProperty("to_list") + @JsonPropertyDescription("The target list to copy values to.") private String toList; @JsonProperty("overwrite_if_to_list_exists") + @JsonPropertyDescription("When set to `true`, the existing value is overwritten if `key` already exists in the event. The default value is `false`.") private boolean overwriteIfToListExists = false; @AssertTrue(message = "Both from_list and to_list should be specified when copying entries between lists.") diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/FileParameterConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/FileParameterConfig.java index 8d5cc674f1..83dc30952e 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/FileParameterConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/FileParameterConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.translate; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.NotNull; import java.util.List; @@ -13,10 +14,12 @@ public class FileParameterConfig { @JsonProperty("name") + @JsonPropertyDescription("The full path to a local file or key name for an S3 object.") @NotNull private String fileName; @JsonProperty("aws") + @JsonPropertyDescription("The AWS configuration when the file is an S3 object. ") @Valid private S3ObjectConfig awsConfig; diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/MappingsParameterConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/MappingsParameterConfig.java index 7146f62a4c..1bcb14c7a8 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/MappingsParameterConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/MappingsParameterConfig.java @@ -1,6 +1,7 @@ package org.opensearch.dataprepper.plugins.processor.translate; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotNull; @@ -12,10 +13,12 @@ public class MappingsParameterConfig { @JsonProperty("source") + @JsonPropertyDescription("The source field to translate. Can be a string or a list of strings.") @NotNull private Object source; @JsonProperty("targets") + @JsonPropertyDescription("A list of target field configurations, such as the target field key or translation maps.") @Valid private List targetsParameterConfigs = new ArrayList<>(); diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/S3ObjectConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/S3ObjectConfig.java index 56e7aacf27..34dd421994 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/S3ObjectConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/S3ObjectConfig.java @@ -6,19 +6,23 @@ package org.opensearch.dataprepper.plugins.processor.translate; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; public class S3ObjectConfig { @JsonProperty("bucket") + @JsonPropertyDescription("The Amazon S3 bucket name.") @NotNull private String bucket; @JsonProperty("region") + @JsonPropertyDescription("The AWS Region to use for credentials.") @NotNull private String region; @JsonProperty("sts_role_arn") + @JsonPropertyDescription("The AWS Security Token Service (AWS STS) role to assume for requests to Amazon S3.") @NotNull private String stsRoleArn; diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TargetsParameterConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TargetsParameterConfig.java index 53a250ee37..18e796b593 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TargetsParameterConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TargetsParameterConfig.java @@ -1,6 +1,7 @@ package org.opensearch.dataprepper.plugins.processor.translate; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -22,18 +23,28 @@ public class TargetsParameterConfig { private final Map individualMappings = new HashMap<>(); private final Map compiledPatterns = new HashMap<>(); @JsonProperty("target") + @JsonPropertyDescription("The key that specifies the field in the output in which the translated value will be placed.") @NotNull @NotEmpty private String target; @JsonProperty("map") + @JsonPropertyDescription("A list of key-value pairs that define the translations. Each key represents a possible " + + "value in the source field, and the corresponding value represents what it should be translated to. " + + "For examples, see [map option](#map-option). At least one of `map` and `regex` should be configured.") private Map map; @JsonProperty("translate_when") + @JsonPropertyDescription("Uses a [Data Prepper expression]({{site.url}}{{site.baseurl}}/data-prepper/pipelines/expression-syntax/) " + + "to specify a condition for performing the translation. When specified, the expression will only translate when the condition is met.") private String translateWhen; @JsonProperty("regex") + @JsonPropertyDescription("A map of keys that defines the translation map. For more options, see [regex option](#regex-option). " + + "At least one of `map` and `regex` should be configured.") private RegexParameterConfiguration regexParameterConfig; @JsonProperty("default") + @JsonPropertyDescription("The default value to use when no match is found during translation.") private String defaultValue; @JsonProperty("type") + @JsonPropertyDescription("Specifies the data type for the target value.") private TargetType targetType = TargetType.STRING; public TargetsParameterConfig(){ diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java index 70744f683a..de0949a31c 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java @@ -8,6 +8,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; @@ -19,10 +20,12 @@ public class TranslateProcessorConfig { @JsonProperty("file") + @JsonPropertyDescription("Points to the file that contains mapping configurations. For more information, see [file](#file).") @Valid private FileParameterConfig fileParameterConfig; @JsonProperty("mappings") + @JsonPropertyDescription("Defines inline mappings. For more information, see [mappings](#mappings).") @Valid private List mappingsParameterConfigs = new ArrayList<>(); From 8f33225a32bc4ba226f6f83eb88d3ffff5212ad6 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Mon, 29 Jul 2024 11:32:31 -0500 Subject: [PATCH 091/159] Add json property description for list-to-map, map-to-list and user-agent processor (#4759) Signed-off-by: Hai Yan --- .../mutateevent/ListToMapProcessorConfig.java | 21 +++++++++++++++++++ .../mutateevent/MapToListProcessorConfig.java | 16 ++++++++++++++ .../useragent/UserAgentProcessorConfig.java | 6 ++++++ 3 files changed, 43 insertions(+) diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java index 228f07f08e..b63deb727c 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -41,35 +42,55 @@ static FlattenedElement fromOptionValue(final String option) { @NotEmpty @NotNull @JsonProperty("source") + @JsonPropertyDescription("The list of objects with `key` fields to be converted into keys for the generated map.") private String source; @JsonProperty("target") + @JsonPropertyDescription("The target for the generated map. When not specified, the generated map will be " + + "placed in the root node.") private String target = null; @JsonProperty("key") + @JsonPropertyDescription("The key of the fields to be extracted as keys in the generated mappings. Must be " + + "specified if `use_source_key` is `false`.") private String key; @JsonProperty("value_key") + @JsonPropertyDescription("When specified, values given a `value_key` in objects contained in the source list " + + "will be extracted and converted into the value specified by this option based on the generated map. " + + "When not specified, objects contained in the source list retain their original value when mapped.") private String valueKey = null; @JsonProperty("use_source_key") + @JsonPropertyDescription("When `true`, keys in the generated map will use original keys from the source. " + + "Default is `false`.") private boolean useSourceKey = false; @JsonProperty("extract_value") + @JsonPropertyDescription("When `true`, object values from the source list will be extracted and added to " + + "the generated map. When `false`, object values from the source list are added to the generated map " + + "as they appear in the source list. Default is `false`") private boolean extractValue = false; @NotNull @JsonProperty("flatten") + @JsonPropertyDescription("When `true`, values in the generated map output flatten into single items based on " + + "the `flattened_element`. Otherwise, objects mapped to values from the generated map appear as lists.") private boolean flatten = false; @NotNull @JsonProperty("flattened_element") + @JsonPropertyDescription("The element to keep, either `first` or `last`, when `flatten` is set to `true`.") private FlattenedElement flattenedElement = FlattenedElement.FIRST; @JsonProperty("list_to_map_when") + @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will be " + + "run on the event. Default is `null`. All events will be processed unless otherwise stated.") private String listToMapWhen; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of tags to add to the event metadata when the event fails to process.") private List tagsOnFailure; public String getSource() { diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java index 46a2ec79f0..ce317eca49 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -20,32 +21,47 @@ public class MapToListProcessorConfig { @NotNull @JsonProperty("source") + @JsonPropertyDescription("The source map used to perform the mapping operation. When set to an empty " + + "string (`\"\"`), it will use the root of the event as the `source`.") private String source; @NotEmpty @NotNull @JsonProperty("target") + @JsonPropertyDescription("The target for the generated list.") private String target; @JsonProperty("key_name") + @JsonPropertyDescription("The name of the field in which to store the original key. Default is `key`.") private String keyName = DEFAULT_KEY_NAME; @JsonProperty("value_name") + @JsonPropertyDescription("The name of the field in which to store the original value. Default is `value`.") private String valueName = DEFAULT_VALUE_NAME; @JsonProperty("map_to_list_when") + @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will " + + "be run on the event. Default is `null`. All events will be processed unless otherwise stated.") private String mapToListWhen; @JsonProperty("exclude_keys") + @JsonPropertyDescription("The keys in the source map that will be excluded from processing. Default is an " + + "empty list (`[]`).") private List excludeKeys = DEFAULT_EXCLUDE_KEYS; @JsonProperty("remove_processed_fields") + @JsonPropertyDescription("When `true`, the processor will remove the processed fields from the source map. " + + "Default is `false`.") private boolean removeProcessedFields = DEFAULT_REMOVE_PROCESSED_FIELDS; @JsonProperty("convert_field_to_list") + @JsonPropertyDescription("If `true`, the processor will convert the fields from the source map into lists and " + + "place them in fields in the target list. Default is `false`.") private boolean convertFieldToList = false; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of tags to add to the event metadata when the event fails to process.") private List tagsOnFailure; public String getSource() { diff --git a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java index 0dcf46e2a1..b1660b37d0 100644 --- a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java +++ b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.useragent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.model.event.EventKey; @@ -21,21 +22,26 @@ public class UserAgentProcessorConfig { @NotEmpty @NotNull @JsonProperty("source") + @JsonPropertyDescription("The field in the event that will be parsed.") @EventKeyConfiguration(EventKeyFactory.EventAction.GET) private EventKey source; @NotNull @JsonProperty("target") + @JsonPropertyDescription("The field to which the parsed event will write. Default is `user_agent`.") private String target = "user_agent"; @NotNull @JsonProperty("exclude_original") + @JsonPropertyDescription("Determines whether to exclude the original UA string from the parsing result. Defaults to `false`. ") private boolean excludeOriginal = false; @JsonProperty("cache_size") + @JsonPropertyDescription("The cache size of the parser in megabytes. Defaults to `1000`.") private int cacheSize = DEFAULT_CACHE_SIZE; @JsonProperty("tags_on_parse_failure") + @JsonPropertyDescription("The tag to add to an event if the `user_agent` processor fails to parse the UA string.") private List tagsOnParseFailure; public EventKey getSource() { From 1ca5c693a7c996f8521254dcc3fb240a7ee29354 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 29 Jul 2024 18:34:34 -0500 Subject: [PATCH 092/159] Corrects the TRIAGING.md with a video meeting since we currently use Chime. (#4743) Signed-off-by: David Venable --- TRIAGING.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/TRIAGING.md b/TRIAGING.md index a4a25e1932..ba20857061 100644 --- a/TRIAGING.md +++ b/TRIAGING.md @@ -19,7 +19,7 @@ However, should we run out of time before your issue is discussed, you are alway Meetings are hosted regularly Tuesdays at 2:30 PM US Central Time (12:30 PM Pacific Time) and can be joined via the links posted on the [OpenSearch Meetup Group](https://www.meetup.com/opensearch/events/) list of events. The event will be titled `Data Prepper Triage Meeting`. -After joining the Zoom meeting, you can enable your video / voice to join the discussion. +After joining the video meeting, you can enable your video / voice to join the discussion. If you do not have a webcam or microphone available, you can still join in via the text chat. If you have an issue you'd like to bring forth please consider getting a link to the issue so it can be presented to everyone in the meeting. From 9e084a0356388dac4326c54c993343c20eb873e0 Mon Sep 17 00:00:00 2001 From: mishavay-aws <140549901+mishavay-aws@users.noreply.github.com> Date: Mon, 29 Jul 2024 21:41:20 -0400 Subject: [PATCH 093/159] 4602 one way hash (#4750) added capabilities for working with OneWay Hash Signed-off-by: mishavay-aws <140549901+mishavay-aws@users.noreply.github.com> --- .../obfuscate-processor/README.md | 53 +++++- .../obfuscate-processor/build.gradle | 1 + .../obfuscation/ObfuscationProcessor.java | 2 +- .../obfuscation/action/MaskAction.java | 4 +- .../obfuscation/action/ObfuscationAction.java | 6 +- .../obfuscation/action/OneWayHashAction.java | 142 ++++++++++++++ .../action/OneWayHashActionConfig.java | 58 ++++++ .../obfuscation/ObfuscationProcessorTest.java | 3 +- .../obfuscation/action/MaskActionTest.java | 6 +- .../action/ObfuscationActionTest.java | 24 +++ .../action/OneWayHashActionTest.java | 174 ++++++++++++++++++ 11 files changed, 465 insertions(+), 8 deletions(-) create mode 100644 data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashAction.java create mode 100644 data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionConfig.java create mode 100644 data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationActionTest.java create mode 100644 data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionTest.java diff --git a/data-prepper-plugins/obfuscate-processor/README.md b/data-prepper-plugins/obfuscate-processor/README.md index 8e48582cf1..842d08106d 100644 --- a/data-prepper-plugins/obfuscate-processor/README.md +++ b/data-prepper-plugins/obfuscate-processor/README.md @@ -64,7 +64,7 @@ Below are the list of configuration options. * `patterns` - (optional) - A list of Regex patterns. You can define multiple patterns for the same field. Only the parts that matched the Regex patterns to be obfuscated. If not provided, the full field will be obfuscated. * `single_word_only` - (optional) - When set to `true`, a word boundary `\b` is added to the pattern, due to which obfuscation would be applied only to words that are standalone in the input text. By default, it is `false`, meaning obfuscation patterns are applied to all occurrences. -* `action` - (optional) - Obfuscation action, default to `mask`. Currently, `mask` is the only supported action. +* `action` - (optional) - Obfuscation action, `mask` or `hash` to use one way hashing. Default to `mask` ### Configuration - Mask Action @@ -75,6 +75,57 @@ There are some additional configuration options for Mask action. * `mask_character_length` - (optional) - Default to 3. The value must be between 1 and 10. There will be n numbers of obfuscation characters, e.g. '***' +### Configuration - One Way Hash Action + +There are some additional configuration options for One Way Hash action. + +* `format` - (optional) - Default to SHA-512. Format of One Way Hash to use. +* `salt` - (optional) - Default to generate random salt. +* `salt_key` - (optional) - Instructs to generate salt for each record based on a value of a specified field in the message + +```yaml +pipeline: + source: + http: + processor: + - obfuscate: + source: "log" + target: "new_log" + patterns: + - "[A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-]{2,4}" + action: + hash: + salt_key: "/" + salt: "" + - obfuscate: + source: "phone" + action: + hash: + salt: "" + sink: + - stdout: +``` + +Take below input + +```json +{ + "id": 1, + "phone": "(555) 555 5555", + "log": "My name is Bob and my email address is abc@example.com" +} +``` + +When run, the processor will parse the message into the following output: + +```json +{ + "id": 1, + "phone": "***", + "log": "My name is Bob and my email address is ", + "newLog": "My name is Bob and my email address is " +} +``` --- ## FAQ: diff --git a/data-prepper-plugins/obfuscate-processor/build.gradle b/data-prepper-plugins/obfuscate-processor/build.gradle index 22909eecd9..83e21a5889 100644 --- a/data-prepper-plugins/obfuscate-processor/build.gradle +++ b/data-prepper-plugins/obfuscate-processor/build.gradle @@ -4,6 +4,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-core' implementation 'com.fasterxml.jackson.core:jackson-databind' testImplementation project(':data-prepper-test-common') + testImplementation project(':data-prepper-test-event') } test { diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessor.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessor.java index bbb1a1600a..472ffec940 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessor.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessor.java @@ -126,7 +126,7 @@ public Collection> doExecute(Collection> records) { String rawValue = recordEvent.get(source, String.class); // Call obfuscation action - String newValue = this.action.obfuscate(rawValue, patterns); + String newValue = this.action.obfuscate(rawValue, patterns, record); // No changes means it does not match any patterns if (rawValue.equals(newValue)) { diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskAction.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskAction.java index 45fc27fe27..2435156b5f 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskAction.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskAction.java @@ -7,6 +7,8 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; import java.util.List; import java.util.regex.Pattern; @@ -21,7 +23,7 @@ public MaskAction(final MaskActionConfig config) { } @Override - public String obfuscate(String source, List patterns) { + public String obfuscate(String source, List patterns, Record record) { if (patterns == null || patterns.size() == 0) { // This is to replace the whole field. diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationAction.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationAction.java index 1a0376cb89..0e6b71e2aa 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationAction.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationAction.java @@ -8,6 +8,9 @@ import java.util.List; import java.util.regex.Pattern; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; + /** * Interface represents a specific action to be taken for obfuscation. @@ -20,7 +23,8 @@ public interface ObfuscationAction { * * @param source source string * @param patterns a list of patterns to match + * @param record raw record * @return obfuscated string */ - String obfuscate(String source, List patterns); + String obfuscate(String source, List patterns, Record record); } diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashAction.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashAction.java new file mode 100644 index 0000000000..28e47eae08 --- /dev/null +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashAction.java @@ -0,0 +1,142 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.obfuscation.action; + +import java.nio.charset.StandardCharsets; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.security.SecureRandom; +import java.util.Arrays; +import java.util.Base64; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.processor.obfuscation.ObfuscationProcessor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.opensearch.dataprepper.model.event.EventKey; + +@DataPrepperPlugin(name = "hash", pluginType = ObfuscationAction.class, pluginConfigurationType = OneWayHashActionConfig.class) +public class OneWayHashAction implements ObfuscationAction { + + + private final MessageDigest messageDigest; + private final byte[] salt; + private EventKey saltKey; + private static final Logger LOG = LoggerFactory.getLogger(ObfuscationProcessor.class); + + @DataPrepperPluginConstructor + public OneWayHashAction(final OneWayHashActionConfig config) { + + this.saltKey = config.getSaltKey(); + + if (config.getSalt() == null || config.getSalt().isEmpty() ) { + this.salt = generateSalt(); + } else { + this.salt = config.getSalt().getBytes(StandardCharsets.UTF_8); + } + + try { + messageDigest = MessageDigest.getInstance(config.getFormat()); + } catch (NoSuchAlgorithmException noSuchAlgorithmException){ + LOG.error("The hash format provided ({}) is not a known algorithm [{}]", config.getFormat(), noSuchAlgorithmException); + throw new RuntimeException(noSuchAlgorithmException); + } + } + + @Override + public String obfuscate(String source, List patterns, Record record) { + + byte [] saltToApply = this.salt; + + // Resolve salt to compute based on a path provided in the configuration. + // For records where path was not found, the salt value defined in the pipeline configuration will be used, if salt value was not configured, one will be generated. + + if(saltKey != null && saltKey.equals("") == false) { + + final Event recordEvent = record.getData(); + + if (recordEvent.containsKey(saltKey)) { + + saltToApply = computeSaltBasedOnKeyValue(recordEvent.get(saltKey, String.class)); + } else { + LOG.info("Unable to find a key '{}' for using as salt, using default salt pipeline configuration for the record instead", saltKey); + } + } + + if (patterns == null || patterns.size() == 0) { + // no pattern to match, replace the whole string + return oneWayHashString(source,saltToApply); + } + + String replacementString = source; + + for (Pattern pattern : patterns) { + + Matcher matcher = Pattern.compile(pattern.pattern()).matcher(replacementString); + StringBuffer stringBuffer = new StringBuffer(); + + while (matcher.find()) { + + String stringToHash = replacementString.substring(matcher.start(),matcher.end()); + matcher.appendReplacement(stringBuffer, oneWayHashString(stringToHash,saltToApply)); + } + + matcher.appendTail(stringBuffer); + replacementString = stringBuffer.toString(); + } + return replacementString; + + + } + + private String oneWayHashString(String source, byte[] salt) { + + String oneWayHashedSource = ""; + + try { + MessageDigest messageDigestClone = (MessageDigest) messageDigest.clone(); + + messageDigestClone.update(salt); + byte[] bytes = messageDigestClone.digest(source.getBytes(StandardCharsets.UTF_8)); + + oneWayHashedSource = Base64.getEncoder().encodeToString(bytes); + + } catch (CloneNotSupportedException cloneNotSupportedException) { + LOG.error("There was an exception while processing Event [{}]", cloneNotSupportedException); + throw new RuntimeException(cloneNotSupportedException); + } + + return oneWayHashedSource; + } + + private byte [] computeSaltBasedOnKeyValue(String saltValue) { + + byte [] value = saltValue.getBytes(StandardCharsets.UTF_8); + byte [] result = new byte [64]; + + Arrays.fill(result, Byte.MIN_VALUE); + + System.arraycopy(value, 0, result, 0, + (value.length >= result.length) ? result.length : value.length); + + return result; + } + + private byte[] generateSalt() { + + byte [] saltBytes = new byte[64]; + SecureRandom secureRandom = new SecureRandom(); + secureRandom.nextBytes(saltBytes); + return saltBytes; + } +} diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionConfig.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionConfig.java new file mode 100644 index 0000000000..3e3ab622c5 --- /dev/null +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionConfig.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.obfuscation.action; + +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; + +import jakarta.validation.constraints.Pattern; +import jakarta.validation.constraints.Size; + +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyConfiguration; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + + +public class OneWayHashActionConfig { + + @JsonProperty("salt") + @JsonPropertyDescription("Salt value to use when generating hash. If not specified, salt will be randomly generated by the processor.") + @Size(min = 16, message = "Minimum size of salt string is 16.") + @Size(max = 64, message = "Maximum size of salt string is 64") + private String salt; + + @JsonProperty("format") + @Pattern(regexp = "SHA-512", message = "Valid values: SHA-512") + @JsonPropertyDescription("Format of one way hash to generate. Default to SHA-512.") + private String format = "SHA-512"; + + @JsonProperty("salt_key") + @JsonPropertyDescription("A key to compute salt based on a value provided as part of a record." + + "If key or value was not found in the record(s), a salt defined in the pipeline configuration will be used instead.") + @EventKeyConfiguration(EventKeyFactory.EventAction.GET) + private EventKey saltKey; + + public OneWayHashActionConfig(){ + + } + + public String getSalt () { + return salt; + } + + public String getFormat() { + return format; + } + + public EventKey getSaltKey() { + return saltKey; + } + +} + diff --git a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorTest.java b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorTest.java index be35b2cf01..8e1f556110 100644 --- a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorTest.java +++ b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorTest.java @@ -140,11 +140,12 @@ void testBasicProcessor(String message) { void testProcessorWithDifferentAction() { final PluginModel mockModel = mock(PluginModel.class); final ObfuscationAction mockAction = mock(ObfuscationAction.class); + when(mockModel.getPluginName()).thenReturn("mock"); when(mockModel.getPluginSettings()).thenReturn(new HashMap<>()); when(mockConfig.getAction()).thenReturn(mockModel); when(mockConfig.getTarget()).thenReturn(""); - when(mockAction.obfuscate(anyString(), anyList())).thenReturn("abc"); + when(mockAction.obfuscate(anyString(), anyList(),any())).thenReturn("abc"); when(mockFactory.loadPlugin(eq(ObfuscationAction.class), any(PluginSetting.class))) .thenReturn(mockAction); diff --git a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskActionTest.java b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskActionTest.java index 3abedf5a61..ffcb336a01 100644 --- a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskActionTest.java +++ b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/MaskActionTest.java @@ -16,7 +16,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; -class MaskActionTest { +class MaskActionTest implements ObfuscationActionTest { private MaskAction maskAction; @@ -25,7 +25,7 @@ class MaskActionTest { void testObfuscateWithPatternAsNull() { String message = "Hello"; maskAction = createMaskAction("*", 3); - String result = maskAction.obfuscate(message, null); + String result = maskAction.obfuscate(message, null, createRecord(message)); assertThat(result, equalTo("***")); } @@ -39,7 +39,7 @@ void testObfuscateWithPatternAsNull() { void testObfuscateWithDifferentConfig(String message, String maskCharacter, int maskCharacterLength, String expected) { maskAction = createMaskAction(maskCharacter, maskCharacterLength); List patterns = new ArrayList<>(); - String result = maskAction.obfuscate(message, patterns); + String result = maskAction.obfuscate(message, patterns,createRecord(message)); assertThat(result, equalTo(expected)); } diff --git a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationActionTest.java b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationActionTest.java new file mode 100644 index 0000000000..f43f3f6f99 --- /dev/null +++ b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/ObfuscationActionTest.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.obfuscation.action; + +import java.util.HashMap; +import java.util.Map; + +import org.opensearch.dataprepper.event.TestEventFactory; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; + +interface ObfuscationActionTest { + + default Record createRecord(String message) { + final Map testData = new HashMap<>(); + testData.put("message", message); + + return new Record<>(TestEventFactory.getTestEventFactory().eventBuilder(EventBuilder.class).withEventType("event").withData(testData).build()); + } +} diff --git a/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionTest.java b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionTest.java new file mode 100644 index 0000000000..8b974bed30 --- /dev/null +++ b/data-prepper-plugins/obfuscate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/obfuscation/action/OneWayHashActionTest.java @@ -0,0 +1,174 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.processor.obfuscation.action; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.provider.CsvSource; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.not; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.when; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import org.opensearch.dataprepper.event.TestEventKeyFactory; +import org.opensearch.dataprepper.model.event.EventKeyFactory; + +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class OneWayHashActionTest implements ObfuscationActionTest { + + @Mock + OneWayHashActionConfig mockConfig; + + private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); + + @Test + void testObfuscateWithPatternAsNull() { + String message = "Hello"; + when(mockConfig.getSaltKey()).thenReturn(null); + when(mockConfig.getSalt()).thenReturn(""); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + String result = new OneWayHashAction(mockConfig).obfuscate(message, null,createRecord(message)); + assertNotNull(result); + + assertThat(result, not(containsString(message))); + assertThat(result.length(), equalTo(88)); + } + + + @ParameterizedTest + @CsvSource({ + "Hello,AAAAAAAAAAAAAAAA,2NYZBaQ9nySumhHENpiKatKJhU3jqHC8jJ4DZC612RPGvkzPK1K12DskOI8Cn3qeOMSCTNIWErcGZr8JV4i9HQ==", + "Hi,BBBBBBBBBBBBBBBB,s3S4lyurJvJpQJ6EHN3gi/kexv79Ox+nIqXuVdbvgZP0b718AAxX0bOCPLeOZCnq3p3+DS+a0q0xLSJoMqjsNQ==", + "Hello,CCCCCCCCCCCCCCCC,SsUUpl/+GtU7cRg3ffuRKAtPU7cftdN440sNKR+gABy6JV6crwn5VTNSIqGKaTgBcZeYICy2ZmxP1DiHcW31rA==", + "H,DDDDDDDDDDDDDDDD,XR6utNkOp9te4+0vaRE0+ky/Zyw/gok1sI8qR/stZqFPoU733KwFcur36FCTUZd+i/UpyyJ9L/W6ObwPIf7iuw==", + }) + void testObfuscateWithDifferentConfig(String message, String salt, String expected) { + + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getSaltKey()).thenReturn(null); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + List patterns = new ArrayList<>(); + String result = oneWayHashAction.obfuscate(message, patterns,createRecord(message)); + assertThat(result, equalTo(expected)); + } + + + @ParameterizedTest + @CsvSource({ + "testing this functionality, test, AAAAAAAAAAAAAAAA, ILsULwmg32tiEQGqeX1rpWI9PGZXSX2Q9tRzXCD0cD/OKMMEBEXKYZhnXj1Xr9q+Dxa11iOmuXd+hx4ZTUaBCg==ing this functionality", + "test this functionality, test, BBBBBBBBBBBBBBBB, QT4wuvJSvgrxa/27gf4cZ1jzeNyiOnDxsY0oS7SsC/eVpBNyhj2I8Rh6/wCsvqRyzAvVoksTKOuRzSFUm6vAQw== this functionality", + "another test of this functionality, test, CCCCCCCCCCCCCCCC, another H9YrqOIlLtaoSCkNR2M0go3npf118KbsHFemyvJUX4+zt8FvjoiReq/0pk5va5i+7eX6XTOMwNokUUl4r+PTHw== of this functionality", + "My name is Bob and my email address is abc@example.com as of now and xyz@example.org in the future, [A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-] ,DDDDDDDDDDDDDDDD, My name is Bob and my email address is DdijIn6L3Cs4+PCYwCy+3bzLZ7w228quoodeI+VDlyMeFe+uZ/Ec1x/DK7MHSmZm8N5SZrINhvGgyig7aEBflg==om as of now and XQGlFjysVX1lkTFoRVCY+QEOfOf6nCoaRy5lxGAHyaFRgMGDpq93PwgZd18DZ3ZfWFRCwgPDGaExJDuRa0kkEQ==rg in the future", + }) + void testObfuscateWithPatterns(String message, String pattern, String salt, String expected) { + + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + + Pattern compiledPattern = Pattern.compile(pattern); + List patterns = new ArrayList<>(); + patterns.add(compiledPattern); + String result = oneWayHashAction.obfuscate(message, patterns,createRecord(message)); + assertThat(result, equalTo(expected)); + } + + @ParameterizedTest + @CsvSource({ + "testing this functionality and this test, test, this, AAAAAAAAAAAAAAAA, ILsULwmg32tiEQGqeX1rpWI9PGZXSX2Q9tRzXCD0cD/OKMMEBEXKYZhnXj1Xr9q+Dxa11iOmuXd+hx4ZTUaBCg==ing VsljIdInUvEk2ShjqBF94jgwWDk1lqcE/Fmb/LACPRlwIKsdmlk2PPX2o0XHObp4kRDqd+gUU5iUa/4HXhaA8g== functionality and VsljIdInUvEk2ShjqBF94jgwWDk1lqcE/Fmb/LACPRlwIKsdmlk2PPX2o0XHObp4kRDqd+gUU5iUa/4HXhaA8g== ILsULwmg32tiEQGqeX1rpWI9PGZXSX2Q9tRzXCD0cD/OKMMEBEXKYZhnXj1Xr9q+Dxa11iOmuXd+hx4ZTUaBCg==", + "test this functionality, test, this, BBBBBBBBBBBBBBBB, QT4wuvJSvgrxa/27gf4cZ1jzeNyiOnDxsY0oS7SsC/eVpBNyhj2I8Rh6/wCsvqRyzAvVoksTKOuRzSFUm6vAQw== LAD8UPdf/1cMoKY7Py17uRFNA+OEpVpa9lulTW8wEhsfQsDf/FvBIYxt/YO04sBI8CA1WY+i4elM5nY0xh13Lw== functionality", + "another test of this functionality, test, this, CCCCCCCCCCCCCCCC, another H9YrqOIlLtaoSCkNR2M0go3npf118KbsHFemyvJUX4+zt8FvjoiReq/0pk5va5i+7eX6XTOMwNokUUl4r+PTHw== of oAY9W4VW35Z14mrUisMks9mTILHsswbjjrJt96swt20/lnkMyf0izXV8OhQIh2N7Ml88uXU1fUfk0jTq41udfw== functionality", + "My name is Bob and my email address is abc@example.com as of now and xyz@example.org in the future, [A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-], Bob ,DDDDDDDDDDDDDDDD, My name is aDNCnlEqYbJO9KKnHEhhJSSyy2BB10CUSJxRMCSGLD1gdRNFVTo+Pz7xFepWfVOhuUGulvbnitdPoc8JIlEIFg== and my email address is DdijIn6L3Cs4+PCYwCy+3bzLZ7w228quoodeI+VDlyMeFe+uZ/Ec1x/DK7MHSmZm8N5SZrINhvGgyig7aEBflg==om as of now and XQGlFjysVX1lkTFoRVCY+QEOfOf6nCoaRy5lxGAHyaFRgMGDpq93PwgZd18DZ3ZfWFRCwgPDGaExJDuRa0kkEQ==rg in the future", + }) + void testObfuscateWithTwoPatterns(String message, String pattern1, String pattern2, String salt, String expected) { + + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + Pattern compiledPattern1 = Pattern.compile(pattern1); + Pattern compiledPattern2 = Pattern.compile(pattern2); + + List patterns = new ArrayList<>(); + patterns.add(compiledPattern1); + patterns.add(compiledPattern2); + String result = oneWayHashAction.obfuscate(message, patterns,createRecord(message)); + assertThat(result, equalTo(expected)); + } + + @ParameterizedTest + @CsvSource({ + "testing this functionality, test, AAAAAAAAAAAAAAAA, ILsULwmg32tiEQGqeX1rpWI9PGZXSX2Q9tRzXCD0cD/OKMMEBEXKYZhnXj1Xr9q+Dxa11iOmuXd+hx4ZTUaBCg==ing this functionality", + "test this functionality, test, BBBBBBBBBBBBBBBB, QT4wuvJSvgrxa/27gf4cZ1jzeNyiOnDxsY0oS7SsC/eVpBNyhj2I8Rh6/wCsvqRyzAvVoksTKOuRzSFUm6vAQw== this functionality", + "another test of this functionality, test, CCCCCCCCCCCCCCCC, another H9YrqOIlLtaoSCkNR2M0go3npf118KbsHFemyvJUX4+zt8FvjoiReq/0pk5va5i+7eX6XTOMwNokUUl4r+PTHw== of this functionality", + "My name is Bob and my email address is abc@example.com as of now and xyz@example.org in the future, [A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-] ,DDDDDDDDDDDDDDDD, My name is Bob and my email address is DdijIn6L3Cs4+PCYwCy+3bzLZ7w228quoodeI+VDlyMeFe+uZ/Ec1x/DK7MHSmZm8N5SZrINhvGgyig7aEBflg==om as of now and XQGlFjysVX1lkTFoRVCY+QEOfOf6nCoaRy5lxGAHyaFRgMGDpq93PwgZd18DZ3ZfWFRCwgPDGaExJDuRa0kkEQ==rg in the future", + }) + void testObfuscateWithPatternsAndInvalidSaltKey(String message, String pattern, String salt, String expected) { + + //adding SaltKey that cannot be found, to ensure that logic is defaulted back to the configured salt value. + when(mockConfig.getSaltKey()).thenReturn(eventKeyFactory.createEventKey("id")); + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + Pattern compiledPattern = Pattern.compile(pattern); + List patterns = new ArrayList<>(); + patterns.add(compiledPattern); + String result = oneWayHashAction.obfuscate(message, patterns,createRecord(message)); + assertThat(result, equalTo(expected)); + } + + @ParameterizedTest + @CsvSource({ + "testing this functionality, test, AAAAAAAAAAAAAAAA, 8g+p3Td+ClA+PttgNrZ8Qsg+tIc9/46TwNDtLeM6lQILI8jcQzPz0bOUM4IrbTlqgHYuOD8r6j6EElj4E6dZLw==ing this functionality", + "test this functionality, test, BBBBBBBBBBBBBBBB, 8g+p3Td+ClA+PttgNrZ8Qsg+tIc9/46TwNDtLeM6lQILI8jcQzPz0bOUM4IrbTlqgHYuOD8r6j6EElj4E6dZLw== this functionality", + "another test of this functionality, test, CCCCCCCCCCCCCCCC, another 8g+p3Td+ClA+PttgNrZ8Qsg+tIc9/46TwNDtLeM6lQILI8jcQzPz0bOUM4IrbTlqgHYuOD8r6j6EElj4E6dZLw== of this functionality", + "My name is Bob and my email address is abc@example.com as of now and xyz@example.org in the future, [A-Za-z0-9+_.-]+@([\\w-]+\\.)+[\\w-] ,DDDDDDDDDDDDDDDD, My name is Bob and my email address is 9zuqdjZfSkx7Xh6rO7bxRpREOmEA8EdtlNXOSviW6C41+sAK2QE/z9PGtRTf+T4bvTuzWBVv7SKVov6jII5+gw==om as of now and KAn0LtIRQYzoPtJqHczu21+gWcXl1OUUwbT9nY+2s+6164/PG4OuW/CZJIUZvOfrUICiL6BUJE32JCEaOfrwjA==rg in the future", + }) + void testObfuscateWithPatternsAndValidSaltKey(String message, String pattern, String salt, String expected) { + + //adding SaltKey that cannot be found, to ensure that logic is defaulted back to the configured salt value. + when(mockConfig.getSaltKey()).thenReturn(eventKeyFactory.createEventKey("message")); + when(mockConfig.getSalt()).thenReturn(salt); + when(mockConfig.getFormat()).thenReturn("SHA-512"); + + OneWayHashAction oneWayHashAction = new OneWayHashAction(mockConfig); + + final Map testData = new HashMap<>(); + testData.put("message", message); + Pattern compiledPattern = Pattern.compile(pattern); + List patterns = new ArrayList<>(); + patterns.add(compiledPattern); + + String result = oneWayHashAction.obfuscate(message, patterns,createRecord("12345")); + assertThat(result, equalTo(expected)); + } + +} From 00f5864f0f6a92ab31080ff80f84c113d8cfee79 Mon Sep 17 00:00:00 2001 From: David Venable Date: Tue, 30 Jul 2024 11:35:12 -0500 Subject: [PATCH 094/159] Reapply "Run tests on the current JVM for Java 17 & 21 / Gradle 8.8 (#4730)" (#4762) (#4771) This reverts commit 5c7d58c03059c7a753d882f5b74fa6ed32f45641. Signed-off-by: David Venable --- build.gradle | 3 ++ .../dataprepper/model/sink/AbstractSink.java | 6 ++-- .../dataprepper/model/sink/SinkThread.java | 8 ++++- .../model/sink/AbstractSinkTest.java | 22 +++++++----- data-prepper-core/build.gradle | 3 -- .../avro/AvroAutoSchemaGeneratorTest.java | 4 +-- .../blockingbuffer/BlockingBufferTests.java | 2 +- .../event_json/EventJsonInputCodecTest.java | 34 +++++++++++-------- .../EventJsonInputOutputCodecTest.java | 26 ++++++++------ .../event_json/EventJsonOutputCodecTest.java | 10 +++--- gradle/wrapper/gradle-wrapper.properties | 2 +- gradlew | 2 +- 12 files changed, 72 insertions(+), 50 deletions(-) diff --git a/build.gradle b/build.gradle index f77ecc442b..3dccd497cf 100644 --- a/build.gradle +++ b/build.gradle @@ -226,6 +226,9 @@ subprojects { test { useJUnitPlatform() + javaLauncher = javaToolchains.launcherFor { + languageVersion = JavaLanguageVersion.current() + } reports { junitXml.required html.required diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java index 1c3e596265..26dd7e98a6 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/AbstractSink.java @@ -28,6 +28,7 @@ public abstract class AbstractSink> implements Sink { private Thread retryThread; private int maxRetries; private int waitTimeMs; + private SinkThread sinkThread; public AbstractSink(final PluginSetting pluginSetting, int numRetries, int waitTimeMs) { this.pluginMetrics = PluginMetrics.fromPluginSetting(pluginSetting); @@ -51,7 +52,8 @@ public void initialize() { // the exceptions which are not retryable. doInitialize(); if (!isReady() && retryThread == null) { - retryThread = new Thread(new SinkThread(this, maxRetries, waitTimeMs)); + sinkThread = new SinkThread(this, maxRetries, waitTimeMs); + retryThread = new Thread(sinkThread); retryThread.start(); } } @@ -76,7 +78,7 @@ public void output(Collection records) { @Override public void shutdown() { if (retryThread != null) { - retryThread.stop(); + sinkThread.stop(); } } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java index c304de37af..451cef7dff 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/sink/SinkThread.java @@ -10,6 +10,8 @@ class SinkThread implements Runnable { private int maxRetries; private int waitTimeMs; + private volatile boolean isStopped = false; + public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) { this.sink = sink; this.maxRetries = maxRetries; @@ -19,11 +21,15 @@ public SinkThread(AbstractSink sink, int maxRetries, int waitTimeMs) { @Override public void run() { int numRetries = 0; - while (!sink.isReady() && numRetries++ < maxRetries) { + while (!sink.isReady() && numRetries++ < maxRetries && !isStopped) { try { Thread.sleep(waitTimeMs); sink.doInitialize(); } catch (InterruptedException e){} } } + + public void stop() { + isStopped = true; + } } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java index 3b9fe7c007..8d1af7ea44 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/sink/AbstractSinkTest.java @@ -11,15 +11,10 @@ import org.opensearch.dataprepper.metrics.MetricNames; import org.opensearch.dataprepper.metrics.MetricsTestUtil; import org.opensearch.dataprepper.model.configuration.PluginSetting; -import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.event.EventHandle; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; -import static org.mockito.Mockito.when; -import static org.mockito.Mockito.mock; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; import java.time.Duration; import java.util.Arrays; @@ -30,6 +25,12 @@ import java.util.UUID; import static org.awaitility.Awaitility.await; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; class AbstractSinkTest { private int count; @@ -71,13 +72,13 @@ void testMetrics() { } @Test - void testSinkNotReady() { + void testSinkNotReady() throws InterruptedException { final String sinkName = "testSink"; final String pipelineName = "pipelineName"; MetricsTestUtil.initMetrics(); PluginSetting pluginSetting = new PluginSetting(sinkName, Collections.emptyMap()); pluginSetting.setPipelineName(pipelineName); - AbstractSink> abstractSink = new AbstractSinkNotReadyImpl(pluginSetting); + AbstractSinkNotReadyImpl abstractSink = new AbstractSinkNotReadyImpl(pluginSetting); abstractSink.initialize(); assertEquals(abstractSink.isReady(), false); assertEquals(abstractSink.getRetryThreadState(), Thread.State.RUNNABLE); @@ -87,7 +88,10 @@ void testSinkNotReady() { await().atMost(Duration.ofSeconds(5)) .until(abstractSink::isReady); assertEquals(abstractSink.getRetryThreadState(), Thread.State.TERMINATED); + int initCountBeforeShutdown = abstractSink.initCount; abstractSink.shutdown(); + Thread.sleep(200); + assertThat(abstractSink.initCount, equalTo(initCountBeforeShutdown)); } @Test diff --git a/data-prepper-core/build.gradle b/data-prepper-core/build.gradle index 080538c5e4..c939129a1c 100644 --- a/data-prepper-core/build.gradle +++ b/data-prepper-core/build.gradle @@ -48,7 +48,6 @@ dependencies { exclude group: 'commons-logging', module: 'commons-logging' } implementation 'software.amazon.cloudwatchlogs:aws-embedded-metrics:2.0.0-beta-1' - testImplementation 'org.apache.logging.log4j:log4j-jpl:2.23.0' testImplementation testLibs.spring.test implementation libs.armeria.core implementation libs.armeria.grpc @@ -89,8 +88,6 @@ task integrationTest(type: Test) { classpath = sourceSets.integrationTest.runtimeClasspath - systemProperty 'log4j.configurationFile', 'src/test/resources/log4j2.properties' - filter { includeTestsMatching '*IT' } diff --git a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java index 622eb56a1b..1b66b62c37 100644 --- a/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java +++ b/data-prepper-plugins/avro-codecs/src/test/java/org/opensearch/dataprepper/avro/AvroAutoSchemaGeneratorTest.java @@ -17,7 +17,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Random; +import java.util.Timer; import java.util.UUID; import java.util.stream.Stream; @@ -218,7 +218,7 @@ static class SomeUnknownTypesArgumentsProvider implements ArgumentsProvider { @Override public Stream provideArguments(ExtensionContext context) { return Stream.of( - arguments(Random.class), + arguments(Timer.class), arguments(InputStream.class), arguments(File.class) ); diff --git a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java index 194c810ec4..f3f28db174 100644 --- a/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java +++ b/data-prepper-plugins/blocking-buffer/src/test/java/org/opensearch/dataprepper/plugins/buffer/blockingbuffer/BlockingBufferTests.java @@ -328,7 +328,7 @@ public Stream provideArguments(final ExtensionContext conte return Stream.of( Arguments.of(0, randomInt + 1, 0.0), Arguments.of(1, 100, 1.0), - Arguments.of(randomInt, randomInt, 100.0), + Arguments.of(randomInt + 1, randomInt + 1, 100.0), Arguments.of(randomInt, randomInt + 250, ((double) randomInt / (randomInt + 250)) * 100), Arguments.of(6, 9, 66.66666666666666), Arguments.of(531, 1000, 53.1), diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java index f85d1c6605..a4b0377963 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputCodecTest.java @@ -11,9 +11,12 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; + import static org.mockito.Mockito.when; import static org.mockito.Mockito.mock; + import org.mockito.Mock; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.CoreMatchers.not; @@ -28,6 +31,7 @@ import java.io.ByteArrayInputStream; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.List; import java.util.LinkedList; import java.util.Map; @@ -56,7 +60,7 @@ public EventJsonInputCodec createInputCodec() { @ParameterizedTest @ValueSource(strings = {"", "{}"}) public void emptyTest(String input) throws Exception { - input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["+input+"]}"; + input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":[" + input + "]}"; ByteArrayInputStream inputStream = new ByteArrayInputStream(input.getBytes()); inputCodec = createInputCodec(); Consumer> consumer = mock(Consumer.class); @@ -70,15 +74,15 @@ public void inCompatibleVersionTest() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\""+EventJsonDefines.VERSION+"\":\"3.0\", \""+EventJsonDefines.EVENTS+"\":["; + String input = "{\"" + EventJsonDefines.VERSION + "\":\"3.0\", \"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } input += "]}"; @@ -95,15 +99,15 @@ public void basicTest() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["; + String input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } input += "]}"; @@ -111,8 +115,8 @@ public void basicTest() throws Exception { List> records = new LinkedList<>(); inputCodec.parse(inputStream, records::add); assertThat(records.size(), equalTo(2)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -126,15 +130,15 @@ public void test_with_timeReceivedOverridden() throws Exception { final String key = UUID.randomUUID().toString(); final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now().minusSeconds(5); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS).minusSeconds(5); Event event = createEvent(data, startTime); Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); - String input = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\", \""+EventJsonDefines.EVENTS+"\":["; + String input = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\", \"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - input += comma+"{\"data\":"+objectMapper.writeValueAsString(dataMap)+","+"\"metadata\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + input += comma + "{\"data\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"metadata\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } input += "]}"; @@ -142,8 +146,8 @@ public void test_with_timeReceivedOverridden() throws Exception { List> records = new LinkedList<>(); inputCodec.parse(inputStream, records::add); assertThat(records.size(), equalTo(2)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), not(equalTo(startTime))); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -159,7 +163,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent)logBuilder.build(); + final JacksonEvent event = (JacksonEvent) logBuilder.build(); return event; } diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java index 85e91e5a55..7ea8c49cd0 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonInputOutputCodecTest.java @@ -6,9 +6,12 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; + import static org.mockito.Mockito.when; import static org.mockito.Mockito.mock; + import org.mockito.Mock; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -22,6 +25,7 @@ import org.opensearch.dataprepper.model.log.JacksonLog; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.List; import java.util.LinkedList; import java.util.Map; @@ -64,7 +68,7 @@ public void basicTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); inputCodec = createInputCodec(); @@ -75,8 +79,8 @@ public void basicTest() throws Exception { inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(1)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -90,7 +94,7 @@ public void multipleEventsTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); inputCodec = createInputCodec(); @@ -103,8 +107,8 @@ public void multipleEventsTest() throws Exception { inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(3)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags().size(), equalTo(0)); @@ -122,7 +126,7 @@ public void extendedTest() throws Exception { Set tags = Set.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); List tagsList = tags.stream().collect(Collectors.toList()); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); Instant origTime = startTime.minusSeconds(5); event.getMetadata().setExternalOriginationTime(origTime); @@ -135,11 +139,11 @@ public void extendedTest() throws Exception { outputCodec.complete(outputStream); assertThat(outputCodec.getExtension(), equalTo(EventJsonOutputCodec.EVENT_JSON)); List> records = new LinkedList<>(); -inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); + inputCodec.parse(new ByteArrayInputStream(outputStream.toByteArray()), records::add); assertThat(records.size(), equalTo(1)); - for(Record record : records) { - Event e = (Event)record.getData(); + for (Record record : records) { + Event e = (Event) record.getData(); assertThat(e.get(key, String.class), equalTo(value)); assertThat(e.getMetadata().getTimeReceived(), equalTo(startTime)); assertThat(e.getMetadata().getTags(), equalTo(tags)); @@ -157,7 +161,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent)logBuilder.build(); + final JacksonEvent event = (JacksonEvent) logBuilder.build(); return event; } diff --git a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java index 51dda545cb..b32d2b62e9 100644 --- a/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java +++ b/data-prepper-plugins/event-json-codecs/src/test/java/org/opensearch/dataprepper/plugins/codec/event_json/EventJsonOutputCodecTest.java @@ -11,6 +11,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mock; + import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; @@ -22,6 +23,7 @@ import org.opensearch.dataprepper.model.log.JacksonLog; import java.time.Instant; +import java.time.temporal.ChronoUnit; import java.util.Map; import java.util.UUID; @@ -49,7 +51,7 @@ public void basicTest() throws Exception { final String value = UUID.randomUUID().toString(); Map data = Map.of(key, value); - Instant startTime = Instant.now(); + Instant startTime = Instant.now().truncatedTo(ChronoUnit.MICROS); Event event = createEvent(data, startTime); outputCodec = createOutputCodec(); outputCodec.start(outputStream, null, null); @@ -59,10 +61,10 @@ public void basicTest() throws Exception { Map dataMap = event.toMap(); Map metadataMap = objectMapper.convertValue(event.getMetadata(), Map.class); //String expectedOutput = "{\"version\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\",\""+EventJsonDefines.EVENTS+"\":["; - String expectedOutput = "{\""+EventJsonDefines.VERSION+"\":\""+DataPrepperVersion.getCurrentVersion().toString()+"\",\""+EventJsonDefines.EVENTS+"\":["; + String expectedOutput = "{\"" + EventJsonDefines.VERSION + "\":\"" + DataPrepperVersion.getCurrentVersion().toString() + "\",\"" + EventJsonDefines.EVENTS + "\":["; String comma = ""; for (int i = 0; i < 2; i++) { - expectedOutput += comma+"{\""+EventJsonDefines.DATA+"\":"+objectMapper.writeValueAsString(dataMap)+","+"\""+EventJsonDefines.METADATA+"\":"+objectMapper.writeValueAsString(metadataMap)+"}"; + expectedOutput += comma + "{\"" + EventJsonDefines.DATA + "\":" + objectMapper.writeValueAsString(dataMap) + "," + "\"" + EventJsonDefines.METADATA + "\":" + objectMapper.writeValueAsString(metadataMap) + "}"; comma = ","; } expectedOutput += "]}"; @@ -78,7 +80,7 @@ private Event createEvent(final Map json, final Instant timeRece if (timeReceived != null) { logBuilder.withTimeReceived(timeReceived); } - final JacksonEvent event = (JacksonEvent)logBuilder.build(); + final JacksonEvent event = (JacksonEvent) logBuilder.build(); return event; } diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index b82aa23a4f..a4413138c9 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-8.7-bin.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-8.8-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 1aa94a4269..b740cf1339 100755 --- a/gradlew +++ b/gradlew @@ -55,7 +55,7 @@ # Darwin, MinGW, and NonStop. # # (3) This script is generated from the Groovy template -# https://github.com/gradle/gradle/blob/HEAD/subprojects/plugins/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt +# https://github.com/gradle/gradle/blob/HEAD/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt # within the Gradle project. # # You can find Gradle at https://github.com/gradle/gradle/. From 944681f438ba6c84c1d9c01f72594d22a93a49f5 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Tue, 30 Jul 2024 12:09:20 -0500 Subject: [PATCH 095/159] Add rds source metrics (#4769) * Add rds source metrics Signed-off-by: Hai Yan * Remove unused imports Signed-off-by: Hai Yan * Add exportS3ObjectsErrors metric Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- data-prepper-plugins/rds-source/build.gradle | 3 + .../plugins/source/rds/RdsService.java | 2 +- .../source/rds/export/DataFileLoader.java | 46 +++++- .../source/rds/export/DataFileScheduler.java | 24 ++- .../source/rds/export/ExportScheduler.java | 27 +++- .../rds/stream/BinlogEventListener.java | 52 ++++++- .../source/rds/stream/StreamScheduler.java | 3 +- .../source/rds/export/DataFileLoaderTest.java | 137 ++++++++++++++++-- .../rds/export/DataFileSchedulerTest.java | 63 +++++++- .../rds/export/ExportSchedulerTest.java | 82 ++++++++++- .../rds/stream/BinlogEventListenerTest.java | 6 +- 11 files changed, 404 insertions(+), 41 deletions(-) diff --git a/data-prepper-plugins/rds-source/build.gradle b/data-prepper-plugins/rds-source/build.gradle index 6d6a681646..14c851f645 100644 --- a/data-prepper-plugins/rds-source/build.gradle +++ b/data-prepper-plugins/rds-source/build.gradle @@ -26,4 +26,7 @@ dependencies { testImplementation project(path: ':data-prepper-test-common') testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation project(path: ':data-prepper-test-event') + testImplementation libs.avro.core + testImplementation libs.parquet.hadoop + testImplementation libs.parquet.avro } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index 982751a3db..73b71e0085 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -88,7 +88,7 @@ public void start(Buffer> buffer) { exportScheduler = new ExportScheduler( sourceCoordinator, snapshotManager, exportTaskManager, s3Client, pluginMetrics); dataFileScheduler = new DataFileScheduler( - sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer); + sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer, pluginMetrics); runnableList.add(exportScheduler); runnableList.add(dataFileScheduler); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java index f12d44a75f..42a5b3a0d5 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -5,7 +5,10 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; @@ -18,12 +21,18 @@ import java.io.InputStream; import java.time.Duration; import java.util.List; +import java.util.concurrent.atomic.AtomicLong; public class DataFileLoader implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(DataFileLoader.class); static final Duration VERSION_OVERLAP_TIME_FOR_EXPORT = Duration.ofMinutes(5); + static final String EXPORT_RECORDS_TOTAL_COUNT = "exportRecordsTotal"; + static final String EXPORT_RECORDS_PROCESSED_COUNT = "exportRecordsProcessed"; + static final String EXPORT_RECORDS_PROCESSING_ERROR_COUNT = "exportRecordsProcessingErrors"; + static final String BYTES_RECEIVED = "bytesReceived"; + static final String BYTES_PROCESSED = "bytesProcessed"; private final DataFilePartition dataFilePartition; private final String bucket; @@ -32,12 +41,18 @@ public class DataFileLoader implements Runnable { private final InputCodec codec; private final BufferAccumulator> bufferAccumulator; private final ExportRecordConverter recordConverter; + private final Counter exportRecordsTotalCounter; + private final Counter exportRecordSuccessCounter; + private final Counter exportRecordErrorCounter; + private final DistributionSummary bytesReceivedSummary; + private final DistributionSummary bytesProcessedSummary; private DataFileLoader(final DataFilePartition dataFilePartition, - final InputCodec codec, - final BufferAccumulator> bufferAccumulator, - final S3ObjectReader objectReader, - final ExportRecordConverter recordConverter) { + final InputCodec codec, + final BufferAccumulator> bufferAccumulator, + final S3ObjectReader objectReader, + final ExportRecordConverter recordConverter, + final PluginMetrics pluginMetrics) { this.dataFilePartition = dataFilePartition; bucket = dataFilePartition.getBucket(); objectKey = dataFilePartition.getKey(); @@ -45,24 +60,37 @@ private DataFileLoader(final DataFilePartition dataFilePartition, this.codec = codec; this.bufferAccumulator = bufferAccumulator; this.recordConverter = recordConverter; + + exportRecordsTotalCounter = pluginMetrics.counter(EXPORT_RECORDS_TOTAL_COUNT); + exportRecordSuccessCounter = pluginMetrics.counter(EXPORT_RECORDS_PROCESSED_COUNT); + exportRecordErrorCounter = pluginMetrics.counter(EXPORT_RECORDS_PROCESSING_ERROR_COUNT); + bytesReceivedSummary = pluginMetrics.summary(BYTES_RECEIVED); + bytesProcessedSummary = pluginMetrics.summary(BYTES_PROCESSED); } public static DataFileLoader create(final DataFilePartition dataFilePartition, final InputCodec codec, final BufferAccumulator> bufferAccumulator, final S3ObjectReader objectReader, - final ExportRecordConverter recordConverter) { - return new DataFileLoader(dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter); + final ExportRecordConverter recordConverter, + final PluginMetrics pluginMetrics) { + return new DataFileLoader(dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter, pluginMetrics); } @Override public void run() { LOG.info("Start loading s3://{}/{}", bucket, objectKey); + AtomicLong eventCount = new AtomicLong(); try (InputStream inputStream = objectReader.readFile(bucket, objectKey)) { - codec.parse(inputStream, record -> { try { + exportRecordsTotalCounter.increment(); + final Event event = record.getData(); + final String string = event.toJsonString(); + final long bytes = string.getBytes().length; + bytesReceivedSummary.record(bytes); + DataFileProgressState progressState = dataFilePartition.getProgressState().get(); // TODO: primary key to be obtained by querying database schema @@ -79,6 +107,8 @@ public void run() { snapshotTime, eventVersionNumber)); bufferAccumulator.add(transformedRecord); + eventCount.getAndIncrement(); + bytesProcessedSummary.record(bytes); } catch (Exception e) { throw new RuntimeException(e); } @@ -92,8 +122,10 @@ public void run() { try { bufferAccumulator.flush(); + exportRecordSuccessCounter.increment(eventCount.get()); } catch (Exception e) { LOG.error("Failed to write events to buffer", e); + exportRecordErrorCounter.increment(eventCount.get()); } } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java index 0a2b2fb638..f766aec3d2 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; @@ -48,6 +50,9 @@ public class DataFileScheduler implements Runnable { static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; + static final String EXPORT_S3_OBJECTS_PROCESSED_COUNT = "exportS3ObjectsProcessed"; + static final String EXPORT_S3_OBJECTS_ERROR_COUNT = "exportS3ObjectsErrors"; + static final String ACTIVE_EXPORT_S3_OBJECT_CONSUMERS_GAUGE = "activeExportS3ObjectConsumers"; private final EnhancedSourceCoordinator sourceCoordinator; @@ -57,6 +62,11 @@ public class DataFileScheduler implements Runnable { private final InputCodec codec; private final BufferAccumulator> bufferAccumulator; private final ExportRecordConverter recordConverter; + private final PluginMetrics pluginMetrics; + + private final Counter exportFileSuccessCounter; + private final Counter exportFileErrorCounter; + private final AtomicInteger activeExportS3ObjectConsumersGauge; private volatile boolean shutdownRequested = false; @@ -64,7 +74,8 @@ public DataFileScheduler(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig, final S3Client s3Client, final EventFactory eventFactory, - final Buffer> buffer) { + final Buffer> buffer, + final PluginMetrics pluginMetrics) { this.sourceCoordinator = sourceCoordinator; this.sourceConfig = sourceConfig; codec = new ParquetInputCodec(eventFactory); @@ -72,6 +83,12 @@ public DataFileScheduler(final EnhancedSourceCoordinator sourceCoordinator, objectReader = new S3ObjectReader(s3Client); recordConverter = new ExportRecordConverter(); executor = Executors.newFixedThreadPool(DATA_LOADER_MAX_JOB_COUNT); + this.pluginMetrics = pluginMetrics; + + this.exportFileSuccessCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_PROCESSED_COUNT); + this.exportFileErrorCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_ERROR_COUNT); + this.activeExportS3ObjectConsumersGauge = pluginMetrics.gauge( + ACTIVE_EXPORT_S3_OBJECT_CONSUMERS_GAUGE, numOfWorkers, AtomicInteger::get); } @Override @@ -116,15 +133,18 @@ public void shutdown() { } private void processDataFilePartition(DataFilePartition dataFilePartition) { - Runnable loader = DataFileLoader.create(dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter); + Runnable loader = DataFileLoader.create( + dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter, pluginMetrics); CompletableFuture runLoader = CompletableFuture.runAsync(loader, executor); runLoader.whenComplete((v, ex) -> { if (ex == null) { + exportFileSuccessCounter.increment(); // Update global state so we know if all s3 files have been loaded updateLoadStatus(dataFilePartition.getExportTaskId(), DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT); sourceCoordinator.completePartition(dataFilePartition); } else { + exportFileErrorCounter.increment(); LOG.error("There was an exception while processing an S3 data file", ex); sourceCoordinator.giveUpPartition(dataFilePartition); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index 79ef3d5a61..343ade8b85 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; @@ -42,12 +43,15 @@ public class ExportScheduler implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(ExportScheduler.class); private static final int DEFAULT_TAKE_LEASE_INTERVAL_MILLIS = 60_000; - private static final Duration DEFAULT_CLOSE_DURATION = Duration.ofMinutes(10); - private static final int DEFAULT_MAX_CLOSE_COUNT = 36; + static final Duration DEFAULT_CLOSE_DURATION = Duration.ofMinutes(10); + static final int DEFAULT_MAX_CLOSE_COUNT = 36; private static final int DEFAULT_CHECKPOINT_INTERVAL_MILLS = 5 * 60_000; private static final int DEFAULT_CHECK_STATUS_INTERVAL_MILLS = 30 * 1000; private static final Duration DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT = Duration.ofMinutes(60); static final String PARQUET_SUFFIX = ".parquet"; + static final String EXPORT_JOB_SUCCESS_COUNT = "exportJobSuccess"; + static final String EXPORT_JOB_FAILURE_COUNT = "exportJobFailure"; + static final String EXPORT_S3_OBJECTS_TOTAL_COUNT = "exportS3ObjectsTotal"; private final S3Client s3Client; private final PluginMetrics pluginMetrics; @@ -56,6 +60,10 @@ public class ExportScheduler implements Runnable { private final ExportTaskManager exportTaskManager; private final SnapshotManager snapshotManager; + private final Counter exportJobSuccessCounter; + private final Counter exportJobFailureCounter; + private final Counter exportS3ObjectsTotalCounter; + private volatile boolean shutdownRequested = false; public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, @@ -69,6 +77,10 @@ public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, this.executor = Executors.newCachedThreadPool(); this.snapshotManager = snapshotManager; this.exportTaskManager = exportTaskManager; + + exportJobSuccessCounter = pluginMetrics.counter(EXPORT_JOB_SUCCESS_COUNT); + exportJobFailureCounter = pluginMetrics.counter(EXPORT_JOB_FAILURE_COUNT); + exportS3ObjectsTotalCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_TOTAL_COUNT); } @Override @@ -133,8 +145,7 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { progressState.setSnapshotId(snapshotInfo.getSnapshotId()); sourceCoordinator.saveProgressStateForPartition(exportPartition, null); } else { - LOG.error("The snapshot failed to create, it will be retried"); - closeExportPartitionWithError(exportPartition); + LOG.error("The snapshot failed to create. The export will be retried"); return null; } @@ -142,8 +153,7 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { try { snapshotInfo = checkSnapshotStatus(snapshotId, DEFAULT_SNAPSHOT_STATUS_CHECK_TIMEOUT); } catch (Exception e) { - LOG.warn("Check snapshot status for {} failed", snapshotId, e); - sourceCoordinator.giveUpPartition(exportPartition); + LOG.warn("Check snapshot status for {} failed. The export will be retried", snapshotId, e); return null; } progressState.setSnapshotTime(snapshotInfo.getCreateTime().toEpochMilli()); @@ -159,7 +169,6 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { sourceCoordinator.saveProgressStateForPartition(exportPartition, null); } else { LOG.error("The export task failed to create, it will be retried"); - closeExportPartitionWithError(exportPartition); return null; } @@ -167,6 +176,7 @@ private String getOrCreateExportTaskId(ExportPartition exportPartition) { } private void closeExportPartitionWithError(ExportPartition exportPartition) { + exportJobFailureCounter.increment(); ExportProgressState exportProgressState = exportPartition.getProgressState().get(); // Clear current task id, so that a new export can be submitted. exportProgressState.setExportTaskId(null); @@ -309,12 +319,15 @@ private void createDataFilePartitions(String bucket, String exportTaskId, List> bufferAccumulator; private final List tableNames; private final String s3Prefix; + private final PluginMetrics pluginMetrics; - public BinlogEventListener(final Buffer> buffer, final RdsSourceConfig sourceConfig) { + private final Counter changeEventSuccessCounter; + private final Counter changeEventErrorCounter; + private final DistributionSummary bytesReceivedSummary; + private final DistributionSummary bytesProcessedSummary; + + public BinlogEventListener(final Buffer> buffer, + final RdsSourceConfig sourceConfig, + final PluginMetrics pluginMetrics) { tableMetadataMap = new HashMap<>(); recordConverter = new StreamRecordConverter(sourceConfig.getStream().getPartitionCount()); bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); s3Prefix = sourceConfig.getS3Prefix(); tableNames = sourceConfig.getTableNames(); + this.pluginMetrics = pluginMetrics; + + changeEventSuccessCounter = pluginMetrics.counter(CHANGE_EVENTS_PROCESSED_COUNT); + changeEventErrorCounter = pluginMetrics.counter(CHANGE_EVENTS_PROCESSING_ERROR_COUNT); + bytesReceivedSummary = pluginMetrics.summary(BYTES_RECEIVED); + bytesProcessedSummary = pluginMetrics.summary(BYTES_PROCESSED); } @Override @@ -95,7 +116,9 @@ void handleTableMapEvent(com.github.shyiko.mysql.binlog.event.Event event) { } void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { - // get new row data from the event + final long bytes = event.toString().getBytes().length; + bytesReceivedSummary.record(bytes); + LOG.debug("Handling insert event"); final WriteRowsEventData data = event.getData(); if (!tableMetadataMap.containsKey(data.getTableId())) { @@ -113,6 +136,7 @@ void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { final long eventTimestampMillis = event.getHeader().getTimestamp(); // Construct data prepper JacksonEvent + int eventCount = 0; for (final Object[] rowDataArray : data.getRows()) { final Map rowDataMap = new HashMap<>(); for (int i = 0; i < rowDataArray.length; i++) { @@ -130,12 +154,17 @@ void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { eventTimestampMillis, eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); + eventCount++; } + bytesProcessedSummary.record(bytes); - flushBuffer(); + flushBuffer(eventCount); } void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { + final long bytes = event.toString().getBytes().length; + bytesReceivedSummary.record(bytes); + LOG.debug("Handling update event"); final UpdateRowsEventData data = event.getData(); if (!tableMetadataMap.containsKey(data.getTableId())) { @@ -151,6 +180,7 @@ void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = event.getHeader().getTimestamp(); + int eventCount = 0; for (Map.Entry updatedRow : data.getRows()) { // updatedRow contains data before update as key and data after update as value final Object[] rowData = updatedRow.getValue(); @@ -171,12 +201,17 @@ void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { eventTimestampMillis, eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); + eventCount++; } + bytesProcessedSummary.record(bytes); - flushBuffer(); + flushBuffer(eventCount); } void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { + final long bytes = event.toString().getBytes().length; + bytesReceivedSummary.record(bytes); + LOG.debug("Handling delete event"); final DeleteRowsEventData data = event.getData(); if (!tableMetadataMap.containsKey(data.getTableId())) { @@ -193,6 +228,7 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = event.getHeader().getTimestamp(); + int eventCount = 0; for (Object[] rowDataArray : data.getRows()) { final Map rowDataMap = new HashMap<>(); for (int i = 0; i < rowDataArray.length; i++) { @@ -210,9 +246,11 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { eventTimestampMillis, eventTimestampMillis); addToBuffer(new Record<>(pipelineEvent)); + eventCount++; } + bytesProcessedSummary.record(bytes); - flushBuffer(); + flushBuffer(eventCount); } private boolean isTableOfInterest(String tableName) { @@ -227,11 +265,13 @@ private void addToBuffer(final Record record) { } } - private void flushBuffer() { + private void flushBuffer(int eventCount) { try { bufferAccumulator.flush(); + changeEventSuccessCounter.increment(eventCount); } catch (Exception e) { LOG.error("Failed to flush buffer", e); + changeEventErrorCounter.increment(eventCount); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java index 0b42c95c38..acb4ea3f85 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java @@ -41,8 +41,9 @@ public StreamScheduler(final EnhancedSourceCoordinator sourceCoordinator, this.sourceCoordinator = sourceCoordinator; this.sourceConfig = sourceConfig; this.binaryLogClient = binaryLogClient; - this.binaryLogClient.registerEventListener(new BinlogEventListener(buffer, sourceConfig)); + this.binaryLogClient.registerEventListener(new BinlogEventListener(buffer, sourceConfig, pluginMetrics)); this.pluginMetrics = pluginMetrics; + } @Override diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java index 1ed91bc031..ccb36347fa 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java @@ -5,38 +5,61 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; +import org.apache.avro.generic.GenericRecord; +import org.apache.parquet.avro.AvroParquetReader; +import org.apache.parquet.hadoop.ParquetReader; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; import org.mockito.Mock; +import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.BaseEventBuilder; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.io.InputFile; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.codec.parquet.ParquetInputCodec; import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.DataFileProgressState; import java.io.InputStream; +import java.util.Optional; import java.util.UUID; -import java.util.function.Consumer; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.BYTES_PROCESSED; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.BYTES_RECEIVED; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.EXPORT_RECORDS_PROCESSED_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.EXPORT_RECORDS_PROCESSING_ERROR_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileLoader.EXPORT_RECORDS_TOTAL_COUNT; @ExtendWith(MockitoExtension.class) class DataFileLoaderTest { - @Mock + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private DataFilePartition dataFilePartition; @Mock private BufferAccumulator> bufferAccumulator; @Mock - private InputCodec codec; + private EventFactory eventFactory; @Mock private S3ObjectReader s3ObjectReader; @@ -44,24 +67,120 @@ class DataFileLoaderTest { @Mock private ExportRecordConverter recordConverter; + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private Counter exportRecordsTotalCounter; + + @Mock + private Counter exportRecordSuccessCounter; + + @Mock + private Counter exportRecordErrorCounter; + + @Mock + private DistributionSummary bytesReceivedSummary; + + @Mock + private DistributionSummary bytesProcessedSummary; + + @BeforeEach + void setUp() { + when(pluginMetrics.counter(EXPORT_RECORDS_TOTAL_COUNT)).thenReturn(exportRecordsTotalCounter); + when(pluginMetrics.counter(EXPORT_RECORDS_PROCESSED_COUNT)).thenReturn(exportRecordSuccessCounter); + when(pluginMetrics.counter(EXPORT_RECORDS_PROCESSING_ERROR_COUNT)).thenReturn(exportRecordErrorCounter); + when(pluginMetrics.summary(BYTES_RECEIVED)).thenReturn(bytesReceivedSummary); + when(pluginMetrics.summary(BYTES_PROCESSED)).thenReturn(bytesProcessedSummary); + } + + @Test + void test_run_success() throws Exception { + final String bucket = UUID.randomUUID().toString(); + final String key = UUID.randomUUID().toString(); + when(dataFilePartition.getBucket()).thenReturn(bucket); + when(dataFilePartition.getKey()).thenReturn(key); + final DataFileProgressState progressState = mock(DataFileProgressState.class, RETURNS_DEEP_STUBS); + when(dataFilePartition.getProgressState()).thenReturn(Optional.of(progressState)); + + InputStream inputStream = mock(InputStream.class); + when(s3ObjectReader.readFile(bucket, key)).thenReturn(inputStream); + + DataFileLoader dataFileLoader = createObjectUnderTest(); + + final String randomString = UUID.randomUUID().toString(); + final long sizeBytes = randomString.getBytes().length; + final BaseEventBuilder eventBuilder = mock(EventBuilder.class, RETURNS_DEEP_STUBS); + final Event event = mock(Event.class); + when(eventFactory.eventBuilder(any())).thenReturn(eventBuilder); + when(eventBuilder.withEventType(any()).withData(any()).build()).thenReturn(event); + when(event.toJsonString()).thenReturn(randomString); + + try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class)) { + ParquetReader parquetReader = mock(ParquetReader.class); + AvroParquetReader.Builder builder = mock(AvroParquetReader.Builder.class); + readerMockedStatic.when(() -> AvroParquetReader.builder(any(InputFile.class), any())).thenReturn(builder); + when(builder.build()).thenReturn(parquetReader); + when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + + dataFileLoader.run(); + } + + verify(bufferAccumulator).add(any(Record.class)); + verify(bufferAccumulator).flush(); + + verify(exportRecordsTotalCounter).increment(); + verify(bytesReceivedSummary).record(sizeBytes); + verify(bytesProcessedSummary).record(sizeBytes); + verify(exportRecordSuccessCounter).increment(1); + verify(exportRecordErrorCounter, never()).increment(1); + } + @Test - void test_run() throws Exception { + void test_flush_failure_then_error_metric_updated() throws Exception { final String bucket = UUID.randomUUID().toString(); final String key = UUID.randomUUID().toString(); when(dataFilePartition.getBucket()).thenReturn(bucket); when(dataFilePartition.getKey()).thenReturn(key); + final DataFileProgressState progressState = mock(DataFileProgressState.class, RETURNS_DEEP_STUBS); + when(dataFilePartition.getProgressState()).thenReturn(Optional.of(progressState)); InputStream inputStream = mock(InputStream.class); when(s3ObjectReader.readFile(bucket, key)).thenReturn(inputStream); - DataFileLoader objectUnderTest = createObjectUnderTest(); - objectUnderTest.run(); + DataFileLoader dataFileLoader = createObjectUnderTest(); - verify(codec).parse(eq(inputStream), any(Consumer.class)); + final String randomString = UUID.randomUUID().toString(); + final long sizeBytes = randomString.getBytes().length; + final BaseEventBuilder eventBuilder = mock(EventBuilder.class, RETURNS_DEEP_STUBS); + final Event event = mock(Event.class); + when(eventFactory.eventBuilder(any())).thenReturn(eventBuilder); + when(eventBuilder.withEventType(any()).withData(any()).build()).thenReturn(event); + when(event.toJsonString()).thenReturn(randomString); + doThrow(new RuntimeException("testing")).when(bufferAccumulator).flush(); + + try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class)) { + ParquetReader parquetReader = mock(ParquetReader.class); + AvroParquetReader.Builder builder = mock(AvroParquetReader.Builder.class); + readerMockedStatic.when(() -> AvroParquetReader.builder(any(InputFile.class), any())).thenReturn(builder); + when(builder.build()).thenReturn(parquetReader); + when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + + dataFileLoader.run(); + } + + verify(bufferAccumulator).add(any(Record.class)); verify(bufferAccumulator).flush(); + + verify(exportRecordsTotalCounter).increment(); + verify(bytesReceivedSummary).record(sizeBytes); + verify(bytesProcessedSummary).record(sizeBytes); + verify(exportRecordSuccessCounter, never()).increment(1); + verify(exportRecordErrorCounter).increment(1); } private DataFileLoader createObjectUnderTest() { - return DataFileLoader.create(dataFilePartition, codec, bufferAccumulator, s3ObjectReader, recordConverter); + final InputCodec codec = new ParquetInputCodec(eventFactory); + return DataFileLoader.create(dataFilePartition, codec, bufferAccumulator, s3ObjectReader, recordConverter, pluginMetrics); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java index ee0d0e2852..5a5a56c6fd 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -12,6 +13,7 @@ import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; @@ -32,17 +34,22 @@ import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler.ACTIVE_EXPORT_S3_OBJECT_CONSUMERS_GAUGE; +import static org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler.EXPORT_S3_OBJECTS_PROCESSED_COUNT; @ExtendWith(MockitoExtension.class) class DataFileSchedulerTest { @@ -62,20 +69,37 @@ class DataFileSchedulerTest { @Mock private Buffer> buffer; + @Mock + private PluginMetrics pluginMetrics; + @Mock private DataFilePartition dataFilePartition; + @Mock + private Counter exportFileSuccessCounter; + + @Mock + private Counter exportFileErrorCounter; + + @Mock + private AtomicInteger activeExportS3ObjectConsumersGauge; + private Random random; @BeforeEach void setUp() { random = new Random(); + when(pluginMetrics.counter(EXPORT_S3_OBJECTS_PROCESSED_COUNT)).thenReturn(exportFileSuccessCounter); + when(pluginMetrics.counter(eq(DataFileScheduler.EXPORT_S3_OBJECTS_ERROR_COUNT))).thenReturn(exportFileErrorCounter); + when(pluginMetrics.gauge(eq(ACTIVE_EXPORT_S3_OBJECT_CONSUMERS_GAUGE), any(AtomicInteger.class), any())) + .thenReturn(activeExportS3ObjectConsumersGauge); } @Test void test_given_no_datafile_partition_then_no_export() throws InterruptedException { when(sourceCoordinator.acquireAvailablePartition(DataFilePartition.PARTITION_TYPE)).thenReturn(Optional.empty()); + final DataFileScheduler objectUnderTest = createObjectUnderTest(); final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(objectUnderTest); @@ -84,12 +108,11 @@ void test_given_no_datafile_partition_then_no_export() throws InterruptedExcepti Thread.sleep(100); executorService.shutdownNow(); - verifyNoInteractions(s3Client, buffer); + verifyNoInteractions(s3Client, buffer, exportFileSuccessCounter, activeExportS3ObjectConsumersGauge); } @Test void test_given_available_datafile_partition_then_load_datafile() { - DataFileScheduler objectUnderTest = createObjectUnderTest(); final String exportTaskId = UUID.randomUUID().toString(); when(dataFilePartition.getExportTaskId()).thenReturn(exportTaskId); @@ -100,13 +123,15 @@ void test_given_available_datafile_partition_then_load_datafile() { when(globalStatePartition.getProgressState()).thenReturn(Optional.of(loadStatusMap)); when(sourceCoordinator.getPartition(exportTaskId)).thenReturn(Optional.of(globalStatePartition)); + DataFileScheduler objectUnderTest = createObjectUnderTest(); final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(() -> { // MockedStatic needs to be created on the same thread it's used try (MockedStatic dataFileLoaderMockedStatic = mockStatic(DataFileLoader.class)) { DataFileLoader dataFileLoader = mock(DataFileLoader.class); dataFileLoaderMockedStatic.when(() -> DataFileLoader.create( - eq(dataFilePartition), any(InputCodec.class), any(BufferAccumulator.class), any(S3ObjectReader.class), any(ExportRecordConverter.class))) + eq(dataFilePartition), any(InputCodec.class), any(BufferAccumulator.class), any(S3ObjectReader.class), + any(ExportRecordConverter.class), any(PluginMetrics.class))) .thenReturn(dataFileLoader); doNothing().when(dataFileLoader).run(); objectUnderTest.run(); @@ -116,9 +141,39 @@ void test_given_available_datafile_partition_then_load_datafile() { .untilAsserted(() -> verify(sourceCoordinator).completePartition(dataFilePartition)); executorService.shutdownNow(); + verify(exportFileSuccessCounter).increment(); + verify(exportFileErrorCounter, never()).increment(); verify(sourceCoordinator).completePartition(dataFilePartition); } + @Test + void test_data_file_loader_throws_exception_then_give_up_partition() { + + when(sourceCoordinator.acquireAvailablePartition(DataFilePartition.PARTITION_TYPE)).thenReturn(Optional.of(dataFilePartition)); + + DataFileScheduler objectUnderTest = createObjectUnderTest(); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> { + // MockedStatic needs to be created on the same thread it's used + try (MockedStatic dataFileLoaderMockedStatic = mockStatic(DataFileLoader.class)) { + DataFileLoader dataFileLoader = mock(DataFileLoader.class); + dataFileLoaderMockedStatic.when(() -> DataFileLoader.create( + eq(dataFilePartition), any(InputCodec.class), any(BufferAccumulator.class), any(S3ObjectReader.class), + any(ExportRecordConverter.class), any(PluginMetrics.class))) + .thenReturn(dataFileLoader); + doThrow(new RuntimeException()).when(dataFileLoader).run(); + objectUnderTest.run(); + } + }); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).giveUpPartition(dataFilePartition)); + executorService.shutdownNow(); + + verify(exportFileSuccessCounter, never()).increment(); + verify(exportFileErrorCounter).increment(); + verify(sourceCoordinator).giveUpPartition(dataFilePartition); + } + @Test void test_shutdown() { DataFileScheduler objectUnderTest = createObjectUnderTest(); @@ -132,6 +187,6 @@ void test_shutdown() { } private DataFileScheduler createObjectUnderTest() { - return new DataFileScheduler(sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer); + return new DataFileScheduler(sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer, pluginMetrics); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java index a1a520a47a..f5036e8890 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.source.rds.export; +import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -42,6 +43,11 @@ import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.DEFAULT_CLOSE_DURATION; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.DEFAULT_MAX_CLOSE_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.EXPORT_JOB_FAILURE_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.EXPORT_JOB_SUCCESS_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.EXPORT_S3_OBJECTS_TOTAL_COUNT; import static org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler.PARQUET_SUFFIX; @@ -63,6 +69,15 @@ class ExportSchedulerTest { @Mock private PluginMetrics pluginMetrics; + @Mock + private Counter exportJobSuccessCounter; + + @Mock + private Counter exportJobFailureCounter; + + @Mock + private Counter exportS3ObjectsTotalCounter; + @Mock private ExportPartition exportPartition; @@ -73,6 +88,10 @@ class ExportSchedulerTest { @BeforeEach void setUp() { + when(pluginMetrics.counter(EXPORT_JOB_SUCCESS_COUNT)).thenReturn(exportJobSuccessCounter); + when(pluginMetrics.counter(EXPORT_JOB_FAILURE_COUNT)).thenReturn(exportJobFailureCounter); + when(pluginMetrics.counter(EXPORT_S3_OBJECTS_TOTAL_COUNT)).thenReturn(exportS3ObjectsTotalCounter); + exportScheduler = createObjectUnderTest(); } @@ -87,7 +106,8 @@ void test_given_no_export_partition_then_not_export() throws InterruptedExceptio Thread.sleep(100); executorService.shutdownNow(); - verifyNoInteractions(snapshotManager, exportTaskManager, s3Client); + verifyNoInteractions(snapshotManager, exportTaskManager, s3Client, exportJobSuccessCounter, + exportJobFailureCounter, exportS3ObjectsTotalCounter); } @Test @@ -123,9 +143,11 @@ void test_given_export_partition_and_export_task_id_then_complete_export() throw any(String.class), any(String.class), any(List.class)); verify(sourceCoordinator).createPartition(any(DataFilePartition.class)); verify(sourceCoordinator).completePartition(exportPartition); + verify(exportJobSuccessCounter).increment(); + verify(exportS3ObjectsTotalCounter).increment(1); + verify(exportJobFailureCounter, never()).increment(); } - @Test void test_given_export_partition_without_export_task_id_then_start_and_complete_export() throws InterruptedException { when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.of(exportPartition)); @@ -184,6 +206,59 @@ void test_given_export_partition_without_export_task_id_then_start_and_complete_ any(String.class), any(String.class), any(List.class)); verify(sourceCoordinator).createPartition(any(DataFilePartition.class)); verify(sourceCoordinator).completePartition(exportPartition); + verify(exportJobSuccessCounter).increment(); + verify(exportS3ObjectsTotalCounter).increment(1); + verify(exportJobFailureCounter, never()).increment(); + } + + @Test + void test_given_export_partition_and_null_export_task_id_then_close_partition_with_error() throws InterruptedException { + when(sourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).thenReturn(Optional.of(exportPartition)); + when(exportPartition.getPartitionKey()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getExportTaskId()).thenReturn(null); + when(exportPartition.getProgressState()).thenReturn(Optional.of(exportProgressState)); + final String dbIdentifier = UUID.randomUUID().toString(); + when(exportPartition.getDbIdentifier()).thenReturn(dbIdentifier); + + // Mock snapshot response + final String snapshotId = UUID.randomUUID().toString(); + final String snapshotArn = "arn:aws:rds:us-east-1:123456789012:snapshot:" + snapshotId; + final Instant createTime = Instant.now(); + final SnapshotInfo snapshotInfoWhenCreate = new SnapshotInfo( + snapshotId, snapshotArn, createTime, SnapshotStatus.CREATING.getStatusName()); + final SnapshotInfo snapshotInfoWhenComplete = new SnapshotInfo( + snapshotId, snapshotArn, createTime, SnapshotStatus.AVAILABLE.getStatusName()); + when(snapshotManager.createSnapshot(dbIdentifier)).thenReturn(snapshotInfoWhenCreate); + when(snapshotManager.checkSnapshotStatus(snapshotId)).thenReturn(snapshotInfoWhenComplete); + + // Mock export response + when(exportProgressState.getIamRoleArn()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getBucket()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getPrefix()).thenReturn(UUID.randomUUID().toString()); + when(exportProgressState.getKmsKeyId()).thenReturn(UUID.randomUUID().toString()); + when(exportTaskManager.startExportTask(any(String.class), any(String.class), any(String.class), + any(String.class), any(String.class), any(List.class))).thenReturn(null); + + // Act + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(exportScheduler); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(sourceCoordinator).acquireAvailablePartition(ExportPartition.PARTITION_TYPE)); + Thread.sleep(200); + executorService.shutdownNow(); + + // Assert + verify(snapshotManager).createSnapshot(dbIdentifier); + verify(exportTaskManager).startExportTask( + any(String.class), any(String.class), any(String.class), + any(String.class), any(String.class), any(List.class)); + verify(sourceCoordinator).closePartition(exportPartition, DEFAULT_CLOSE_DURATION, DEFAULT_MAX_CLOSE_COUNT); + verify(sourceCoordinator, never()).createPartition(any(DataFilePartition.class)); + verify(sourceCoordinator, never()).completePartition(exportPartition); + + verify(exportJobFailureCounter).increment(); + verify(exportJobSuccessCounter, never()).increment(); + verify(exportS3ObjectsTotalCounter, never()).increment(1); } @Test @@ -193,7 +268,8 @@ void test_shutDown() { final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(exportScheduler); exportScheduler.shutdown(); - verifyNoMoreInteractions(sourceCoordinator, snapshotManager, exportTaskManager, s3Client); + verifyNoMoreInteractions(sourceCoordinator, snapshotManager, exportTaskManager, s3Client, + exportJobSuccessCounter, exportJobFailureCounter, exportS3ObjectsTotalCounter); executorService.shutdownNow(); } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java index 406a89cec9..30f622c5d7 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java @@ -14,6 +14,7 @@ import org.mockito.Answers; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; @@ -33,6 +34,9 @@ class BinlogEventListenerTest { @Mock(answer = Answers.RETURNS_DEEP_STUBS) private RdsSourceConfig sourceConfig; + @Mock + private PluginMetrics pluginMetrics; + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private com.github.shyiko.mysql.binlog.event.Event binlogEvent; @@ -87,6 +91,6 @@ void test_given_DeleteRows_event_then_calls_correct_handler(EventType eventType) } private BinlogEventListener createObjectUnderTest() { - return new BinlogEventListener(buffer, sourceConfig); + return new BinlogEventListener(buffer, sourceConfig, pluginMetrics); } } \ No newline at end of file From e66f9bc0129770727d1922e4b95119c6763bbc2a Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Tue, 30 Jul 2024 13:33:51 -0700 Subject: [PATCH 096/159] Cleanup resources properly when Opensearch sink fails to initialize (#4758) * dplive1.yaml Signed-off-by: Krishna Kondaka * rebased to latest Signed-off-by: Krishna Kondaka * removed unnecessary file Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../sink/opensearch/OpenSearchClientRefresher.java | 5 ++++- .../plugins/sink/opensearch/OpenSearchSink.java | 7 ++++--- .../sink/opensearch/OpenSearchClientRefresherTest.java | 9 ++++++++- 3 files changed, 16 insertions(+), 5 deletions(-) diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresher.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresher.java index 6859dee8be..b697fb26bf 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresher.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresher.java @@ -30,7 +30,7 @@ public OpenSearchClientRefresher(final PluginMetrics pluginMetrics, final Function clientFunction) { this.clientFunction = clientFunction; this.currentConfig = connectionConfiguration; - this.currentClient = clientFunction.apply(connectionConfiguration); + this.currentClient = null; credentialsChangeCounter = pluginMetrics.counter(CREDENTIALS_CHANGED); clientRefreshErrorsCounter = pluginMetrics.counter(CLIENT_REFRESH_ERRORS); } @@ -44,6 +44,9 @@ public Class getComponentClass() { public OpenSearchClient get() { readWriteLock.readLock().lock(); try { + if (currentClient == null) { + currentClient = clientFunction.apply(currentConfig); + } return currentClient; } finally { readWriteLock.readLock().unlock(); 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 199b4e1e0e..1b6f44fde6 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 @@ -186,6 +186,7 @@ public OpenSearchSink(final PluginSetting pluginSetting, this.bulkRequestMap = new ConcurrentHashMap<>(); this.lastFlushTimeMap = new ConcurrentHashMap<>(); this.pluginConfigObservable = pluginConfigObservable; + this.objectMapper = new ObjectMapper(); final Optional dlqConfig = openSearchSinkConfig.getRetryConfiguration().getDlq(); if (dlqConfig.isPresent()) { @@ -201,7 +202,7 @@ public void doInitialize() { doInitializeInternal(); } catch (IOException e) { LOG.warn("Failed to initialize OpenSearch sink, retrying: {} ", e.getMessage()); - closeFiles(); + this.shutdown(); } catch (InvalidPluginConfigurationException e) { LOG.error("Failed to initialize OpenSearch sink due to a configuration error.", e); this.shutdown(); @@ -212,7 +213,7 @@ public void doInitialize() { throw e; } catch (Exception e) { LOG.warn("Failed to initialize OpenSearch sink with a retryable exception. ", e); - closeFiles(); + this.shutdown(); } } @@ -279,7 +280,6 @@ private void doInitializeInternal() throws IOException { bulkRequestSupplier, pluginSetting); - objectMapper = new ObjectMapper(); this.initialized = true; LOG.info("Initialized OpenSearch sink"); } @@ -615,6 +615,7 @@ private void closeFiles() { public void shutdown() { super.shutdown(); closeFiles(); + openSearchClient.shutdown(); } private void maybeUpdateServerlessNetworkPolicy() { diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresherTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresherTest.java index b9326c606f..584051dff6 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresherTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/OpenSearchClientRefresherTest.java @@ -71,6 +71,7 @@ void testGet() { @Test void testGetAfterUpdateWithDeprecatedBasicAuthUnchanged() { final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); when(connectionConfiguration.getUsername()).thenReturn(TEST_USERNAME); when(connectionConfiguration.getPassword()).thenReturn(TEST_PASSWORD); @@ -91,6 +92,7 @@ void testGetAfterUpdateWithDeprecatedBasicAuthUnchanged() { @Test void testGetAfterUpdateWithBasicAuthUnchanged() { final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); when(connectionConfiguration.getAuthConfig()).thenReturn(authConfig); when(authConfig.getUsername()).thenReturn(TEST_USERNAME); @@ -115,6 +117,7 @@ void testGetAfterUpdateWithBasicAuthUnchanged() { void testGetAfterUpdateWithDeprecatedUsernameChanged() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getUsername()).thenReturn(TEST_USERNAME); @@ -138,6 +141,7 @@ void testGetAfterUpdateWithDeprecatedUsernameChanged() { void testGetAfterUpdateWithUsernameChanged() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getAuthConfig()).thenReturn(authConfig); @@ -165,6 +169,7 @@ void testGetAfterUpdateWithUsernameChanged() { void testGetAfterUpdateWithDeprecatedPasswordChanged() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getUsername()).thenReturn(TEST_USERNAME); @@ -190,6 +195,7 @@ void testGetAfterUpdateWithDeprecatedPasswordChanged() { void testGetAfterUpdateWithPasswordChanged() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getAuthConfig()).thenReturn(authConfig); @@ -219,6 +225,7 @@ void testGetAfterUpdateClientFailure() { when(pluginMetrics.counter(CREDENTIALS_CHANGED)).thenReturn(credentialsChangeCounter); when(pluginMetrics.counter(CLIENT_REFRESH_ERRORS)).thenReturn(clientRefreshErrorsCounter); final OpenSearchClientRefresher objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.get(), equalTo(openSearchClient)); verify(clientFunction, times(1)).apply(any()); assertThat(objectUnderTest.get(), equalTo(openSearchClient)); when(connectionConfiguration.getUsername()).thenReturn(TEST_USERNAME); @@ -240,4 +247,4 @@ void testGetAfterUpdateClientFailure() { verify(clientRefreshErrorsCounter).increment(); verify(clientFunction, times(2)).apply(any()); } -} \ No newline at end of file +} From fd87e3d0c9359d5745f391e536e910c255292400 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 31 Jul 2024 18:45:39 -0700 Subject: [PATCH 097/159] Increase timeout in Acknowledgement IT tests (#4774) Increase timeout for acknowledgement IT tests Signed-off-by: Krishna Kondaka Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../integration/PipelinesWithAcksIT.java | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/PipelinesWithAcksIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/PipelinesWithAcksIT.java index 744105d46d..7d3a73d7a5 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/PipelinesWithAcksIT.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/PipelinesWithAcksIT.java @@ -68,7 +68,7 @@ void simple_pipeline_with_single_record() { final int numRecords = 1; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -84,7 +84,7 @@ void simple_pipeline_with_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -99,7 +99,7 @@ void two_pipelines_with_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -114,7 +114,7 @@ void three_pipelines_with_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -129,7 +129,7 @@ void three_pipelines_with_all_unrouted_records() { final int numRecords = 2; inMemorySourceAccessor.submitWithStatus(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { assertTrue(inMemorySourceAccessor != null); assertTrue(inMemorySourceAccessor.getAckReceived() != null); @@ -145,7 +145,7 @@ void three_pipelines_with_route_and_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submitWithStatus(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -161,7 +161,7 @@ void three_pipelines_with_default_route_and_multiple_records() { inMemorySourceAccessor.submitWithStatus(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -176,7 +176,7 @@ void two_parallel_pipelines_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -191,7 +191,7 @@ void three_pipelines_multi_sink_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -206,7 +206,7 @@ void one_pipeline_three_sinks_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -221,7 +221,7 @@ void one_pipeline_ack_expiry_multiple_records() { final int numRecords = 100; inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); @@ -237,7 +237,7 @@ void one_pipeline_three_sinks_negative_ack_multiple_records() { inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, numRecords); inMemorySinkAccessor.setResult(false); - await().atMost(20000, TimeUnit.MILLISECONDS) + await().atMost(40000, TimeUnit.MILLISECONDS) .untilAsserted(() -> { List> outputRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); assertThat(outputRecords, not(empty())); From ecc3bfb93070b6a82f81413a045fa969ce283399 Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 1 Aug 2024 16:11:53 -0500 Subject: [PATCH 098/159] Removes @asifsmohammed from the CODEOWNERS to allow the release to proceed. (#4800) Signed-off-by: David Venable --- .github/CODEOWNERS | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/CODEOWNERS b/.github/CODEOWNERS index babe6b3d56..54c084c0f8 100644 --- a/.github/CODEOWNERS +++ b/.github/CODEOWNERS @@ -1,2 +1,2 @@ # This should match the owning team set up in https://github.com/orgs/opensearch-project/teams -* @chenqi0805 @engechas @graytaylor0 @dinujoh @kkondaka @asifsmohammed @KarstenSchnitter @dlvenable @oeyh \ No newline at end of file +* @chenqi0805 @engechas @graytaylor0 @dinujoh @kkondaka @KarstenSchnitter @dlvenable @oeyh \ No newline at end of file From e22e96916bfe1cb314a021ed7f0bf33d6dd8371c Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 2 Aug 2024 11:28:55 -0500 Subject: [PATCH 099/159] Using Awaitility and mocks in the LogGeneratorSourceTest to attempt to improve reliability. (#4746) Signed-off-by: David Venable --- .../loggenerator/LogGeneratorSourceTest.java | 57 +++++++++++-------- 1 file changed, 32 insertions(+), 25 deletions(-) diff --git a/data-prepper-plugins/log-generator-source/src/test/java/org/opensearch/dataprepper/plugins/source/loggenerator/LogGeneratorSourceTest.java b/data-prepper-plugins/log-generator-source/src/test/java/org/opensearch/dataprepper/plugins/source/loggenerator/LogGeneratorSourceTest.java index 180fcae952..28d58f8cff 100644 --- a/data-prepper-plugins/log-generator-source/src/test/java/org/opensearch/dataprepper/plugins/source/loggenerator/LogGeneratorSourceTest.java +++ b/data-prepper-plugins/log-generator-source/src/test/java/org/opensearch/dataprepper/plugins/source/loggenerator/LogGeneratorSourceTest.java @@ -6,36 +6,37 @@ package org.opensearch.dataprepper.plugins.source.loggenerator; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mock; -import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.dataprepper.plugins.buffer.blockingbuffer.BlockingBuffer; import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.awaitility.Awaitility.await; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.spy; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; import static org.opensearch.dataprepper.plugins.source.loggenerator.LogGeneratorSourceConfig.INFINITE_LOG_COUNT; @@ -50,6 +51,9 @@ public class LogGeneratorSourceTest { private LogGeneratorSource logGeneratorSource; @Mock private PluginModel mockLogPluginModel; + @Mock + private Buffer buffer; + @BeforeEach public void setup() { when(sourceConfig.getLogType()).thenReturn(mockLogPluginModel); @@ -79,21 +83,23 @@ private LogGeneratorSource createObjectUnderTest() { @Test void GIVEN_logGeneratorSourceAndBlockingBuffer_WHEN_noLimit_THEN_keepsWritingToBufferUntilStopped() - throws InterruptedException, TimeoutException { + throws TimeoutException { logGeneratorSource = createObjectUnderTest(); - BlockingBuffer> spyBuffer = spy(new BlockingBuffer>("SamplePipeline")); - Duration interval = Duration.ofMillis(100); lenient().when(sourceConfig.getInterval()).thenReturn(interval); lenient().when(sourceConfig.getCount()).thenReturn(INFINITE_LOG_COUNT); // no limit to log count - logGeneratorSource.start(spyBuffer); - Thread.sleep((long) (interval.toMillis() * 1.5)); - verify(spyBuffer, atLeast(1)).write(any(Record.class), anyInt()); - Thread.sleep((long) (interval.toMillis() * 0.7)); - verify(spyBuffer, atLeast(2)).write(any(Record.class), anyInt()); + logGeneratorSource.start(buffer); + await() + .atMost((long) (interval.toMillis() * 1.5), TimeUnit.MILLISECONDS) + .untilAsserted(() -> verify(buffer, atLeast(1)).write(any(Record.class), anyInt())); + verify(buffer, atLeast(1)).write(any(Record.class), anyInt()); + await() + .atMost((long) (interval.toMillis() * 1.5), TimeUnit.MILLISECONDS) + .untilAsserted(() -> verify(buffer, atLeast(1)).write(any(Record.class), anyInt())); + verify(buffer, atLeast(2)).write(any(Record.class), anyInt()); } @Test @@ -101,20 +107,21 @@ void GIVEN_logGeneratorSourceAndBlockingBuffer_WHEN_reachedLimit_THEN_stopsWriti throws InterruptedException, TimeoutException { logGeneratorSource = createObjectUnderTest(); - BlockingBuffer> spyBuffer = spy(new BlockingBuffer>("SamplePipeline")); - Duration interval = Duration.ofMillis(100); lenient().when(sourceConfig.getInterval()).thenReturn(interval); lenient().when(sourceConfig.getCount()).thenReturn(1); // max log count of 1 in logGeneratorSource - assertEquals(spyBuffer.isEmpty(), true); - logGeneratorSource.start(spyBuffer); - Thread.sleep((long) (interval.toMillis() * 1.1)); + verifyNoInteractions(buffer); - verify(spyBuffer, times(1)).write(any(Record.class), anyInt()); + logGeneratorSource.start(buffer); - Thread.sleep(interval.toMillis()); - verify(spyBuffer, times(1)).write(any(Record.class), anyInt()); + await() + .atMost(interval.multipliedBy(3)) + .untilAsserted(() -> verify(buffer, atLeast(1)).write(any(Record.class), anyInt())); + verify(buffer, times(1)).write(any(Record.class), anyInt()); + + Thread.sleep((long) (interval.toMillis() * 1.1)); + verify(buffer, times(1)).write(any(Record.class), anyInt()); } } \ No newline at end of file From 642db0d49bb2ac9581c1c58602d9417cec1eb9d8 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 2 Aug 2024 14:44:32 -0500 Subject: [PATCH 100/159] ADD: data prepper plugin schema generation (#4777) * ADD: data-prepper-plugin-schema Signed-off-by: George Chen --- config/checkstyle/checkstyle-suppressions.xml | 2 + .../model/configuration/PipelineModel.java | 15 +- data-prepper-plugin-schema-cli/README.md | 12 ++ data-prepper-plugin-schema-cli/build.gradle | 29 ++++ .../DataPrepperPluginSchemaExecute.java | 74 ++++++++++ .../schemas/JsonSchemaConverter.java | 52 +++++++ .../PluginConfigsJsonSchemaConverter.java | 135 ++++++++++++++++++ .../schemas/module/CustomJacksonModule.java | 31 ++++ .../schemas/JsonSchemaConverterTest.java | 60 ++++++++ .../PluginConfigsJsonSchemaConverterIT.java | 80 +++++++++++ .../PluginConfigsJsonSchemaConverterTest.java | 110 ++++++++++++++ settings.gradle | 1 + 12 files changed, 596 insertions(+), 5 deletions(-) create mode 100644 data-prepper-plugin-schema-cli/README.md create mode 100644 data-prepper-plugin-schema-cli/build.gradle create mode 100644 data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java create mode 100644 data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java create mode 100644 data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java create mode 100644 data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java create mode 100644 data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java create mode 100644 data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java create mode 100644 data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java diff --git a/config/checkstyle/checkstyle-suppressions.xml b/config/checkstyle/checkstyle-suppressions.xml index 42c37e7dd5..ab3ba001a9 100644 --- a/config/checkstyle/checkstyle-suppressions.xml +++ b/config/checkstyle/checkstyle-suppressions.xml @@ -14,4 +14,6 @@ + + diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelineModel.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelineModel.java index 1c8221f899..7af56175a0 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelineModel.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelineModel.java @@ -24,25 +24,30 @@ * @since 1.2 */ public class PipelineModel { + public static final String SOURCE_PLUGIN_TYPE = "source"; + public static final String PROCESSOR_PLUGIN_TYPE = "processor"; + public static final String BUFFER_PLUGIN_TYPE = "buffer"; + public static final String ROUTE_PLUGIN_TYPE = "route"; + public static final String SINK_PLUGIN_TYPE = "sink"; private static final Logger LOG = LoggerFactory.getLogger(PipelineModel.class); - @JsonProperty("source") + @JsonProperty(SOURCE_PLUGIN_TYPE) private final PluginModel source; - @JsonProperty("processor") + @JsonProperty(PROCESSOR_PLUGIN_TYPE) @JsonInclude(JsonInclude.Include.NON_NULL) private final List processors; - @JsonProperty("buffer") + @JsonProperty(BUFFER_PLUGIN_TYPE) @JsonInclude(JsonInclude.Include.NON_NULL) private final PluginModel buffer; @JsonProperty("routes") - @JsonAlias("route") + @JsonAlias(ROUTE_PLUGIN_TYPE) @JsonInclude(JsonInclude.Include.NON_EMPTY) private final List routes; - @JsonProperty("sink") + @JsonProperty(SINK_PLUGIN_TYPE) private final List sinks; @JsonProperty("workers") diff --git a/data-prepper-plugin-schema-cli/README.md b/data-prepper-plugin-schema-cli/README.md new file mode 100644 index 0000000000..7a4d9bc11b --- /dev/null +++ b/data-prepper-plugin-schema-cli/README.md @@ -0,0 +1,12 @@ +# Data Prepper Plugin Schema CLI + +This module includes the SDK and CLI for generating schemas for Data Prepper pipeline plugins. + +## CLI Usage + +``` +./gradlew :data-prepper-plugin-schema-cli:run --args='--plugin_type=processor --plugin_names=grok' +``` + +* plugin_type: A required parameter specifies type of processor. Valid options are `source`, `buffer`, `processor`, `route`, `sink`. +* plugin_names: An optional parameter filters the result by plugin names separated by `,`, e.g. `grok,date`. diff --git a/data-prepper-plugin-schema-cli/build.gradle b/data-prepper-plugin-schema-cli/build.gradle new file mode 100644 index 0000000000..2c2db93ee6 --- /dev/null +++ b/data-prepper-plugin-schema-cli/build.gradle @@ -0,0 +1,29 @@ +plugins { + id 'data-prepper.publish' + id 'application' +} + +application { + mainClass = 'org.opensearch.dataprepper.schemas.DataPrepperPluginSchemaExecute' +} + +dependencies { + implementation project(':data-prepper-plugins') + implementation project(':data-prepper-plugin-framework') + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'org.reflections:reflections:0.10.2' + implementation 'com.github.victools:jsonschema-maven-plugin:4.35.0' + implementation 'com.github.victools:jsonschema-generator:4.35.0' + implementation 'com.github.victools:jsonschema-module-jackson:4.35.0' + implementation 'com.github.victools:jsonschema-module-jakarta-validation:4.35.0' + implementation 'javax.inject:javax.inject:1' + implementation 'info.picocli:picocli:4.6.1' + implementation(libs.spring.core) { + exclude group: 'commons-logging', module: 'commons-logging' + } + implementation(libs.spring.context) { + exclude group: 'commons-logging', module: 'commons-logging' + } + testImplementation(platform("org.junit:junit-bom:5.9.1")) + testImplementation("org.junit.jupiter:junit-jupiter") +} \ No newline at end of file diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java new file mode 100644 index 0000000000..a1a76c0510 --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java @@ -0,0 +1,74 @@ +package org.opensearch.dataprepper.schemas; + +import com.github.victools.jsonschema.generator.Module; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import com.github.victools.jsonschema.module.jakarta.validation.JakartaValidationModule; +import com.github.victools.jsonschema.module.jakarta.validation.JakartaValidationOption; +import org.opensearch.dataprepper.schemas.module.CustomJacksonModule; +import org.reflections.Reflections; +import org.reflections.scanners.Scanners; +import org.reflections.util.ClasspathHelper; +import org.reflections.util.ConfigurationBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import picocli.CommandLine; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static com.github.victools.jsonschema.module.jackson.JacksonOption.RESPECT_JSONPROPERTY_REQUIRED; + +public class DataPrepperPluginSchemaExecute implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(DataPrepperPluginSchemaExecute.class); + static final String DEFAULT_PLUGINS_CLASSPATH = "org.opensearch.dataprepper.plugins"; + + @CommandLine.Option(names = {"--plugin_type"}, required = true) + private String pluginTypeName; + + @CommandLine.Option(names = {"--plugin_names"}) + private String pluginNames; + + @CommandLine.Option(names = {"--site.url"}, defaultValue = "https://opensearch.org") + private String siteUrl; + @CommandLine.Option(names = {"--site.baseurl"}, defaultValue = "/docs/latest") + private String siteBaseUrl; + + public static void main(String[] args) { + final int exitCode = new CommandLine(new DataPrepperPluginSchemaExecute()).execute(args); + System.exit(exitCode); + } + + @Override + public void run() { + final List modules = List.of( + new CustomJacksonModule(RESPECT_JSONPROPERTY_REQUIRED), + new JakartaValidationModule(JakartaValidationOption.NOT_NULLABLE_FIELD_IS_REQUIRED, + JakartaValidationOption.INCLUDE_PATTERN_EXPRESSIONS) + ); + final Reflections reflections = new Reflections(new ConfigurationBuilder() + .setUrls(ClasspathHelper.forPackage(DEFAULT_PLUGINS_CLASSPATH)) + .setScanners(Scanners.TypesAnnotated, Scanners.SubTypes)); + final PluginConfigsJsonSchemaConverter pluginConfigsJsonSchemaConverter = new PluginConfigsJsonSchemaConverter( + reflections, new JsonSchemaConverter(modules), siteUrl, siteBaseUrl); + final Class pluginType = pluginConfigsJsonSchemaConverter.pluginTypeNameToPluginType(pluginTypeName); + final Map pluginNameToJsonSchemaMap = pluginConfigsJsonSchemaConverter.convertPluginConfigsIntoJsonSchemas( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, pluginType); + if (pluginNames == null) { + pluginNameToJsonSchemaMap.values().forEach(System.out::println); + } else { + final Set pluginNamesSet = Set.of(pluginNames.split(",")); + final List result = pluginNamesSet.stream().flatMap(name -> { + if (!pluginNameToJsonSchemaMap.containsKey(name)) { + LOG.error("plugin name: {} not found", name); + return Stream.empty(); + } + return Stream.of(pluginNameToJsonSchemaMap.get(name)); + }).collect(Collectors.toList()); + result.forEach(System.out::println); + } + } +} diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java new file mode 100644 index 0000000000..fe08825af4 --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java @@ -0,0 +1,52 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.github.victools.jsonschema.generator.FieldScope; +import com.github.victools.jsonschema.generator.Module; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaGenerator; +import com.github.victools.jsonschema.generator.SchemaGeneratorConfig; +import com.github.victools.jsonschema.generator.SchemaGeneratorConfigBuilder; +import com.github.victools.jsonschema.generator.SchemaGeneratorConfigPart; +import com.github.victools.jsonschema.generator.SchemaVersion; + +import java.util.List; + +public class JsonSchemaConverter { + static final String DEPRECATED_SINCE_KEY = "deprecated"; + private final List jsonSchemaGeneratorModules; + + public JsonSchemaConverter(final List jsonSchemaGeneratorModules) { + this.jsonSchemaGeneratorModules = jsonSchemaGeneratorModules; + } + + public ObjectNode convertIntoJsonSchema( + final SchemaVersion schemaVersion, final OptionPreset optionPreset, final Class clazz) + throws JsonProcessingException { + final SchemaGeneratorConfigBuilder configBuilder = new SchemaGeneratorConfigBuilder( + schemaVersion, optionPreset); + loadJsonSchemaGeneratorModules(configBuilder); + final SchemaGeneratorConfigPart scopeSchemaGeneratorConfigPart = configBuilder.forFields(); + overrideInstanceAttributeWithDeprecated(scopeSchemaGeneratorConfigPart); + + final SchemaGeneratorConfig config = configBuilder.build(); + final SchemaGenerator generator = new SchemaGenerator(config); + return generator.generateSchema(clazz); + } + + private void loadJsonSchemaGeneratorModules(final SchemaGeneratorConfigBuilder configBuilder) { + jsonSchemaGeneratorModules.forEach(configBuilder::with); + } + + private void overrideInstanceAttributeWithDeprecated( + final SchemaGeneratorConfigPart scopeSchemaGeneratorConfigPart) { + scopeSchemaGeneratorConfigPart.withInstanceAttributeOverride((node, field, context) -> { + final Deprecated deprecatedAnnotation = field.getAnnotationConsideringFieldAndGetter( + Deprecated.class); + if (deprecatedAnnotation != null) { + node.put(DEPRECATED_SINCE_KEY, deprecatedAnnotation.since()); + } + }); + } +} diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java new file mode 100644 index 0000000000..b7f4c1a531 --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java @@ -0,0 +1,135 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.configuration.ConditionalRoute; +import org.opensearch.dataprepper.model.processor.Processor; +import org.opensearch.dataprepper.model.sink.Sink; +import org.opensearch.dataprepper.model.source.Source; +import org.reflections.Reflections; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.opensearch.dataprepper.model.configuration.PipelineModel.BUFFER_PLUGIN_TYPE; +import static org.opensearch.dataprepper.model.configuration.PipelineModel.PROCESSOR_PLUGIN_TYPE; +import static org.opensearch.dataprepper.model.configuration.PipelineModel.ROUTE_PLUGIN_TYPE; +import static org.opensearch.dataprepper.model.configuration.PipelineModel.SINK_PLUGIN_TYPE; +import static org.opensearch.dataprepper.model.configuration.PipelineModel.SOURCE_PLUGIN_TYPE; + +public class PluginConfigsJsonSchemaConverter { + private static final Logger LOG = LoggerFactory.getLogger(PluginConfigsJsonSchemaConverter.class); + static final String SITE_URL_PLACEHOLDER = "{{site.url}}"; + static final String SITE_BASE_URL_PLACEHOLDER = "{{site.baseurl}}"; + static final String DOCUMENTATION_LINK_KEY = "documentation"; + static final String PLUGIN_NAME_KEY = "name"; + static final String PLUGIN_DOCUMENTATION_URL_FORMAT = + "%s%s/data-prepper/pipelines/configuration/%s/%s/"; + static final Map, String> PLUGIN_TYPE_TO_URI_PARAMETER_MAP = Map.of( + Source.class, "sources", + Processor.class, "processors", + ConditionalRoute.class, "processors", + Buffer.class, "buffers", + Sink.class, "sinks" + ); + static final String CONDITIONAL_ROUTE_PROCESSOR_NAME = "routes"; + static final Map> PLUGIN_TYPE_NAME_TO_CLASS_MAP = Map.of( + SOURCE_PLUGIN_TYPE, Source.class, + PROCESSOR_PLUGIN_TYPE, Processor.class, + ROUTE_PLUGIN_TYPE, ConditionalRoute.class, + BUFFER_PLUGIN_TYPE, Buffer.class, + SINK_PLUGIN_TYPE, Sink.class); + + private final String siteUrl; + private final String siteBaseUrl; + private final Reflections reflections; + private final JsonSchemaConverter jsonSchemaConverter; + + public PluginConfigsJsonSchemaConverter( + final Reflections reflections, + final JsonSchemaConverter jsonSchemaConverter, + final String siteUrl, + final String siteBaseUrl) { + this.reflections = reflections; + this.jsonSchemaConverter = jsonSchemaConverter; + this.siteUrl = siteUrl == null ? SITE_URL_PLACEHOLDER : siteUrl; + this.siteBaseUrl = siteBaseUrl == null ? SITE_BASE_URL_PLACEHOLDER : siteBaseUrl; + } + + public Set validPluginTypeNames() { + return PLUGIN_TYPE_NAME_TO_CLASS_MAP.keySet(); + } + + public Class pluginTypeNameToPluginType(final String pluginTypeName) { + final Class pluginType = PLUGIN_TYPE_NAME_TO_CLASS_MAP.get(pluginTypeName); + if (pluginType == null) { + throw new IllegalArgumentException(String.format("Invalid plugin type name: %s.", pluginTypeName)); + } + return pluginType; + } + + public Map convertPluginConfigsIntoJsonSchemas( + final SchemaVersion schemaVersion, final OptionPreset optionPreset, final Class pluginType) { + final Map> nameToConfigClass = scanForPluginConfigs(pluginType); + return nameToConfigClass.entrySet().stream() + .flatMap(entry -> { + final String pluginName = entry.getKey(); + String value; + try { + final ObjectNode jsonSchemaNode = jsonSchemaConverter.convertIntoJsonSchema( + schemaVersion, optionPreset, entry.getValue()); + addPluginName(jsonSchemaNode, pluginName); + addDocumentationLink(jsonSchemaNode, pluginName, pluginType); + value = jsonSchemaNode.toPrettyString(); + } catch (JsonProcessingException e) { + LOG.error("Encountered error retrieving JSON schema for {}", pluginName); + return Stream.empty(); + } + return Stream.of(Map.entry(entry.getKey(), value)); + }) + .filter(entry -> Objects.nonNull(entry.getValue())) + .collect(Collectors.toMap( + Map.Entry::getKey, + Map.Entry::getValue + )); + } + + private Map> scanForPluginConfigs(final Class pluginType) { + if (ConditionalRoute.class.equals(pluginType)) { + return Map.of(CONDITIONAL_ROUTE_PROCESSOR_NAME, ConditionalRoute.class); + } + return reflections.getTypesAnnotatedWith(DataPrepperPlugin.class).stream() + .map(clazz -> clazz.getAnnotation(DataPrepperPlugin.class)) + .filter(dataPrepperPlugin -> pluginType.equals(dataPrepperPlugin.pluginType())) + .collect(Collectors.toMap( + DataPrepperPlugin::name, + DataPrepperPlugin::pluginConfigurationType + )); + } + + private void addDocumentationLink(final ObjectNode jsonSchemaNode, + final String pluginName, + final Class pluginType) { + jsonSchemaNode.put(DOCUMENTATION_LINK_KEY, + String.format( + PLUGIN_DOCUMENTATION_URL_FORMAT, + siteUrl, + siteBaseUrl, + PLUGIN_TYPE_TO_URI_PARAMETER_MAP.get(pluginType), + pluginName)); + } + + private void addPluginName(final ObjectNode jsonSchemaNode, + final String pluginName) { + jsonSchemaNode.put(PLUGIN_NAME_KEY, pluginName); + } +} diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java new file mode 100644 index 0000000000..09c649cc4c --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java @@ -0,0 +1,31 @@ +package org.opensearch.dataprepper.schemas.module; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.PropertyNamingStrategies; +import com.github.victools.jsonschema.generator.MemberScope; +import com.github.victools.jsonschema.module.jackson.JacksonModule; +import com.github.victools.jsonschema.module.jackson.JacksonOption; + +public class CustomJacksonModule extends JacksonModule { + + public CustomJacksonModule() { + super(); + } + + public CustomJacksonModule(JacksonOption... options) { + super(options); + } + + @Override + protected String getPropertyNameOverrideBasedOnJsonPropertyAnnotation(MemberScope member) { + JsonProperty annotation = member.getAnnotationConsideringFieldAndGetter(JsonProperty.class); + if (annotation != null) { + String nameOverride = annotation.value(); + // check for invalid overrides + if (nameOverride != null && !nameOverride.isEmpty() && !nameOverride.equals(member.getDeclaredName())) { + return nameOverride; + } + } + return PropertyNamingStrategies.SNAKE_CASE.nameForField(null, null, member.getName()); + } +} diff --git a/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java new file mode 100644 index 0000000000..d5d172f8c0 --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java @@ -0,0 +1,60 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.annotation.JsonClassDescription; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.github.victools.jsonschema.generator.Module; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.schemas.module.CustomJacksonModule; + +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +class JsonSchemaConverterTest { + + public JsonSchemaConverter createObjectUnderTest(final List modules) { + return new JsonSchemaConverter(modules); + } + + @Test + void testConvertIntoJsonSchemaWithDefaultModules() throws JsonProcessingException { + final JsonSchemaConverter jsonSchemaConverter = createObjectUnderTest(Collections.emptyList()); + final ObjectNode jsonSchemaNode = jsonSchemaConverter.convertIntoJsonSchema( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, TestConfig.class); + assertThat(jsonSchemaNode, instanceOf(ObjectNode.class)); + } + + @Test + void testConvertIntoJsonSchemaWithCustomJacksonModule() throws JsonProcessingException { + final JsonSchemaConverter jsonSchemaConverter = createObjectUnderTest( + Collections.singletonList(new CustomJacksonModule())); + final ObjectNode jsonSchemaNode = jsonSchemaConverter.convertIntoJsonSchema( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, TestConfig.class); + assertThat(jsonSchemaNode, instanceOf(ObjectNode.class)); + assertThat(jsonSchemaNode.has("description"), is(true)); + final JsonNode propertiesNode = jsonSchemaNode.at("/properties"); + assertThat(propertiesNode, instanceOf(ObjectNode.class)); + assertThat(propertiesNode.has("test_attribute_with_getter"), is(true)); + assertThat(propertiesNode.has("custom_test_attribute"), is(true)); + } + + @JsonClassDescription("test config") + static class TestConfig { + private String testAttributeWithGetter; + + @JsonProperty("custom_test_attribute") + private String testAttributeWithJsonPropertyAnnotation; + + public String getTestAttributeWithGetter() { + return testAttributeWithGetter; + } + } +} \ No newline at end of file diff --git a/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java new file mode 100644 index 0000000000..71e9bf5faa --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java @@ -0,0 +1,80 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.github.victools.jsonschema.generator.Module; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import com.github.victools.jsonschema.module.jackson.JacksonModule; +import com.github.victools.jsonschema.module.jakarta.validation.JakartaValidationModule; +import com.github.victools.jsonschema.module.jakarta.validation.JakartaValidationOption; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.reflections.Reflections; +import org.reflections.scanners.Scanners; +import org.reflections.util.ClasspathHelper; +import org.reflections.util.ConfigurationBuilder; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Stream; + +import static com.github.victools.jsonschema.module.jackson.JacksonOption.RESPECT_JSONPROPERTY_REQUIRED; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.DOCUMENTATION_LINK_KEY; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.PLUGIN_NAME_KEY; + +class PluginConfigsJsonSchemaConverterIT { + static final String DEFAULT_PLUGINS_CLASSPATH = "org.opensearch.dataprepper.plugins"; + private static final String TEST_URL = String.format("https://%s/", UUID.randomUUID()); + private static final String TEST_BASE_URL = String.format("/%s", UUID.randomUUID()); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference<>() {}; + + private PluginConfigsJsonSchemaConverter objectUnderTest; + + @BeforeEach + void setUp() { + final List modules = List.of( + new JacksonModule(RESPECT_JSONPROPERTY_REQUIRED), + new JakartaValidationModule(JakartaValidationOption.NOT_NULLABLE_FIELD_IS_REQUIRED, + JakartaValidationOption.INCLUDE_PATTERN_EXPRESSIONS) + ); + final Reflections reflections = new Reflections(new ConfigurationBuilder() + .setUrls(ClasspathHelper.forPackage(DEFAULT_PLUGINS_CLASSPATH)) + .setScanners(Scanners.TypesAnnotated, Scanners.SubTypes)); + objectUnderTest = new PluginConfigsJsonSchemaConverter( + reflections, new JsonSchemaConverter(modules), TEST_URL, TEST_BASE_URL); + } + + @ParameterizedTest + @MethodSource("getValidPluginTypes") + void testConvertPluginConfigsIntoJsonSchemas(final Class pluginType) { + final Map result = objectUnderTest.convertPluginConfigsIntoJsonSchemas( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, pluginType); + assertThat(result.isEmpty(), is(false)); + result.values().forEach(schema -> { + final Map schemaMap; + try { + schemaMap = OBJECT_MAPPER.readValue(schema, MAP_TYPE_REFERENCE); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + assertThat(schemaMap, notNullValue()); + assertThat(schemaMap.containsKey(PLUGIN_NAME_KEY), is(true)); + assertThat(((String) schemaMap.get(DOCUMENTATION_LINK_KEY)).startsWith(TEST_URL + TEST_BASE_URL), + is(true)); + }); + } + + private static Stream getValidPluginTypes() { + return PluginConfigsJsonSchemaConverter.PLUGIN_TYPE_TO_URI_PARAMETER_MAP.keySet() + .stream().flatMap(clazz -> Stream.of(Arguments.of(clazz))); + } +} \ No newline at end of file diff --git a/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java new file mode 100644 index 0000000000..3d1c1b585a --- /dev/null +++ b/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java @@ -0,0 +1,110 @@ +package org.opensearch.dataprepper.schemas; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.github.victools.jsonschema.generator.OptionPreset; +import com.github.victools.jsonschema.generator.SchemaVersion; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.reflections.Reflections; + +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.DOCUMENTATION_LINK_KEY; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.PLUGIN_NAME_KEY; +import static org.opensearch.dataprepper.schemas.PluginConfigsJsonSchemaConverter.PLUGIN_TYPE_NAME_TO_CLASS_MAP; + +@ExtendWith(MockitoExtension.class) +class PluginConfigsJsonSchemaConverterTest { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference<>() {}; + + @Mock + private JsonSchemaConverter jsonSchemaConverter; + + @Mock + private Reflections reflections; + + @InjectMocks + private PluginConfigsJsonSchemaConverter objectUnderTest; + + @Test + void testValidPluginTypeNames() { + assertThat(PLUGIN_TYPE_NAME_TO_CLASS_MAP.keySet().containsAll(objectUnderTest.validPluginTypeNames()), + is(true)); + } + + @Test + void testPluginTypeNameToPluginTypeWithValidInput() { + objectUnderTest.validPluginTypeNames().forEach( + pluginType -> assertThat(objectUnderTest.pluginTypeNameToPluginType(pluginType), + equalTo(PLUGIN_TYPE_NAME_TO_CLASS_MAP.get(pluginType)))); + } + + @Test + void testPluginTypeNameToPluginTypeWithInValidInput() { + final String inValidPluginType = "invalid-" + UUID.randomUUID(); + assertThrows( + IllegalArgumentException.class, () -> objectUnderTest.pluginTypeNameToPluginType(inValidPluginType)); + } + + @Test + void testConvertPluginConfigsIntoJsonSchemasHappyPath() throws JsonProcessingException { + when(reflections.getTypesAnnotatedWith(eq(DataPrepperPlugin.class))).thenReturn(Set.of(TestPlugin.class)); + final ObjectNode objectNode = OBJECT_MAPPER.createObjectNode(); + when(jsonSchemaConverter.convertIntoJsonSchema( + any(SchemaVersion.class), any(OptionPreset.class), eq(TestPluginConfig.class))).thenReturn(objectNode); + final Map result = objectUnderTest.convertPluginConfigsIntoJsonSchemas( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, TestPluginType.class); + assertThat(result.size(), equalTo(1)); + final Map schemaMap = OBJECT_MAPPER.readValue(result.get("test_plugin"), MAP_TYPE_REFERENCE); + assertThat(schemaMap, notNullValue()); + assertThat(schemaMap.get(DOCUMENTATION_LINK_KEY), equalTo( + "{{site.url}}{{site.baseurl}}/data-prepper/pipelines/configuration/null/test_plugin/" + )); + assertThat(schemaMap.containsKey(PLUGIN_NAME_KEY), is(true)); + } + + @Test + void testConvertPluginConfigsIntoJsonSchemasWithError() throws JsonProcessingException { + when(reflections.getTypesAnnotatedWith(eq(DataPrepperPlugin.class))).thenReturn(Set.of(TestPlugin.class)); + final JsonProcessingException jsonProcessingException = mock(JsonProcessingException.class); + when(jsonSchemaConverter.convertIntoJsonSchema( + any(SchemaVersion.class), any(OptionPreset.class), eq(TestPluginConfig.class))).thenThrow( + jsonProcessingException); + final Map result = objectUnderTest.convertPluginConfigsIntoJsonSchemas( + SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, TestPluginType.class); + assertThat(result.isEmpty(), is(true)); + } + + @DataPrepperPlugin( + name = "test_plugin", pluginType = TestPluginType.class, pluginConfigurationType = TestPluginConfig.class) + static class TestPlugin { + + } + + static class TestPluginConfig { + + } + + static class TestPluginType { + + } +} \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index cb7e888c53..18ccd4dc7b 100644 --- a/settings.gradle +++ b/settings.gradle @@ -104,6 +104,7 @@ include 'data-prepper-core' include 'data-prepper-main' include 'data-prepper-pipeline-parser' include 'data-prepper-plugin-framework' +include 'data-prepper-plugin-schema-cli' include 'data-prepper-plugins:common' include 'data-prepper-plugins:armeria-common' include 'data-prepper-plugins:anomaly-detector-processor' From 04de9eb7a9e2086078521c921bbbbc5866bc9e3d Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Thu, 8 Aug 2024 16:51:12 -0500 Subject: [PATCH 101/159] Several improvements to RDS source (#4810) * Add schema manager to query database Signed-off-by: Hai Yan * Get real primary keys for export Signed-off-by: Hai Yan * Get binlog start position for stream Signed-off-by: Hai Yan * Refactor SnapshotStrategy to RdsApiStrategy Signed-off-by: Hai Yan * Update unit tests Signed-off-by: Hai Yan * address comments Signed-off-by: Hai Yan * Add retry to database queries Signed-off-by: Hai Yan * Handle describe exceptions Signed-off-by: Hai Yan * Address more comments Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- data-prepper-plugins/rds-source/build.gradle | 1 + .../plugins/source/rds/RdsService.java | 32 +++-- .../state/DataFileProgressState.java | 17 +++ .../state/ExportProgressState.java | 15 +++ .../source/rds/export/DataFileLoader.java | 7 +- .../source/rds/export/ExportScheduler.java | 25 ++-- .../source/rds/export/SnapshotManager.java | 5 +- .../ClusterApiStrategy.java} | 42 +++++-- .../InstanceApiStrategy.java} | 43 +++++-- .../source/rds/leader/LeaderScheduler.java | 34 +++++- .../RdsApiStrategy.java} | 14 ++- .../plugins/source/rds/model/DbMetadata.java | 44 +++++++ .../source/rds/schema/ConnectionManager.java | 47 ++++++++ .../source/rds/schema/SchemaManager.java | 79 +++++++++++++ .../rds/stream/BinlogClientFactory.java | 48 ++------ .../source/rds/stream/StreamWorker.java | 29 +++-- .../plugins/source/rds/RdsServiceTest.java | 33 +++++- .../rds/export/SnapshotManagerTest.java | 3 +- .../ClusterApiStrategyTest.java} | 40 ++++++- .../InstanceApiStrategyTest.java} | 42 ++++++- .../rds/leader/LeaderSchedulerTest.java | 10 +- .../rds/schema/ConnectionManagerTest.java | 80 +++++++++++++ .../source/rds/schema/SchemaManagerTest.java | 111 ++++++++++++++++++ .../rds/stream/BinlogClientFactoryTest.java | 30 ++--- .../source/rds/stream/StreamWorkerTest.java | 11 +- 25 files changed, 705 insertions(+), 137 deletions(-) rename data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/{export/ClusterSnapshotStrategy.java => leader/ClusterApiStrategy.java} (55%) rename data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/{export/InstanceSnapshotStrategy.java => leader/InstanceApiStrategy.java} (53%) rename data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/{export/SnapshotStrategy.java => leader/RdsApiStrategy.java} (61%) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/DbMetadata.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java rename data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/{export/ClusterSnapshotStrategyTest.java => leader/ClusterApiStrategyTest.java} (76%) rename data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/{export/InstanceSnapshotStrategyTest.java => leader/InstanceApiStrategyTest.java} (72%) create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java diff --git a/data-prepper-plugins/rds-source/build.gradle b/data-prepper-plugins/rds-source/build.gradle index 14c851f645..0203cccf10 100644 --- a/data-prepper-plugins/rds-source/build.gradle +++ b/data-prepper-plugins/rds-source/build.gradle @@ -22,6 +22,7 @@ dependencies { implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.zendesk:mysql-binlog-connector-java:0.29.2' + implementation 'com.mysql:mysql-connector-j:8.4.0' testImplementation project(path: ':data-prepper-test-common') testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index 73b71e0085..005fee88fd 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -13,14 +13,17 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; -import org.opensearch.dataprepper.plugins.source.rds.export.ClusterSnapshotStrategy; import org.opensearch.dataprepper.plugins.source.rds.export.DataFileScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportScheduler; import org.opensearch.dataprepper.plugins.source.rds.export.ExportTaskManager; -import org.opensearch.dataprepper.plugins.source.rds.export.InstanceSnapshotStrategy; import org.opensearch.dataprepper.plugins.source.rds.export.SnapshotManager; -import org.opensearch.dataprepper.plugins.source.rds.export.SnapshotStrategy; +import org.opensearch.dataprepper.plugins.source.rds.leader.ClusterApiStrategy; +import org.opensearch.dataprepper.plugins.source.rds.leader.InstanceApiStrategy; import org.opensearch.dataprepper.plugins.source.rds.leader.LeaderScheduler; +import org.opensearch.dataprepper.plugins.source.rds.leader.RdsApiStrategy; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; +import org.opensearch.dataprepper.plugins.source.rds.schema.ConnectionManager; +import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; import org.opensearch.dataprepper.plugins.source.rds.stream.BinlogClientFactory; import org.opensearch.dataprepper.plugins.source.rds.stream.StreamScheduler; import org.slf4j.Logger; @@ -77,13 +80,16 @@ public RdsService(final EnhancedSourceCoordinator sourceCoordinator, public void start(Buffer> buffer) { LOG.info("Start running RDS service"); final List runnableList = new ArrayList<>(); - leaderScheduler = new LeaderScheduler(sourceCoordinator, sourceConfig); + + final RdsApiStrategy rdsApiStrategy = sourceConfig.isCluster() ? + new ClusterApiStrategy(rdsClient) : new InstanceApiStrategy(rdsClient); + final DbMetadata dbMetadata = rdsApiStrategy.describeDb(sourceConfig.getDbIdentifier()); + leaderScheduler = new LeaderScheduler( + sourceCoordinator, sourceConfig, getSchemaManager(sourceConfig, dbMetadata), dbMetadata); runnableList.add(leaderScheduler); if (sourceConfig.isExportEnabled()) { - final SnapshotStrategy snapshotStrategy = sourceConfig.isCluster() ? - new ClusterSnapshotStrategy(rdsClient) : new InstanceSnapshotStrategy(rdsClient); - final SnapshotManager snapshotManager = new SnapshotManager(snapshotStrategy); + final SnapshotManager snapshotManager = new SnapshotManager(rdsApiStrategy); final ExportTaskManager exportTaskManager = new ExportTaskManager(rdsClient); exportScheduler = new ExportScheduler( sourceCoordinator, snapshotManager, exportTaskManager, s3Client, pluginMetrics); @@ -94,7 +100,7 @@ public void start(Buffer> buffer) { } if (sourceConfig.isStreamEnabled()) { - BinaryLogClient binaryLogClient = new BinlogClientFactory(sourceConfig, rdsClient).create(); + BinaryLogClient binaryLogClient = new BinlogClientFactory(sourceConfig, rdsClient, dbMetadata).create(); if (sourceConfig.getTlsConfig() == null || !sourceConfig.getTlsConfig().isInsecure()) { binaryLogClient.setSSLMode(SSLMode.REQUIRED); } else { @@ -128,4 +134,14 @@ public void shutdown() { executor.shutdownNow(); } } + + private SchemaManager getSchemaManager(final RdsSourceConfig sourceConfig, final DbMetadata dbMetadata) { + final ConnectionManager connectionManager = new ConnectionManager( + dbMetadata.getHostName(), + dbMetadata.getPort(), + sourceConfig.getAuthenticationConfig().getUsername(), + sourceConfig.getAuthenticationConfig().getPassword(), + !sourceConfig.getTlsConfig().isInsecure()); + return new SchemaManager(connectionManager); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java index 4f94f57d15..9fee60105f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java @@ -7,6 +7,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; +import java.util.List; +import java.util.Map; + public class DataFileProgressState { @JsonProperty("isLoaded") @@ -21,6 +24,12 @@ public class DataFileProgressState { @JsonProperty("sourceTable") private String sourceTable; + /** + * Map of table name to primary keys + */ + @JsonProperty("primaryKeyMap") + private Map> primaryKeyMap; + @JsonProperty("snapshotTime") private long snapshotTime; @@ -63,4 +72,12 @@ public long getSnapshotTime() { public void setSnapshotTime(long snapshotTime) { this.snapshotTime = snapshotTime; } + + public Map> getPrimaryKeyMap() { + return primaryKeyMap; + } + + public void setPrimaryKeyMap(Map> primaryKeyMap) { + this.primaryKeyMap = primaryKeyMap; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java index b224f63a97..e4bbeb4c98 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java @@ -8,6 +8,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import java.util.List; +import java.util.Map; /** * Progress state for an EXPORT partition @@ -32,6 +33,12 @@ public class ExportProgressState { @JsonProperty("tables") private List tables; + /** + * Map of table name to primary keys + */ + @JsonProperty("primaryKeyMap") + private Map> primaryKeyMap; + @JsonProperty("kmsKeyId") private String kmsKeyId; @@ -89,6 +96,14 @@ public void setTables(List tables) { this.tables = tables; } + public Map> getPrimaryKeyMap() { + return primaryKeyMap; + } + + public void setPrimaryKeyMap(Map> primaryKeyMap) { + this.primaryKeyMap = primaryKeyMap; + } + public String getKmsKeyId() { return kmsKeyId; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java index 42a5b3a0d5..16bd559848 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -93,8 +93,8 @@ public void run() { DataFileProgressState progressState = dataFilePartition.getProgressState().get(); - // TODO: primary key to be obtained by querying database schema - final String primaryKeyName = "id"; + final String fullTableName = progressState.getSourceDatabase() + "." + progressState.getSourceTable(); + final List primaryKeys = progressState.getPrimaryKeyMap().getOrDefault(fullTableName, List.of()); final long snapshotTime = progressState.getSnapshotTime(); final long eventVersionNumber = snapshotTime - VERSION_OVERLAP_TIME_FOR_EXPORT.toMillis(); @@ -103,13 +103,14 @@ public void run() { record, progressState.getSourceDatabase(), progressState.getSourceTable(), - List.of(primaryKeyName), + primaryKeys, snapshotTime, eventVersionNumber)); bufferAccumulator.add(transformedRecord); eventCount.getAndIncrement(); bytesProcessedSummary.record(bytes); } catch (Exception e) { + LOG.error("Failed to process record from object s3://{}/{}", bucket, objectKey, e); throw new RuntimeException(e); } }); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index 343ade8b85..ce534747e1 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -30,6 +30,7 @@ import java.time.Instant; import java.util.ArrayList; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -189,12 +190,15 @@ private SnapshotInfo checkSnapshotStatus(String snapshotId, Duration timeout) { LOG.debug("Start checking status of snapshot {}", snapshotId); while (Instant.now().isBefore(endTime)) { SnapshotInfo snapshotInfo = snapshotManager.checkSnapshotStatus(snapshotId); - String status = snapshotInfo.getStatus(); - // Valid snapshot statuses are: available, copying, creating - // The status should never be "copying" here - if (SnapshotStatus.AVAILABLE.getStatusName().equals(status)) { - LOG.info("Snapshot {} is available.", snapshotId); - return snapshotInfo; + + if (snapshotInfo != null) { + String status = snapshotInfo.getStatus(); + // Valid snapshot statuses are: available, copying, creating + // The status should never be "copying" here + if (SnapshotStatus.AVAILABLE.getStatusName().equals(status)) { + LOG.info("Snapshot {} is available.", snapshotId); + return snapshotInfo; + } } LOG.debug("Snapshot {} is still creating. Wait and check later", snapshotId); @@ -272,7 +276,7 @@ private BiConsumer completeExport(ExportPartition exportParti // Create data file partitions for processing S3 files List dataFileObjectKeys = getDataFileObjectKeys(bucket, prefix, exportTaskId); - createDataFilePartitions(bucket, exportTaskId, dataFileObjectKeys, snapshotTime); + createDataFilePartitions(bucket, exportTaskId, dataFileObjectKeys, snapshotTime, state.getPrimaryKeyMap()); completeExportPartition(exportPartition); } @@ -301,7 +305,11 @@ private List getDataFileObjectKeys(String bucket, String prefix, String return objectKeys; } - private void createDataFilePartitions(String bucket, String exportTaskId, List dataFileObjectKeys, long snapshotTime) { + private void createDataFilePartitions(String bucket, + String exportTaskId, + List dataFileObjectKeys, + long snapshotTime, + Map> primaryKeyMap) { LOG.info("Total of {} data files generated for export {}", dataFileObjectKeys.size(), exportTaskId); AtomicInteger totalFiles = new AtomicInteger(); for (final String objectKey : dataFileObjectKeys) { @@ -313,6 +321,7 @@ private void createDataFilePartitions(String bucket, String exportTaskId, List> getPrimaryKeyMap() { + return sourceConfig.getTableNames().stream() + .collect(Collectors.toMap( + fullTableName -> fullTableName, + fullTableName -> schemaManager.getPrimaryKeys(fullTableName.split("\\.")[0], fullTableName.split("\\.")[1]) + )); + } + private void createStreamPartition(RdsSourceConfig sourceConfig) { final StreamProgressState progressState = new StreamProgressState(); progressState.setWaitForExport(sourceConfig.isExportEnabled()); + getCurrentBinlogPosition().ifPresent(progressState::setStartPosition); StreamPartition streamPartition = new StreamPartition(sourceConfig.getDbIdentifier(), progressState); sourceCoordinator.createPartition(streamPartition); } + + private Optional getCurrentBinlogPosition() { + return schemaManager.getCurrentBinaryLogPosition(); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/RdsApiStrategy.java similarity index 61% rename from data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java rename to data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/RdsApiStrategy.java index af2f0507f7..5b8fe157c1 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotStrategy.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/RdsApiStrategy.java @@ -3,14 +3,22 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.rds.export; +package org.opensearch.dataprepper.plugins.source.rds.leader; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; /** - * Provides a strategy for creating and describing RDS snapshots. + * Provides a strategy for running RDS APIs. */ -public interface SnapshotStrategy { +public interface RdsApiStrategy { + /** + * Describes an RDS instance or cluster. + * @param dbIdentifier The identifier of the RDS instance or cluster to describe + * @return An {@link DbMetadata} object describing the instance or cluster + */ + DbMetadata describeDb(String dbIdentifier); + /** * Creates a snapshot of an RDS instance or cluster. * diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/DbMetadata.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/DbMetadata.java new file mode 100644 index 0000000000..02918e6da4 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/DbMetadata.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.model; + +import java.util.Map; + +public class DbMetadata { + + private static final String DB_IDENTIFIER_KEY = "dbIdentifier"; + private static final String HOST_NAME_KEY = "hostName"; + private static final String PORT_KEY = "port"; + private final String dbIdentifier; + private final String hostName; + private final int port; + + public DbMetadata(final String dbIdentifier, final String hostName, final int port) { + this.dbIdentifier = dbIdentifier; + this.hostName = hostName; + this.port = port; + } + + public String getDbIdentifier() { + return dbIdentifier; + } + + public String getHostName() { + return hostName; + } + + public int getPort() { + return port; + } + + public Map toMap() { + return Map.of( + DB_IDENTIFIER_KEY, dbIdentifier, + HOST_NAME_KEY, hostName, + PORT_KEY, port + ); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java new file mode 100644 index 0000000000..90a93428c0 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManager.java @@ -0,0 +1,47 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.SQLException; +import java.util.Properties; + +public class ConnectionManager { + static final String JDBC_URL_FORMAT = "jdbc:mysql://%s:%d"; + private final String hostName; + private final int port; + private final String username; + private final String password; + private final boolean requireSSL; + + public ConnectionManager(String hostName, int port, String username, String password, boolean requireSSL) { + this.hostName = hostName; + this.port = port; + this.username = username; + this.password = password; + this.requireSSL = requireSSL; + } + + public Connection getConnection() throws SQLException { + final Properties props = new Properties(); + props.setProperty("user", username); + props.setProperty("password", password); + if (requireSSL) { + props.setProperty("useSSL", "true"); + props.setProperty("requireSSL", "true"); + } else { + props.setProperty("useSSL", "false"); + } + final String jdbcUrl = String.format(JDBC_URL_FORMAT, hostName, port); + return doGetConnection(jdbcUrl, props); + } + + // VisibleForTesting + Connection doGetConnection(String jdbcUrl, Properties props) throws SQLException { + return DriverManager.getConnection(jdbcUrl, props); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java new file mode 100644 index 0000000000..b7623fc85c --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManager.java @@ -0,0 +1,79 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.Statement; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; + +public class SchemaManager { + private static final Logger LOG = LoggerFactory.getLogger(SchemaManager.class); + static final String COLUMN_NAME = "COLUMN_NAME"; + static final String BINLOG_STATUS_QUERY = "SHOW MASTER STATUS"; + static final String BINLOG_FILE = "File"; + static final String BINLOG_POSITION = "Position"; + static final int NUM_OF_RETRIES = 3; + static final int BACKOFF_IN_MILLIS = 500; + private final ConnectionManager connectionManager; + + public SchemaManager(ConnectionManager connectionManager) { + this.connectionManager = connectionManager; + } + + public List getPrimaryKeys(final String database, final String table) { + int retry = 0; + while (retry <= NUM_OF_RETRIES) { + final List primaryKeys = new ArrayList<>(); + try (final Connection connection = connectionManager.getConnection()) { + final ResultSet rs = connection.getMetaData().getPrimaryKeys(database, null, table); + while (rs.next()) { + primaryKeys.add(rs.getString(COLUMN_NAME)); + } + return primaryKeys; + } catch (Exception e) { + LOG.error("Failed to get primary keys for table {}, retrying", table, e); + } + applyBackoff(); + retry++; + } + LOG.warn("Failed to get primary keys for table {}", table); + return List.of(); + } + + public Optional getCurrentBinaryLogPosition() { + int retry = 0; + while (retry <= NUM_OF_RETRIES) { + try (final Connection connection = connectionManager.getConnection()) { + final Statement statement = connection.createStatement(); + final ResultSet rs = statement.executeQuery(BINLOG_STATUS_QUERY); + if (rs.next()) { + return Optional.of(new BinlogCoordinate(rs.getString(BINLOG_FILE), rs.getLong(BINLOG_POSITION))); + } + } catch (Exception e) { + LOG.error("Failed to get current binary log position, retrying", e); + } + applyBackoff(); + retry++; + } + LOG.warn("Failed to get current binary log position"); + return Optional.empty(); + } + + private void applyBackoff() { + try { + Thread.sleep(BACKOFF_IN_MILLIS); + } catch (final InterruptedException e){ + Thread.currentThread().interrupt(); + } + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java index 7c28eea8db..24ed45d7c2 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactory.java @@ -7,60 +7,28 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import software.amazon.awssdk.services.rds.RdsClient; -import software.amazon.awssdk.services.rds.model.DBCluster; -import software.amazon.awssdk.services.rds.model.DBInstance; -import software.amazon.awssdk.services.rds.model.DescribeDbClustersRequest; -import software.amazon.awssdk.services.rds.model.DescribeDbClustersResponse; -import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; -import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; public class BinlogClientFactory { private final RdsSourceConfig sourceConfig; - private final RdsClient rdsClient; + private final DbMetadata dbMetadata; - public BinlogClientFactory(final RdsSourceConfig sourceConfig, final RdsClient rdsClient) { + public BinlogClientFactory(final RdsSourceConfig sourceConfig, + final RdsClient rdsClient, + final DbMetadata dbMetadata) { this.sourceConfig = sourceConfig; this.rdsClient = rdsClient; + this.dbMetadata = dbMetadata; } public BinaryLogClient create() { - // TODO: refactor SnapshotStrategy to RdsApiStrategy to accommodate more APIs for clusters and instances - String hostName; - int port; - if (sourceConfig.isCluster()) { - DBCluster dbCluster = describeDbCluster(sourceConfig.getDbIdentifier()); - hostName = dbCluster.endpoint(); - port = dbCluster.port(); - } else { - DBInstance dbInstance = describeDbInstance(sourceConfig.getDbIdentifier()); - hostName = dbInstance.endpoint().address(); - port = dbInstance.endpoint().port(); - } return new BinaryLogClient( - hostName, - port, + dbMetadata.getHostName(), + dbMetadata.getPort(), sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword()); } - - private DBInstance describeDbInstance(final String dbInstanceIdentifier) { - DescribeDbInstancesRequest request = DescribeDbInstancesRequest.builder() - .dbInstanceIdentifier(dbInstanceIdentifier) - .build(); - - DescribeDbInstancesResponse response = rdsClient.describeDBInstances(request); - return response.dbInstances().get(0); - } - - private DBCluster describeDbCluster(final String dbClusterIdentifier) { - DescribeDbClustersRequest request = DescribeDbClustersRequest.builder() - .dbClusterIdentifier(dbClusterIdentifier) - .build(); - - DescribeDbClustersResponse response = rdsClient.describeDBClusters(request); - return response.dbClusters().get(0); - } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java index e1e169dea5..303ecc2a53 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java @@ -41,29 +41,19 @@ public static StreamWorker create(final EnhancedSourceCoordinator sourceCoordina } public void processStream(final StreamPartition streamPartition) { - // get current binlog position - BinlogCoordinate currentBinlogCoords = streamPartition.getProgressState().get().getCurrentPosition(); - - // set start of binlog stream to current position if exists - if (currentBinlogCoords != null) { - final String binlogFilename = currentBinlogCoords.getBinlogFilename(); - final long binlogPosition = currentBinlogCoords.getBinlogPosition(); - LOG.debug("Will start binlog stream from binlog file {} and position {}.", binlogFilename, binlogPosition); - binaryLogClient.setBinlogFilename(binlogFilename); - binaryLogClient.setBinlogPosition(binlogPosition); - } - while (shouldWaitForExport(streamPartition) && !Thread.currentThread().isInterrupted()) { LOG.info("Initial load not completed yet for {}, waiting...", streamPartition.getPartitionKey()); try { Thread.sleep(DEFAULT_EXPORT_COMPLETE_WAIT_INTERVAL_MILLIS); } catch (final InterruptedException ex) { - LOG.info("The StreamScheduler was interrupted while waiting to retry, stopping processing"); + LOG.info("The Stream Scheduler was interrupted while waiting to retry, stopping processing"); Thread.currentThread().interrupt(); break; } } + setStartBinlogPosition(streamPartition); + try { LOG.info("Connect to database to read change events."); binaryLogClient.connect(); @@ -92,4 +82,17 @@ private boolean isExportDone(StreamPartition streamPartition) { Optional globalStatePartition = sourceCoordinator.getPartition("stream-for-" + dbIdentifier); return globalStatePartition.isPresent(); } + + private void setStartBinlogPosition(final StreamPartition streamPartition) { + final BinlogCoordinate startBinlogPosition = streamPartition.getProgressState().get().getStartPosition(); + + // set start of binlog stream to current position if exists + if (startBinlogPosition != null) { + final String binlogFilename = startBinlogPosition.getBinlogFilename(); + final long binlogPosition = startBinlogPosition.getBinlogPosition(); + LOG.debug("Will start binlog stream from binlog file {} and position {}.", binlogFilename, binlogPosition); + binaryLogClient.setBinlogFilename(binlogFilename); + binaryLogClient.setBinlogPosition(binlogPosition); + } + } } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java index 275d5c0c38..e0b69746c2 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java @@ -76,9 +76,23 @@ void setUp() { @Test void test_normal_service_start_when_export_is_enabled() { - RdsService rdsService = createObjectUnderTest(); + final String dbIdentifier = UUID.randomUUID().toString(); + final String host = UUID.randomUUID().toString(); + final int port = 3306; + final DescribeDbInstancesResponse describeDbInstancesResponse = DescribeDbInstancesResponse.builder() + .dbInstances(DBInstance.builder() + .endpoint(Endpoint.builder() + .address(host) + .port(port) + .build()) + .build()) + .build(); when(sourceConfig.isExportEnabled()).thenReturn(true); when(sourceConfig.isStreamEnabled()).thenReturn(false); + when(sourceConfig.getDbIdentifier()).thenReturn(dbIdentifier); + when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); + + final RdsService rdsService = createObjectUnderTest(); try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); rdsService.start(buffer); @@ -92,7 +106,6 @@ void test_normal_service_start_when_export_is_enabled() { @Test void test_normal_service_start_when_stream_is_enabled() { - RdsService rdsService = createObjectUnderTest(); when(sourceConfig.isStreamEnabled()).thenReturn(true); when(sourceConfig.isExportEnabled()).thenReturn(false); final String dbIdentifier = UUID.randomUUID().toString(); @@ -107,12 +120,14 @@ void test_normal_service_start_when_stream_is_enabled() { .endpoint(hostEndpoint) .build()); when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); + final RdsSourceConfig.AuthenticationConfig authConfig = mock(RdsSourceConfig.AuthenticationConfig.class); when(authConfig.getUsername()).thenReturn(UUID.randomUUID().toString()); when(authConfig.getPassword()).thenReturn(UUID.randomUUID().toString()); when(sourceConfig.getAuthenticationConfig()).thenReturn(authConfig); when(sourceConfig.getTlsConfig()).thenReturn(mock(TlsConfig.class)); + final RdsService rdsService = createObjectUnderTest(); try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); rdsService.start(buffer); @@ -126,6 +141,20 @@ void test_normal_service_start_when_stream_is_enabled() { @Test void test_service_shutdown_calls_executor_shutdownNow() { + final String dbIdentifier = UUID.randomUUID().toString(); + final String host = UUID.randomUUID().toString(); + final int port = 3306; + final DescribeDbInstancesResponse describeDbInstancesResponse = DescribeDbInstancesResponse.builder() + .dbInstances(DBInstance.builder() + .endpoint(Endpoint.builder() + .address(host) + .port(port) + .build()) + .build()) + .build(); + when(sourceConfig.getDbIdentifier()).thenReturn(dbIdentifier); + when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); + RdsService rdsService = createObjectUnderTest(); try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt())).thenReturn(executor); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java index d88531bb65..d517a20700 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/SnapshotManagerTest.java @@ -10,6 +10,7 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.leader.RdsApiStrategy; import java.util.UUID; @@ -21,7 +22,7 @@ class SnapshotManagerTest { @Mock - private SnapshotStrategy snapshotStrategy; + private RdsApiStrategy snapshotStrategy; private SnapshotManager snapshotManager; diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/ClusterApiStrategyTest.java similarity index 76% rename from data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java rename to data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/ClusterApiStrategyTest.java index a0965efad2..a64604e035 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ClusterSnapshotStrategyTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/ClusterApiStrategyTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.rds.export; +package org.opensearch.dataprepper.plugins.source.rds.leader; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -11,16 +11,21 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; import software.amazon.awssdk.services.rds.RdsClient; import software.amazon.awssdk.services.rds.model.CreateDbClusterSnapshotRequest; import software.amazon.awssdk.services.rds.model.CreateDbClusterSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DBCluster; import software.amazon.awssdk.services.rds.model.DBClusterSnapshot; import software.amazon.awssdk.services.rds.model.DescribeDbClusterSnapshotsRequest; import software.amazon.awssdk.services.rds.model.DescribeDbClusterSnapshotsResponse; +import software.amazon.awssdk.services.rds.model.DescribeDbClustersRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbClustersResponse; import java.time.Instant; import java.util.List; +import java.util.Random; import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; @@ -32,18 +37,43 @@ import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -class ClusterSnapshotStrategyTest { +class ClusterApiStrategyTest { @Mock private RdsClient rdsClient; - private ClusterSnapshotStrategy objectUnderTest; + private ClusterApiStrategy objectUnderTest; + + private final Random random = new Random(); @BeforeEach void setUp() { objectUnderTest = createObjectUnderTest(); } + @Test + void test_describeDb_returns_correct_results() { + final String dbClusterId = UUID.randomUUID().toString(); + final String host = UUID.randomUUID().toString(); + final int port = random.nextInt(); + final DescribeDbClustersRequest describeDbClustersRequest = DescribeDbClustersRequest.builder() + .dbClusterIdentifier(dbClusterId) + .build(); + final DescribeDbClustersResponse describeDbClustersResponse = DescribeDbClustersResponse.builder() + .dbClusters(DBCluster.builder() + .endpoint(host) + .port(port) + .build()) + .build(); + when(rdsClient.describeDBClusters(describeDbClustersRequest)).thenReturn(describeDbClustersResponse); + + DbMetadata dbMetadata = objectUnderTest.describeDb(dbClusterId); + + assertThat(dbMetadata.getDbIdentifier(), equalTo(dbClusterId)); + assertThat(dbMetadata.getHostName(), equalTo(host)); + assertThat(dbMetadata.getPort(), equalTo(port)); + } + @Test void test_create_snapshot_with_success() { final String dbInstanceId = UUID.randomUUID().toString(); @@ -112,7 +142,7 @@ void test_check_snapshot_status_returns_correct_result() { assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); } - private ClusterSnapshotStrategy createObjectUnderTest() { - return new ClusterSnapshotStrategy(rdsClient); + private ClusterApiStrategy createObjectUnderTest() { + return new ClusterApiStrategy(rdsClient); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/InstanceApiStrategyTest.java similarity index 72% rename from data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java rename to data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/InstanceApiStrategyTest.java index c9a87ea3a2..8699c4b4be 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/InstanceSnapshotStrategyTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/InstanceApiStrategyTest.java @@ -3,7 +3,7 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.source.rds.export; +package org.opensearch.dataprepper.plugins.source.rds.leader; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -11,16 +11,22 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import org.opensearch.dataprepper.plugins.source.rds.model.SnapshotInfo; import software.amazon.awssdk.services.rds.RdsClient; import software.amazon.awssdk.services.rds.model.CreateDbSnapshotRequest; import software.amazon.awssdk.services.rds.model.CreateDbSnapshotResponse; +import software.amazon.awssdk.services.rds.model.DBInstance; import software.amazon.awssdk.services.rds.model.DBSnapshot; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; +import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsRequest; import software.amazon.awssdk.services.rds.model.DescribeDbSnapshotsResponse; +import software.amazon.awssdk.services.rds.model.Endpoint; import java.time.Instant; import java.util.List; +import java.util.Random; import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; @@ -32,18 +38,44 @@ import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -class InstanceSnapshotStrategyTest { +class InstanceApiStrategyTest { @Mock private RdsClient rdsClient; - private InstanceSnapshotStrategy objectUnderTest; + private InstanceApiStrategy objectUnderTest; + private final Random random = new Random(); @BeforeEach void setUp() { objectUnderTest = createObjectUnderTest(); } + @Test + void test_describeDb_returns_correct_results() { + final String dbInstanceId = UUID.randomUUID().toString(); + final String host = UUID.randomUUID().toString(); + final int port = random.nextInt(); + final DescribeDbInstancesRequest describeDbInstancesRequest = DescribeDbInstancesRequest.builder() + .dbInstanceIdentifier(dbInstanceId) + .build(); + final DescribeDbInstancesResponse describeDbInstancesResponse = DescribeDbInstancesResponse.builder() + .dbInstances(DBInstance.builder() + .endpoint(Endpoint.builder() + .address(host) + .port(port) + .build()) + .build()) + .build(); + when(rdsClient.describeDBInstances(describeDbInstancesRequest)).thenReturn(describeDbInstancesResponse); + + DbMetadata dbMetadata = objectUnderTest.describeDb(dbInstanceId); + + assertThat(dbMetadata.getDbIdentifier(), equalTo(dbInstanceId)); + assertThat(dbMetadata.getHostName(), equalTo(host)); + assertThat(dbMetadata.getPort(), equalTo(port)); + } + @Test void test_create_snapshot_with_success() { final String dbInstanceId = UUID.randomUUID().toString(); @@ -112,7 +144,7 @@ void test_check_snapshot_status_returns_correct_result() { assertThat(snapshotInfo.getCreateTime(), equalTo(createTime)); } - private InstanceSnapshotStrategy createObjectUnderTest() { - return new InstanceSnapshotStrategy(rdsClient); + private InstanceApiStrategy createObjectUnderTest() { + return new InstanceApiStrategy(rdsClient); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java index e844cc0ff4..e4b9cb1304 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java @@ -19,6 +19,8 @@ import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.LeaderPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.LeaderProgressState; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; +import org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager; import java.time.Duration; import java.util.Optional; @@ -46,6 +48,12 @@ class LeaderSchedulerTest { @Mock(answer = Answers.RETURNS_DEFAULTS) private RdsSourceConfig sourceConfig; + @Mock + private SchemaManager schemaManager; + + @Mock + private DbMetadata dbMetadata; + @Mock private LeaderPartition leaderPartition; @@ -130,6 +138,6 @@ void test_shutDown() { } private LeaderScheduler createObjectUnderTest() { - return new LeaderScheduler(sourceCoordinator, sourceConfig); + return new LeaderScheduler(sourceCoordinator, sourceConfig, schemaManager, dbMetadata); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerTest.java new file mode 100644 index 0000000000..7705ac967b --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/ConnectionManagerTest.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; + +import java.sql.Connection; +import java.sql.SQLException; +import java.util.Properties; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + + +class ConnectionManagerTest { + + private String hostName; + private int port; + private String username; + private String password; + private boolean requireSSL; + private final Random random = new Random(); + + @BeforeEach + void setUp() { + hostName = UUID.randomUUID().toString(); + port = random.nextInt(); + username = UUID.randomUUID().toString(); + password = UUID.randomUUID().toString(); + } + + @Test + void test_getConnection_when_requireSSL_is_true() throws SQLException { + requireSSL = true; + final ConnectionManager connectionManager = spy(createObjectUnderTest()); + final ArgumentCaptor jdbcUrlArgumentCaptor = ArgumentCaptor.forClass(String.class); + final ArgumentCaptor propertiesArgumentCaptor = ArgumentCaptor.forClass(Properties.class); + doReturn(mock(Connection.class)).when(connectionManager).doGetConnection(jdbcUrlArgumentCaptor.capture(), propertiesArgumentCaptor.capture()); + + connectionManager.getConnection(); + + assertThat(jdbcUrlArgumentCaptor.getValue(), is(String.format(ConnectionManager.JDBC_URL_FORMAT, hostName, port))); + final Properties properties = propertiesArgumentCaptor.getValue(); + assertThat(properties.getProperty("user"), is(username)); + assertThat(properties.getProperty("password"), is(password)); + assertThat(properties.getProperty("useSSL"), is("true")); + assertThat(properties.getProperty("requireSSL"), is("true")); + } + + @Test + void test_getConnection_when_requireSSL_is_false() throws SQLException { + requireSSL = false; + final ConnectionManager connectionManager = spy(createObjectUnderTest()); + final ArgumentCaptor jdbcUrlArgumentCaptor = ArgumentCaptor.forClass(String.class); + final ArgumentCaptor propertiesArgumentCaptor = ArgumentCaptor.forClass(Properties.class); + doReturn(mock(Connection.class)).when(connectionManager).doGetConnection(jdbcUrlArgumentCaptor.capture(), propertiesArgumentCaptor.capture()); + + connectionManager.getConnection(); + + assertThat(jdbcUrlArgumentCaptor.getValue(), is(String.format(ConnectionManager.JDBC_URL_FORMAT, hostName, port))); + final Properties properties = propertiesArgumentCaptor.getValue(); + assertThat(properties.getProperty("user"), is(username)); + assertThat(properties.getProperty("password"), is(password)); + assertThat(properties.getProperty("useSSL"), is("false")); + } + + private ConnectionManager createObjectUnderTest() { + return new ConnectionManager(hostName, port, username, password, requireSSL); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java new file mode 100644 index 0000000000..2661fd2bf0 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/schema/SchemaManagerTest.java @@ -0,0 +1,111 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.schema; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Answers; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.util.List; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.BINLOG_FILE; +import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.BINLOG_POSITION; +import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.BINLOG_STATUS_QUERY; +import static org.opensearch.dataprepper.plugins.source.rds.schema.SchemaManager.COLUMN_NAME; + +@ExtendWith(MockitoExtension.class) +class SchemaManagerTest { + + @Mock + private ConnectionManager connectionManager; + + @Mock(answer = Answers.RETURNS_DEEP_STUBS) + private Connection connection; + + @Mock + private ResultSet resultSet; + + private SchemaManager schemaManager; + + @BeforeEach + void setUp() { + schemaManager = createObjectUnderTest(); + } + + @Test + void test_getPrimaryKeys_returns_primary_keys() throws SQLException { + final String databaseName = UUID.randomUUID().toString(); + final String tableName = UUID.randomUUID().toString(); + final String primaryKey = UUID.randomUUID().toString(); + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.getMetaData().getPrimaryKeys(databaseName, null, tableName)).thenReturn(resultSet); + when(resultSet.next()).thenReturn(true, false); + when(resultSet.getString(COLUMN_NAME)).thenReturn(primaryKey); + + final List primaryKeys = schemaManager.getPrimaryKeys(databaseName, tableName); + + assertThat(primaryKeys, contains(primaryKey)); + } + + @Test + void test_getPrimaryKeys_throws_exception_then_returns_empty_list() throws SQLException { + final String databaseName = UUID.randomUUID().toString(); + final String tableName = UUID.randomUUID().toString(); + when(connectionManager.getConnection()).thenThrow(SQLException.class); + + final List primaryKeys = schemaManager.getPrimaryKeys(databaseName, tableName); + + assertThat(primaryKeys, empty()); + } + + @Test + void test_getCurrentBinaryLogPosition_returns_binlog_coords() throws SQLException { + final Statement statement = mock(Statement.class); + final String binlogFile = UUID.randomUUID().toString(); + final long binlogPosition = 123L; + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.createStatement()).thenReturn(statement); + when(statement.executeQuery(BINLOG_STATUS_QUERY)).thenReturn(resultSet); + when(resultSet.next()).thenReturn(true); + when(resultSet.getString(BINLOG_FILE)).thenReturn(binlogFile); + when(resultSet.getLong(BINLOG_POSITION)).thenReturn(binlogPosition); + + final Optional binlogCoordinate = schemaManager.getCurrentBinaryLogPosition(); + + assertThat(binlogCoordinate.isPresent(), is(true)); + assertThat(binlogCoordinate.get().getBinlogFilename(), is(binlogFile)); + assertThat(binlogCoordinate.get().getBinlogPosition(), is(binlogPosition)); + } + + @Test + void test_getCurrentBinaryLogPosition_throws_exception_then_returns_empty() throws SQLException { + when(connectionManager.getConnection()).thenThrow(SQLException.class); + + final Optional binlogCoordinate = schemaManager.getCurrentBinaryLogPosition(); + + assertThat(binlogCoordinate.isPresent(), is(false)); + } + + private SchemaManager createObjectUnderTest() { + return new SchemaManager(connectionManager); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java index 52ba312b5a..326ea4eb67 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogClientFactoryTest.java @@ -11,18 +11,13 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import org.opensearch.dataprepper.plugins.source.rds.model.DbMetadata; import software.amazon.awssdk.services.rds.RdsClient; -import software.amazon.awssdk.services.rds.model.DBInstance; -import software.amazon.awssdk.services.rds.model.DescribeDbInstancesRequest; -import software.amazon.awssdk.services.rds.model.DescribeDbInstancesResponse; -import java.util.List; import java.util.Random; -import java.util.UUID; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -34,6 +29,9 @@ class BinlogClientFactoryTest { @Mock private RdsClient rdsClient; + @Mock + private DbMetadata dbMetadata; + private BinlogClientFactory binlogClientFactory; private Random random; @@ -45,22 +43,18 @@ void setUp() { @Test void test_create() { - DescribeDbInstancesResponse describeDbInstancesResponse = mock(DescribeDbInstancesResponse.class); - DBInstance dbInstance = mock(DBInstance.class, RETURNS_DEEP_STUBS); - final String address = UUID.randomUUID().toString(); - final Integer port = random.nextInt(); - when(dbInstance.endpoint().address()).thenReturn(address); - when(dbInstance.endpoint().port()).thenReturn(port); - when(describeDbInstancesResponse.dbInstances()).thenReturn(List.of(dbInstance)); - when(sourceConfig.getDbIdentifier()).thenReturn(UUID.randomUUID().toString()); - when(rdsClient.describeDBInstances(any(DescribeDbInstancesRequest.class))).thenReturn(describeDbInstancesResponse); - RdsSourceConfig.AuthenticationConfig authenticationConfig = mock(RdsSourceConfig.AuthenticationConfig.class); + final RdsSourceConfig.AuthenticationConfig authenticationConfig = mock(RdsSourceConfig.AuthenticationConfig.class); when(sourceConfig.getAuthenticationConfig()).thenReturn(authenticationConfig); binlogClientFactory.create(); + + verify(dbMetadata).getHostName(); + verify(dbMetadata).getPort(); + verify(authenticationConfig).getUsername(); + verify(authenticationConfig).getPassword(); } private BinlogClientFactory createBinlogClientFactory() { - return new BinlogClientFactory(sourceConfig, rdsClient); + return new BinlogClientFactory(sourceConfig, rdsClient, dbMetadata); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java index 4dd3930466..1392c852e8 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java @@ -19,6 +19,7 @@ import java.io.IOException; import java.util.Optional; +import java.util.UUID; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -49,11 +50,11 @@ void setUp() { @Test void test_processStream_with_given_binlog_coordinates() throws IOException { - StreamProgressState streamProgressState = mock(StreamProgressState.class); + final StreamProgressState streamProgressState = mock(StreamProgressState.class); + final String binlogFilename = UUID.randomUUID().toString(); + final long binlogPosition = 100L; when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); - final String binlogFilename = "binlog-001"; - final Long binlogPosition = 100L; - when(streamProgressState.getCurrentPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); + when(streamProgressState.getStartPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); when(streamProgressState.shouldWaitForExport()).thenReturn(false); streamWorker.processStream(streamPartition); @@ -69,7 +70,7 @@ void test_processStream_without_current_binlog_coordinates() throws IOException when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); final String binlogFilename = "binlog-001"; final Long binlogPosition = 100L; - when(streamProgressState.getCurrentPosition()).thenReturn(null); + when(streamProgressState.getStartPosition()).thenReturn(null); when(streamProgressState.shouldWaitForExport()).thenReturn(false); streamWorker.processStream(streamPartition); From 1ddebf68f50b730d3e495f50d53ea6fe9129c7b2 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Sat, 10 Aug 2024 12:04:18 -0700 Subject: [PATCH 102/159] Fix null document in DLQ object (#4814) * dplive1.yaml Signed-off-by: Krishna Kondaka * Delete .github/workflows/static.yml Signed-off-by: Krishna Kondaka * Fix null document in DLQ object Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../dataprepper/plugins/sink/opensearch/OpenSearchSink.java | 6 +++--- 1 file changed, 3 insertions(+), 3 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 1b6f44fde6..2248ba669a 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 @@ -448,9 +448,9 @@ public void doOutput(final Collection> records) { } SerializedJson serializedJsonNode = null; - if (StringUtils.equals(action, OpenSearchBulkActions.UPDATE.toString()) || - StringUtils.equals(action, OpenSearchBulkActions.UPSERT.toString()) || - StringUtils.equals(action, OpenSearchBulkActions.DELETE.toString())) { + if (StringUtils.equals(eventAction, OpenSearchBulkActions.UPDATE.toString()) || + StringUtils.equals(eventAction, OpenSearchBulkActions.UPSERT.toString()) || + StringUtils.equals(eventAction, OpenSearchBulkActions.DELETE.toString())) { serializedJsonNode = SerializedJson.fromJsonNode(event.getJsonNode(), document); } BulkOperation bulkOperation; From 176d8288eaea408304ecde56caa60cbea9a626d8 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 12 Aug 2024 10:20:48 -0500 Subject: [PATCH 103/159] Fixes a regex expression bug. When the left-hand side of the operation is null, always return false rather than throwing an exception. Resolves #4763. (#4798) Signed-off-by: David Venable --- .../expression/GenericRegexMatchOperator.java | 2 + ...ericExpressionEvaluator_ConditionalIT.java | 213 +++++++++--------- .../expression/RegexEqualOperatorTest.java | 6 + .../expression/RegexNotEqualOperatorTest.java | 6 + 4 files changed, 123 insertions(+), 104 deletions(-) diff --git a/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/GenericRegexMatchOperator.java b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/GenericRegexMatchOperator.java index 1154978116..b4d34dadc1 100644 --- a/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/GenericRegexMatchOperator.java +++ b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/GenericRegexMatchOperator.java @@ -37,6 +37,8 @@ public int getSymbol() { @Override public Boolean evaluate(final Object ... args) { checkArgument(args.length == 2, displayName + " requires operands length needs to be 2."); + if(args[0] == null) + return false; checkArgument(args[0] instanceof String, displayName + " requires left operand to be String."); checkArgument(args[1] instanceof String, displayName + " requires right operand to be String."); try { diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java index 0bef1a65a0..a8fc7971f3 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java @@ -35,6 +35,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.params.provider.Arguments.arguments; class GenericExpressionEvaluator_ConditionalIT { /** @@ -145,76 +146,80 @@ private static Stream validExpressionArguments() { int testStringLength = random.nextInt(10); String testString = RandomStringUtils.randomAlphabetic(testStringLength); return Stream.of( - Arguments.of("true", event("{}"), true), - Arguments.of("/status_code == 200", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code == 200", longEvent, true), - Arguments.of("/status_code != 300", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code == 200", event("{}"), false), - Arguments.of("/success == /status_code", event("{\"success\": true, \"status_code\": 200}"), false), - Arguments.of("/success != /status_code", event("{\"success\": true, \"status_code\": 200}"), true), - Arguments.of("/part1@part2.part3 != 111", event("{\"success\": true, \"part1@part2.part3\":111, \"status_code\": 200}"), false), - Arguments.of("/part1.part2@part3 != 111", event("{\"success\": true, \"part1.part2@part3\":222, \"status_code\": 200}"), true), - Arguments.of("/pi == 3.14159", event("{\"pi\": 3.14159}"), true), - Arguments.of("/value == 12345.678", event("{\"value\": 12345.678}"), true), - Arguments.of("/value == 12345.678E12", event("{\"value\": 12345.678E12}"), true), - Arguments.of("/value == 12345.678e-12", event("{\"value\": 12345.678e-12}"), true), - Arguments.of("/value == 12345.0000012", event("{\"value\": 12345.0000012}"), true), - Arguments.of("/value == 12345.00012E6", event("{\"value\": 12345.00012E6}"), true), - Arguments.of("true == (/is_cool == true)", event("{\"is_cool\": true}"), true), - Arguments.of("not /is_cool", event("{\"is_cool\": true}"), false), - Arguments.of("/status_code < 300", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code != null", event("{\"status_code\": 200}"), true), - Arguments.of("null != /status_code", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code == null", event("{\"status_code\": null}"), true), - Arguments.of("/response == null", event("{\"status_code\": 200}"), true), - Arguments.of("null == /response", event("{\"status_code\": 200}"), true), - Arguments.of("/response != null", event("{\"status_code\": 200}"), false), - Arguments.of("/status_code <= 0", event("{\"status_code\": 200}"), false), - Arguments.of("/status_code > 0", event("{\"status_code\": 200}"), true), - Arguments.of("/status_code >= 300", event("{\"status_code\": 200}"), false), - Arguments.of("-/status_code == -200", event("{\"status_code\": 200}"), true), - Arguments.of("/success and /status_code == 200", event("{\"success\": true, \"status_code\": 200}"), true), - Arguments.of("/success or /status_code == 200", event("{\"success\": false, \"status_code\": 200}"), true), - Arguments.of("(/success == true) or (/status_code == 200)", event("{\"success\": false, \"status_code\": 200}"), true), - Arguments.of("/should_drop", event("{\"should_drop\": true}"), true), - Arguments.of("/should_drop", event("{\"should_drop\": false}"), false), - Arguments.of("/logs/2/should_drop", event("{\"logs\": [{}, {}, {\"should_drop\": true}]}"), true), - Arguments.of( + arguments("true", event("{}"), true), + arguments("/status_code == 200", event("{\"status_code\": 200}"), true), + arguments("/status_code == 200", longEvent, true), + arguments("/status_code != 300", event("{\"status_code\": 200}"), true), + arguments("/status_code == 200", event("{}"), false), + arguments("/success == /status_code", event("{\"success\": true, \"status_code\": 200}"), false), + arguments("/success != /status_code", event("{\"success\": true, \"status_code\": 200}"), true), + arguments("/part1@part2.part3 != 111", event("{\"success\": true, \"part1@part2.part3\":111, \"status_code\": 200}"), false), + arguments("/part1.part2@part3 != 111", event("{\"success\": true, \"part1.part2@part3\":222, \"status_code\": 200}"), true), + arguments("/pi == 3.14159", event("{\"pi\": 3.14159}"), true), + arguments("/value == 12345.678", event("{\"value\": 12345.678}"), true), + arguments("/value == 12345.678E12", event("{\"value\": 12345.678E12}"), true), + arguments("/value == 12345.678e-12", event("{\"value\": 12345.678e-12}"), true), + arguments("/value == 12345.0000012", event("{\"value\": 12345.0000012}"), true), + arguments("/value == 12345.00012E6", event("{\"value\": 12345.00012E6}"), true), + arguments("true == (/is_cool == true)", event("{\"is_cool\": true}"), true), + arguments("not /is_cool", event("{\"is_cool\": true}"), false), + arguments("/status_code < 300", event("{\"status_code\": 200}"), true), + arguments("/status_code != null", event("{\"status_code\": 200}"), true), + arguments("null != /status_code", event("{\"status_code\": 200}"), true), + arguments("/status_code == null", event("{\"status_code\": null}"), true), + arguments("/response == null", event("{\"status_code\": 200}"), true), + arguments("null == /response", event("{\"status_code\": 200}"), true), + arguments("/response != null", event("{\"status_code\": 200}"), false), + arguments("/status_code <= 0", event("{\"status_code\": 200}"), false), + arguments("/status_code > 0", event("{\"status_code\": 200}"), true), + arguments("/status_code >= 300", event("{\"status_code\": 200}"), false), + arguments("-/status_code == -200", event("{\"status_code\": 200}"), true), + arguments("/success and /status_code == 200", event("{\"success\": true, \"status_code\": 200}"), true), + arguments("/success or /status_code == 200", event("{\"success\": false, \"status_code\": 200}"), true), + arguments("(/success == true) or (/status_code == 200)", event("{\"success\": false, \"status_code\": 200}"), true), + arguments("/should_drop", event("{\"should_drop\": true}"), true), + arguments("/should_drop", event("{\"should_drop\": false}"), false), + arguments("/logs/2/should_drop", event("{\"logs\": [{}, {}, {\"should_drop\": true}]}"), true), + arguments( escapedJsonPointer(ALL_JACKSON_EVENT_GET_SUPPORTED_CHARACTERS) + " == true", complexEvent(ALL_JACKSON_EVENT_GET_SUPPORTED_CHARACTERS, true), true), - Arguments.of("/durationInNanos > 5000000000", event("{\"durationInNanos\": 6000000000}"), true), - Arguments.of("/response == \"OK\"", event("{\"response\": \"OK\"}"), true), - Arguments.of("length(/response) == "+testStringLength, event("{\"response\": \""+testString+"\"}"), true), - Arguments.of("hasTags(\""+ testTag1+"\")", longEvent, true), - Arguments.of("hasTags(\""+ testTag1+"\",\""+testTag2+"\")", longEvent, true), - Arguments.of("hasTags(\""+ testTag1+"\", \""+testTag2+"\", \""+testTag3+"\")", longEvent, true), - Arguments.of("hasTags(\""+ testTag4+"\")", longEvent, false), - Arguments.of("hasTags(\""+ testTag3+"\",\""+testTag4+"\")", longEvent, false), - Arguments.of("contains(\""+ strValue+"\",\""+strValue.substring(1,5)+"\")", longEvent, true), - Arguments.of("contains(/status,\""+strValue.substring(0,2)+"\")", event("{\"status\":\""+strValue+"\"}"), true), - Arguments.of("contains(\""+strValue+strValue+"\",/status)", event("{\"status\":\""+strValue+"\"}"), true), - Arguments.of("contains(/message,/status)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), true), - Arguments.of("contains(/unknown,/status)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), false), - Arguments.of("contains(/status,/unknown)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), false), - Arguments.of("getMetadata(\"key1\") == \""+strValue+"\"", longEvent, true), - Arguments.of("getMetadata(\"key2\") == "+value4, longEvent, true), - Arguments.of("getMetadata(\"key3\") == "+value5, longEvent, true), - Arguments.of("getMetadata(\"/key1\") == \""+strValue+"\"", longEvent, true), - Arguments.of("getMetadata(\"/key2\") == "+value4, longEvent, true), - Arguments.of("getMetadata(\"key3\") == "+value5, longEvent, true), - Arguments.of("getMetadata(\"/key6\") == \""+value5+"\"", longEvent, false), - Arguments.of("getMetadata(\"key6\") == "+value5, longEvent, false), - Arguments.of("cidrContains(/sourceIp,\"192.0.2.0/24\")", event("{\"sourceIp\": \"192.0.2.3\"}"), true), - Arguments.of("cidrContains(/sourceIp,\"192.0.2.0/24\",\"192.1.1.0/24\")", event("{\"sourceIp\": \"192.0.2.3\"}"), true), - Arguments.of("cidrContains(/sourceIp,\"192.0.2.0/24\",\"192.1.1.0/24\")", event("{\"sourceIp\": \"192.2.2.3\"}"), false), - Arguments.of("cidrContains(/sourceIp,\"2001:0db8::/32\")", event("{\"sourceIp\": \"2001:0db8:aaaa:bbbb::\"}"), true), - Arguments.of("cidrContains(/sourceIp,\"2001:0db8::/32\",\"2001:aaaa::/32\")", event("{\"sourceIp\": \"2001:0db8:aaaa:bbbb::\"}"), true), - Arguments.of("cidrContains(/sourceIp,\"2001:0db8::/32\",\"2001:aaaa::/32\")", event("{\"sourceIp\": \"2001:abcd:aaaa:bbbb::\"}"), false), - Arguments.of("/sourceIp != null", event("{\"sourceIp\": [10, 20]}"), true), - Arguments.of("/sourceIp == null", event("{\"sourceIp\": [\"test\", \"test_two\"]}"), false), - Arguments.of("/sourceIp == null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), false), - Arguments.of("/sourceIp != null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), true) + arguments("/durationInNanos > 5000000000", event("{\"durationInNanos\": 6000000000}"), true), + arguments("/response == \"OK\"", event("{\"response\": \"OK\"}"), true), + arguments("length(/response) == "+testStringLength, event("{\"response\": \""+testString+"\"}"), true), + arguments("hasTags(\""+ testTag1+"\")", longEvent, true), + arguments("hasTags(\""+ testTag1+"\",\""+testTag2+"\")", longEvent, true), + arguments("hasTags(\""+ testTag1+"\", \""+testTag2+"\", \""+testTag3+"\")", longEvent, true), + arguments("hasTags(\""+ testTag4+"\")", longEvent, false), + arguments("hasTags(\""+ testTag3+"\",\""+testTag4+"\")", longEvent, false), + arguments("contains(\""+ strValue+"\",\""+strValue.substring(1,5)+"\")", longEvent, true), + arguments("contains(/status,\""+strValue.substring(0,2)+"\")", event("{\"status\":\""+strValue+"\"}"), true), + arguments("contains(\""+strValue+strValue+"\",/status)", event("{\"status\":\""+strValue+"\"}"), true), + arguments("contains(/message,/status)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), true), + arguments("contains(/unknown,/status)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), false), + arguments("contains(/status,/unknown)", event("{\"status\":\""+strValue+"\", \"message\":\""+strValue+strValue+"\"}"), false), + arguments("getMetadata(\"key1\") == \""+strValue+"\"", longEvent, true), + arguments("getMetadata(\"key2\") == "+value4, longEvent, true), + arguments("getMetadata(\"key3\") == "+value5, longEvent, true), + arguments("getMetadata(\"/key1\") == \""+strValue+"\"", longEvent, true), + arguments("getMetadata(\"/key2\") == "+value4, longEvent, true), + arguments("getMetadata(\"key3\") == "+value5, longEvent, true), + arguments("getMetadata(\"/key6\") == \""+value5+"\"", longEvent, false), + arguments("getMetadata(\"key6\") == "+value5, longEvent, false), + arguments("cidrContains(/sourceIp,\"192.0.2.0/24\")", event("{\"sourceIp\": \"192.0.2.3\"}"), true), + arguments("cidrContains(/sourceIp,\"192.0.2.0/24\",\"192.1.1.0/24\")", event("{\"sourceIp\": \"192.0.2.3\"}"), true), + arguments("cidrContains(/sourceIp,\"192.0.2.0/24\",\"192.1.1.0/24\")", event("{\"sourceIp\": \"192.2.2.3\"}"), false), + arguments("cidrContains(/sourceIp,\"2001:0db8::/32\")", event("{\"sourceIp\": \"2001:0db8:aaaa:bbbb::\"}"), true), + arguments("cidrContains(/sourceIp,\"2001:0db8::/32\",\"2001:aaaa::/32\")", event("{\"sourceIp\": \"2001:0db8:aaaa:bbbb::\"}"), true), + arguments("cidrContains(/sourceIp,\"2001:0db8::/32\",\"2001:aaaa::/32\")", event("{\"sourceIp\": \"2001:abcd:aaaa:bbbb::\"}"), false), + arguments("/sourceIp != null", event("{\"sourceIp\": [10, 20]}"), true), + arguments("/sourceIp == null", event("{\"sourceIp\": [\"test\", \"test_two\"]}"), false), + arguments("/sourceIp == null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), false), + arguments("/sourceIp != null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), true), + arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-0\"}"), true), + arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-212\"}"), true), + arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-abc\"}"), false), + arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"other\": \"dataprepper-abc\"}"), false) ); } @@ -236,43 +241,43 @@ private static Stream invalidExpressionArguments() { int testStringLength = random.nextInt(10); String testString = RandomStringUtils.randomAlphabetic(testStringLength); return Stream.of( - Arguments.of("/missing", event("{}")), - Arguments.of("/success < /status_code", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success <= /status_code", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success > /status_code", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success >= /status_code", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success > null", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/success >= null", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/status_code < null", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("/status_code <= null", event("{\"success\": true, \"status_code\": 200}")), - Arguments.of("not /status_code", event("{\"status_code\": 200}")), - Arguments.of("/status_code >= 200 and 3", event("{\"status_code\": 200}")), - Arguments.of("", event("{}")), - Arguments.of("-false", event("{}")), - Arguments.of("not 5", event("{}")), - Arguments.of("not null", event("{}")), - Arguments.of("not/status_code", event("{\"status_code\": 200}")), - Arguments.of("trueand/status_code", event("{\"status_code\": 200}")), - Arguments.of("trueor/status_code", event("{\"status_code\": 200}")), - Arguments.of("length(\""+testString+") == "+testStringLength, event("{\"response\": \""+testString+"\"}")), - Arguments.of("length(\""+testString+"\") == "+testStringLength, event("{\"response\": \""+testString+"\"}")), - Arguments.of("hasTags(10)", tagEvent), - Arguments.of("hasTags("+ testTag1+")", tagEvent), - Arguments.of("hasTags(\""+ testTag1+")", tagEvent), - Arguments.of("hasTags(\""+ testTag1+"\","+testTag2+"\")", tagEvent), - Arguments.of("hasTags(,\""+testTag2+"\")", tagEvent), - Arguments.of("hasTags(\""+testTag2+"\",)", tagEvent), - Arguments.of("contains(\""+testTag2+"\",)", tagEvent), - Arguments.of("contains(\""+testTag2+"\")", tagEvent), - Arguments.of("contains(/intField, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), - Arguments.of("contains(1234, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), - Arguments.of("contains(str, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), - Arguments.of("contains(/strField, 1234)", event("{\"intField\":1234,\"strField\":\"string\"}")), - Arguments.of("getMetadata(10)", tagEvent), - Arguments.of("getMetadata("+ testMetadataKey+ ")", tagEvent), - Arguments.of("getMetadata(\""+ testMetadataKey+")", tagEvent), - Arguments.of("cidrContains(/sourceIp)", event("{\"sourceIp\": \"192.0.2.3\"}")), - Arguments.of("cidrContains(/sourceIp,123)", event("{\"sourceIp\": \"192.0.2.3\"}")) + arguments("/missing", event("{}")), + arguments("/success < /status_code", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success <= /status_code", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success > /status_code", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success >= /status_code", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success > null", event("{\"success\": true, \"status_code\": 200}")), + arguments("/success >= null", event("{\"success\": true, \"status_code\": 200}")), + arguments("/status_code < null", event("{\"success\": true, \"status_code\": 200}")), + arguments("/status_code <= null", event("{\"success\": true, \"status_code\": 200}")), + arguments("not /status_code", event("{\"status_code\": 200}")), + arguments("/status_code >= 200 and 3", event("{\"status_code\": 200}")), + arguments("", event("{}")), + arguments("-false", event("{}")), + arguments("not 5", event("{}")), + arguments("not null", event("{}")), + arguments("not/status_code", event("{\"status_code\": 200}")), + arguments("trueand/status_code", event("{\"status_code\": 200}")), + arguments("trueor/status_code", event("{\"status_code\": 200}")), + arguments("length(\""+testString+") == "+testStringLength, event("{\"response\": \""+testString+"\"}")), + arguments("length(\""+testString+"\") == "+testStringLength, event("{\"response\": \""+testString+"\"}")), + arguments("hasTags(10)", tagEvent), + arguments("hasTags("+ testTag1+")", tagEvent), + arguments("hasTags(\""+ testTag1+")", tagEvent), + arguments("hasTags(\""+ testTag1+"\","+testTag2+"\")", tagEvent), + arguments("hasTags(,\""+testTag2+"\")", tagEvent), + arguments("hasTags(\""+testTag2+"\",)", tagEvent), + arguments("contains(\""+testTag2+"\",)", tagEvent), + arguments("contains(\""+testTag2+"\")", tagEvent), + arguments("contains(/intField, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), + arguments("contains(1234, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), + arguments("contains(str, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), + arguments("contains(/strField, 1234)", event("{\"intField\":1234,\"strField\":\"string\"}")), + arguments("getMetadata(10)", tagEvent), + arguments("getMetadata("+ testMetadataKey+ ")", tagEvent), + arguments("getMetadata(\""+ testMetadataKey+")", tagEvent), + arguments("cidrContains(/sourceIp)", event("{\"sourceIp\": \"192.0.2.3\"}")), + arguments("cidrContains(/sourceIp,123)", event("{\"sourceIp\": \"192.0.2.3\"}")) ); } diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexEqualOperatorTest.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexEqualOperatorTest.java index 46fcdd9ccf..bb92cd1e49 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexEqualOperatorTest.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexEqualOperatorTest.java @@ -12,6 +12,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.expression.antlr.DataPrepperExpressionParser; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -64,4 +65,9 @@ void testEvalInValidArgType() { void testEvalInValidPattern() { assertThrows(IllegalArgumentException.class, () -> objectUnderTest.evaluate("a", "*")); } + + @Test + void evaluate_with_null_lhs_returns_false() { + assertThat(objectUnderTest.evaluate(null, "a*"), equalTo(false)); + } } diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexNotEqualOperatorTest.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexNotEqualOperatorTest.java index 221c354eb7..30bc199413 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexNotEqualOperatorTest.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/RegexNotEqualOperatorTest.java @@ -12,6 +12,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.expression.antlr.DataPrepperExpressionParser; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -64,4 +65,9 @@ void testEvalInValidArgType() { void testEvalInValidPattern() { assertThrows(IllegalArgumentException.class, () -> objectUnderTest.evaluate("a", "*")); } + + @Test + void evaluate_with_null_lhs_returns_false() { + assertThat(objectUnderTest.evaluate(null, "a*"), equalTo(false)); + } } From 0e00df68a60f9dd57ad34d1182065c0ba4917067 Mon Sep 17 00:00:00 2001 From: Jayesh Parmar <89792517+jayeshjeh@users.noreply.github.com> Date: Mon, 12 Aug 2024 21:26:55 +0530 Subject: [PATCH 104/159] Create docker-compose-dataprepper.yaml (#4756) * Create docker-compose-dataprepper.yaml Signed-off-by: Jayesh Parmar <89792517+jayeshjeh@users.noreply.github.com> Signed-off-by: jayeshjeh * Necessary chnages made Signed-off-by: jayeshjeh --------- Signed-off-by: Jayesh Parmar <89792517+jayeshjeh@users.noreply.github.com> Signed-off-by: jayeshjeh --- examples/log-ingestion/data-prepper-config.yaml | 6 ++++++ .../docker-compose-dataprepper.yaml | 16 ++++++++++++++++ examples/log-ingestion/docker-compose.yaml | 1 + examples/log-ingestion/fluent-bit.conf | 4 +++- examples/log-ingestion/log_pipeline.yaml | 2 +- 5 files changed, 27 insertions(+), 2 deletions(-) create mode 100644 examples/log-ingestion/data-prepper-config.yaml create mode 100644 examples/log-ingestion/docker-compose-dataprepper.yaml diff --git a/examples/log-ingestion/data-prepper-config.yaml b/examples/log-ingestion/data-prepper-config.yaml new file mode 100644 index 0000000000..c42ba72275 --- /dev/null +++ b/examples/log-ingestion/data-prepper-config.yaml @@ -0,0 +1,6 @@ +ssl: false +serverPort: 4900 +authentication: + http_basic: + username: admin + password: admin diff --git a/examples/log-ingestion/docker-compose-dataprepper.yaml b/examples/log-ingestion/docker-compose-dataprepper.yaml new file mode 100644 index 0000000000..29e2fdcc64 --- /dev/null +++ b/examples/log-ingestion/docker-compose-dataprepper.yaml @@ -0,0 +1,16 @@ +version: '3.7' +services: + data-prepper: + image: opensearchproject/data-prepper:2 + container_name: data-prepper + volumes: + - ./log_pipeline.yaml:/usr/share/data-prepper/pipelines/log_pipeline.yaml + - ./data-prepper-config.yaml:/usr/share/data-prepper/config/data-prepper-config.yaml + ports: + - 2021:2021 + networks: + - opensearch-net + +networks: + opensearch-net: + driver: bridge diff --git a/examples/log-ingestion/docker-compose.yaml b/examples/log-ingestion/docker-compose.yaml index 099ae707cf..c7864d50dd 100644 --- a/examples/log-ingestion/docker-compose.yaml +++ b/examples/log-ingestion/docker-compose.yaml @@ -15,6 +15,7 @@ services: - discovery.type=single-node - bootstrap.memory_lock=true # along with the memlock settings below, disables swapping - "OPENSEARCH_JAVA_OPTS=-Xms512m -Xmx512m" # minimum and maximum Java heap size, recommend setting both to 50% of system RAM + - "OPENSEARCH_INITIAL_ADMIN_PASSWORD=Developer@123" ulimits: memlock: soft: -1 diff --git a/examples/log-ingestion/fluent-bit.conf b/examples/log-ingestion/fluent-bit.conf index b7a1a6c5dc..5ea9fdbf96 100644 --- a/examples/log-ingestion/fluent-bit.conf +++ b/examples/log-ingestion/fluent-bit.conf @@ -10,4 +10,6 @@ Host data-prepper Port 2021 URI /log/ingest - Format json \ No newline at end of file + Format json + HTTP_User admin + HTTP_Passwd admin diff --git a/examples/log-ingestion/log_pipeline.yaml b/examples/log-ingestion/log_pipeline.yaml index ee40619d6b..6f357239d2 100644 --- a/examples/log-ingestion/log_pipeline.yaml +++ b/examples/log-ingestion/log_pipeline.yaml @@ -11,5 +11,5 @@ log-pipeline: hosts: [ "https://opensearch:9200" ] insecure: true username: admin - password: admin + password: Developer@123 index: apache_logs From fedd07073f8a5b89b69456a116eed3cf935ba398 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 12 Aug 2024 11:25:22 -0500 Subject: [PATCH 105/159] Release notes for Data Prepper 2.8.1 (#4807) Signed-off-by: David Venable --- .../data-prepper.release-notes-2.8.1.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) create mode 100644 release/release-notes/data-prepper.release-notes-2.8.1.md diff --git a/release/release-notes/data-prepper.release-notes-2.8.1.md b/release/release-notes/data-prepper.release-notes-2.8.1.md new file mode 100644 index 0000000000..72f9989f2f --- /dev/null +++ b/release/release-notes/data-prepper.release-notes-2.8.1.md @@ -0,0 +1,12 @@ +## 2024-08-01 Version 2.8.1 + +--- + +### Bug Fixes +* Jackson 2.17.0 LockFreePool causes memory issues ([#4729](https://github.com/opensearch-project/data-prepper/issues/4729)) + + +### Maintenance +* Updates Jackson to 2.17.2 ([#4753](https://github.com/opensearch-project/data-prepper/pull/4753)) +* Updates to Armeria 1.29.0 ([#4741](https://github.com/opensearch-project/data-prepper/pull/4741)) +* Parquet codec tests fix ([#4742](https://github.com/opensearch-project/data-prepper/pull/4742)) From be85abe67e195c9757c560a9377c7a17f5e2df82 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Mon, 12 Aug 2024 11:51:51 -0500 Subject: [PATCH 106/159] ENH: add folder path as output for schema generation (#4820) * ENH: add folder path as output Signed-off-by: George Chen --- data-prepper-plugin-schema-cli/README.md | 3 +- .../DataPrepperPluginSchemaExecute.java | 69 ++++++++++++++++--- 2 files changed, 61 insertions(+), 11 deletions(-) diff --git a/data-prepper-plugin-schema-cli/README.md b/data-prepper-plugin-schema-cli/README.md index 7a4d9bc11b..30b0612603 100644 --- a/data-prepper-plugin-schema-cli/README.md +++ b/data-prepper-plugin-schema-cli/README.md @@ -5,8 +5,9 @@ This module includes the SDK and CLI for generating schemas for Data Prepper pip ## CLI Usage ``` -./gradlew :data-prepper-plugin-schema-cli:run --args='--plugin_type=processor --plugin_names=grok' +./gradlew :data-prepper-plugin-schema-cli:run --args='--plugin_type=processor --plugin_names=grok --output_folder=/path/to/schemas' ``` * plugin_type: A required parameter specifies type of processor. Valid options are `source`, `buffer`, `processor`, `route`, `sink`. * plugin_names: An optional parameter filters the result by plugin names separated by `,`, e.g. `grok,date`. +* output_folder: An optional parameter to specify the output folder path. diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java index a1a76c0510..a505a013c4 100644 --- a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java @@ -14,11 +14,16 @@ import org.slf4j.LoggerFactory; import picocli.CommandLine; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; -import java.util.stream.Stream; import static com.github.victools.jsonschema.module.jackson.JacksonOption.RESPECT_JSONPROPERTY_REQUIRED; @@ -37,6 +42,9 @@ public class DataPrepperPluginSchemaExecute implements Runnable { @CommandLine.Option(names = {"--site.baseurl"}, defaultValue = "/docs/latest") private String siteBaseUrl; + @CommandLine.Option(names = {"--output_folder"}) + private String folderPath; + public static void main(String[] args) { final int exitCode = new CommandLine(new DataPrepperPluginSchemaExecute()).execute(args); System.exit(exitCode); @@ -57,18 +65,59 @@ public void run() { final Class pluginType = pluginConfigsJsonSchemaConverter.pluginTypeNameToPluginType(pluginTypeName); final Map pluginNameToJsonSchemaMap = pluginConfigsJsonSchemaConverter.convertPluginConfigsIntoJsonSchemas( SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, pluginType); + Map filteredPluginNameToJsonSchemaMap; if (pluginNames == null) { - pluginNameToJsonSchemaMap.values().forEach(System.out::println); + filteredPluginNameToJsonSchemaMap = pluginNameToJsonSchemaMap; } else { final Set pluginNamesSet = Set.of(pluginNames.split(",")); - final List result = pluginNamesSet.stream().flatMap(name -> { - if (!pluginNameToJsonSchemaMap.containsKey(name)) { - LOG.error("plugin name: {} not found", name); - return Stream.empty(); - } - return Stream.of(pluginNameToJsonSchemaMap.get(name)); - }).collect(Collectors.toList()); - result.forEach(System.out::println); + filteredPluginNameToJsonSchemaMap = pluginNamesSet.stream() + .filter(name -> { + if (!pluginNameToJsonSchemaMap.containsKey(name)) { + LOG.error("plugin name: {} not found", name); + return false; + } + return true; + }) + .collect(Collectors.toMap( + Function.identity(), + pluginNameToJsonSchemaMap::get + )); + } + + if (folderPath == null) { + writeCollectionToConsole(filteredPluginNameToJsonSchemaMap.values()); + } else { + writeMapToFiles(filteredPluginNameToJsonSchemaMap, folderPath); + } + } + + private static void writeCollectionToConsole(final Collection values) { + values.forEach(System.out::println); + } + + private static void writeMapToFiles(final Map map, final String folderPath) { + // Ensure the directory exists + final Path directory = Paths.get(folderPath); + if (!Files.exists(directory)) { + try { + Files.createDirectories(directory); + } catch (IOException e) { + System.err.println("Error creating directory: " + e.getMessage()); + return; + } + } + + // Iterate through the map and write each entry to a file + for (final Map.Entry entry : map.entrySet()) { + final String fileName = entry.getKey() + ".json"; + final Path filePath = directory.resolve(fileName); + + try { + Files.write(filePath, entry.getValue().getBytes()); + System.out.println("Written file: " + filePath); + } catch (IOException e) { + System.err.println("Error writing file " + fileName + ": " + e.getMessage()); + } } } } From aa50a1d1fbf2499c5edfe4da8aaa806cf474da86 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Mon, 12 Aug 2024 13:24:50 -0500 Subject: [PATCH 107/159] FIX: build service map relationship even when trace group is missing (#4822) Signed-off-by: George Chen --- .../processor/ServiceMapStatefulProcessor.java | 2 +- .../ServiceMapStatefulProcessorTest.java | 15 ++++++++++----- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java index 75041a09b4..667b8ea882 100644 --- a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java +++ b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessor.java @@ -242,7 +242,7 @@ private Collection> iterateProcessorState(final MapDbProcessorStat } final String traceGroupName = getTraceGroupName(child.traceId); - if (traceGroupName == null || parent == null || parent.serviceName.equals(child.serviceName)) { + if (parent == null || parent.serviceName.equals(child.serviceName)) { return; } diff --git a/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java b/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java index b565642e19..cae81c8134 100644 --- a/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java +++ b/data-prepper-plugins/service-map-stateful/src/test/java/org/opensearch/dataprepper/plugins/processor/ServiceMapStatefulProcessorTest.java @@ -40,6 +40,7 @@ import static io.opentelemetry.proto.trace.v1.Span.SpanKind.SPAN_KIND_CLIENT; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -301,7 +302,6 @@ public void testTraceGroupsWithEventRecordData() throws Exception { assertThat(relationshipCountMeasurement.getValue(), equalTo((double)relationshipsFound.size())); - //Make sure that future relationships that are equivalent are caught by cache final byte[] rootSpanId3Bytes = ServiceMapTestUtils.getRandomBytes(8); final byte[] traceId3Bytes = ServiceMapTestUtils.getRandomBytes(16); final String rootSpanId3 = Hex.encodeHexString(rootSpanId3Bytes); @@ -312,19 +312,24 @@ public void testTraceGroupsWithEventRecordData() throws Exception { AUTHENTICATION_SERVICE, "reset", Hex.encodeHexString(ServiceMapTestUtils.getRandomBytes(8)), frontendSpans3.getSpanId(), traceId3, io.opentelemetry.proto.trace.v1.Span.SpanKind.SPAN_KIND_SERVER); + // relationship missing traceGroupName when(clock.millis()).thenReturn(450L); Future> r7 = ServiceMapTestUtils.startExecuteAsync(threadpool, serviceMapStateful1, Collections.singletonList(new Record<>(frontendSpans3))); Future> r8 = ServiceMapTestUtils.startExecuteAsync(threadpool, serviceMapStateful2, Collections.singletonList(new Record<>(authenticationSpansServer2))); - assertTrue(r7.get().isEmpty()); - assertTrue(r8.get().isEmpty()); + final Set relationshipsFoundWithNoTraceGroupName = new HashSet<>(); + relationshipsFoundWithNoTraceGroupName.addAll(r7.get()); + relationshipsFoundWithNoTraceGroupName.addAll(r8.get()); when(clock.millis()).thenReturn(560L); Future> r9 = ServiceMapTestUtils.startExecuteAsync(threadpool, serviceMapStateful1, Arrays.asList()); Future> r10 = ServiceMapTestUtils.startExecuteAsync(threadpool, serviceMapStateful2, Arrays.asList()); - assertTrue(r9.get().isEmpty()); - assertTrue(r10.get().isEmpty()); + relationshipsFoundWithNoTraceGroupName.addAll(r9.get()); + relationshipsFoundWithNoTraceGroupName.addAll(r10.get()); + assertThat(relationshipsFoundWithNoTraceGroupName.size(), equalTo(4)); + relationshipsFoundWithNoTraceGroupName.forEach( + relationship -> assertThat(relationship.getTraceGroupName(), nullValue())); serviceMapStateful1.shutdown(); serviceMapStateful2.shutdown(); } From 2f21a437091b2a41cfe6a53df9978ec0b456fd2c Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Mon, 12 Aug 2024 16:21:05 -0500 Subject: [PATCH 108/159] Add delete_source parameter to the csv processor (#4828) Signed-off-by: Taylor Gray --- .../plugins/processor/csv/CsvProcessor.java | 4 ++++ .../processor/csv/CsvProcessorConfig.java | 6 ++++++ .../processor/csv/CsvProcessorTest.java | 19 +++++++++++++++++++ 3 files changed, 29 insertions(+) diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java index a9f99e5862..3e8780a417 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java @@ -99,6 +99,10 @@ public Collection> doExecute(final Collection> recor if (thisEventHasHeaderSource && Boolean.TRUE.equals(config.isDeleteHeader())) { event.delete(config.getColumnNamesSourceKey()); } + + if (config.isDeleteSource()) { + event.delete(config.getSource()); + } } catch (final IOException e) { csvInvalidEventsCounter.increment(); LOG.error(EVENT, "An exception occurred while reading event [{}]", event, e); diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java index 8c770b597a..fb803798b2 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java @@ -62,6 +62,10 @@ public class CsvProcessorConfig { "the processor should be applied to the event.") private String csvWhen; + @JsonPropertyDescription("If true, the configured source field will be deleted after the CSV data is parsed into separate fields.") + @JsonProperty + private boolean deleteSource = false; + /** * The field of the Event that contains the CSV data to be processed. * @@ -120,6 +124,8 @@ public List getColumnNames() { public String getCsvWhen() { return csvWhen; } + public Boolean isDeleteSource() { return deleteSource; } + @AssertTrue(message = "delimiter must be exactly one character.") boolean isValidDelimiter() { return delimiter.length() == 1; diff --git a/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java b/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java index 5239679fab..cacfedb609 100644 --- a/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java +++ b/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java @@ -56,6 +56,7 @@ void setup() { lenient().when(processorConfig.getQuoteCharacter()).thenReturn(defaultConfig.getQuoteCharacter()); lenient().when(processorConfig.getColumnNamesSourceKey()).thenReturn(defaultConfig.getColumnNamesSourceKey()); lenient().when(processorConfig.getColumnNames()).thenReturn(defaultConfig.getColumnNames()); + lenient().when(processorConfig.isDeleteSource()).thenReturn(false); lenient().when(pluginMetrics.counter(CsvProcessor.CSV_INVALID_EVENTS)).thenReturn(csvInvalidEventsCounter); @@ -66,6 +67,24 @@ private CsvProcessor createObjectUnderTest() { return new CsvProcessor(pluginMetrics, processorConfig, expressionEvaluator); } + @Test + void delete_source_true_deletes_the_source() { + when(processorConfig.isDeleteSource()).thenReturn(true); + + when(processorConfig.getSource()).thenReturn("different_source"); + + final Map eventData = new HashMap<>(); + eventData.put("different_source","1,2,3"); + final Record eventUnderTest = buildRecordWithEvent(eventData); + + final List> editedEvents = (List>) csvProcessor.doExecute(Collections.singletonList(eventUnderTest)); + final Event parsedEvent = getSingleEvent(editedEvents); + assertThat(parsedEvent.containsKey("different_source"), equalTo(false)); + assertThatKeyEquals(parsedEvent, "column1", "1"); + assertThatKeyEquals(parsedEvent, "column2", "2"); + assertThatKeyEquals(parsedEvent, "column3", "3"); + } + @Test void do_nothing_when_source_is_null_value_or_does_not_exist_in_the_Event() { From 1487973d61f1f92625e75774f94dc6aa41278136 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Tue, 13 Aug 2024 13:41:22 -0700 Subject: [PATCH 109/159] Config description changes for aggregate and anomaly detector processors. (#4829) * dplive1.yaml Signed-off-by: Krishna Kondaka * Delete .github/workflows/static.yml Signed-off-by: Krishna Kondaka * Add json property description for aggregate processor and anomaly detector processors Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka * Fixed build failure Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../processor/aggregate/AggregateProcessorConfig.java | 8 ++++++++ .../aggregate/actions/AppendAggregateActionConfig.java | 2 ++ .../aggregate/actions/CountAggregateActionConfig.java | 7 +++++++ .../aggregate/actions/HistogramAggregateActionConfig.java | 8 ++++++++ .../actions/PercentSamplerAggregateActionConfig.java | 2 ++ .../actions/RateLimiterAggregateActionConfig.java | 3 +++ .../actions/TailSamplerAggregateActionConfig.java | 4 ++++ .../anomalydetector/AnomalyDetectorProcessorConfig.java | 6 ++++++ .../anomalydetector/modes/RandomCutForestModeConfig.java | 7 +++++++ 9 files changed, 47 insertions(+) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java index 7f81a82194..1c2c9fa701 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.aggregate; import org.opensearch.dataprepper.model.configuration.PluginModel; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; @@ -18,27 +19,34 @@ public class AggregateProcessorConfig { static int DEFAULT_GROUP_DURATION_SECONDS = 180; + @JsonPropertyDescription("An unordered list by which to group events. Events with the same values as these keys are put into the same group. If an event does not contain one of the identification_keys, then the value of that key is considered to be equal to null. At least one identification_key is required (for example, [\"sourceIp\", \"destinationIp\", \"port\"].") @JsonProperty("identification_keys") @NotEmpty private List identificationKeys; + @JsonPropertyDescription("The amount of time that a group should exist before it is concluded automatically. Supports ISO_8601 notation strings (\"PT20.345S\", \"PT15M\", etc.) as well as simple notation for seconds (\"60s\") and milliseconds (\"1500ms\"). Default value is 180s.") @JsonProperty("group_duration") private Duration groupDuration = Duration.ofSeconds(DEFAULT_GROUP_DURATION_SECONDS); + @JsonPropertyDescription("The action to be performed on each group. One of the available aggregate actions must be provided, or you can create custom aggregate actions. remove_duplicates and put_all are the available actions. For more information, see Creating New Aggregate Actions.") @JsonProperty("action") @NotNull private PluginModel aggregateAction; + @JsonPropertyDescription("When local_mode is set to true, the aggregation is performed locally on each Data Prepper node instead of forwarding events to a specific node based on the identification_keys using a hash function. Default is false.") @JsonProperty("local_mode") @NotNull private Boolean localMode = false; + @JsonPropertyDescription("A boolean indicating if the unaggregated events should be forwarded to the next processor/sink in the chain.") @JsonProperty("output_unaggregated_events") private Boolean outputUnaggregatedEvents = false; + @JsonPropertyDescription("Tag to be used for aggregated events to distinguish aggregated events from unaggregated events.") @JsonProperty("aggregated_events_tag") private String aggregatedEventsTag; + @JsonPropertyDescription("A Data Prepper conditional expression (https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the processor will be run on the event.") @JsonProperty("aggregate_when") private String whenCondition; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java index 4d021b7c5a..3d4a9b4a86 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java @@ -8,9 +8,11 @@ import java.util.List; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; public class AppendAggregateActionConfig { + @JsonPropertyDescription("List of keys to append.") @JsonProperty("keys_to_append") List keysToAppend; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java index 1144aee261..a0325ee3a9 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/CountAggregateActionConfig.java @@ -9,6 +9,7 @@ import java.util.List; import java.util.Set; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; public class CountAggregateActionConfig { static final String SUM_METRIC_NAME = "count"; @@ -17,21 +18,27 @@ public class CountAggregateActionConfig { public static final String DEFAULT_END_TIME_KEY = "aggr._end_time"; public static final Set validOutputFormats = new HashSet<>(Set.of(OutputFormat.OTEL_METRICS.toString(), OutputFormat.RAW.toString())); + @JsonPropertyDescription("Key used for storing the count. Default name is aggr._count.") @JsonProperty("count_key") String countKey = DEFAULT_COUNT_KEY; + @JsonPropertyDescription("Metric name to be used when otel format is used.") @JsonProperty("metric_name") String metricName = SUM_METRIC_NAME; + @JsonPropertyDescription("List of unique keys to count.") @JsonProperty("unique_keys") List uniqueKeys = null; + @JsonPropertyDescription("Key used for storing the start time. Default name is aggr._start_time.") @JsonProperty("start_time_key") String startTimeKey = DEFAULT_START_TIME_KEY; + @JsonPropertyDescription("Key used for storing the end time. Default name is aggr._end_time.") @JsonProperty("end_time_key") String endTimeKey = DEFAULT_END_TIME_KEY; + @JsonPropertyDescription("Format of the aggregated event. otel_metrics is the default output format which outputs in OTel metrics SUM type with count as value. Other options is - raw - which generates a JSON object with the count_key field as a count value and the start_time_key field with aggregation start time as value.") @JsonProperty("output_format") String outputFormat = OutputFormat.OTEL_METRICS.toString(); diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java index 7c998c123d..6d89a1bd8f 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/HistogramAggregateActionConfig.java @@ -9,6 +9,7 @@ import java.util.List; import java.util.HashSet; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; public class HistogramAggregateActionConfig { @@ -25,27 +26,34 @@ public class HistogramAggregateActionConfig { public static final String DURATION_KEY = "duration"; public static final Set validOutputFormats = new HashSet<>(Set.of(OutputFormat.OTEL_METRICS.toString(), OutputFormat.RAW.toString())); + @JsonPropertyDescription("Name of the field in the events the histogram generates.") @JsonProperty("key") @NotNull String key; + @JsonPropertyDescription("The name of units for the values in the key. For example, bytes, traces etc") @JsonProperty("units") @NotNull String units; + @JsonPropertyDescription("Metric name to be used when otel format is used.") @JsonProperty("metric_name") String metricName = HISTOGRAM_METRIC_NAME; + @JsonPropertyDescription("Key prefix used by all the fields created in the aggregated event. Having a prefix ensures that the names of the histogram event do not conflict with the field names in the event.") @JsonProperty("generated_key_prefix") String generatedKeyPrefix = DEFAULT_GENERATED_KEY_PREFIX; + @JsonPropertyDescription("A list of buckets (values of type double) indicating the buckets in the histogram.") @JsonProperty("buckets") @NotNull List buckets; + @JsonPropertyDescription("Format of the aggregated event. otel_metrics is the default output format which outputs in OTel metrics SUM type with count as value. Other options is - raw - which generates a JSON object with the count_key field as a count value and the start_time_key field with aggregation start time as value.") @JsonProperty("output_format") String outputFormat = OutputFormat.OTEL_METRICS.toString(); + @JsonPropertyDescription("A Boolean value indicating whether the histogram should include the min and max of the values in the aggregation.") @JsonProperty("record_minmax") boolean recordMinMax = false; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java index d0bc2db7d9..be9770400a 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java @@ -7,9 +7,11 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.NotNull; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; public class PercentSamplerAggregateActionConfig { + @JsonPropertyDescription("Percent value of the sampling to be done. 0.0 < percent < 100.0") @JsonProperty("percent") @NotNull private double percent; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateActionConfig.java index 78de6e74b0..158f3b1ac6 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/RateLimiterAggregateActionConfig.java @@ -8,15 +8,18 @@ import java.util.Set; import java.util.HashSet; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; public class RateLimiterAggregateActionConfig { public static final Set validRateLimiterModes = new HashSet<>(Set.of(RateLimiterMode.BLOCK.toString(), RateLimiterMode.DROP.toString())); + @JsonPropertyDescription("The number of events allowed per second.") @JsonProperty("events_per_second") @NotNull int eventsPerSecond; + @JsonPropertyDescription("Indicates what action the rate_limiter takes when the number of events received is greater than the number of events allowed per second. Default value is block, which blocks the processor from running after the maximum number of events allowed per second is reached until the next second. Alternatively, the drop option drops the excess events received in that second. Default is block") @JsonProperty("when_exceeds") String whenExceedsMode = RateLimiterMode.BLOCK.toString(); diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java index 37475a88e7..f86672e3b9 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java @@ -6,20 +6,24 @@ package org.opensearch.dataprepper.plugins.processor.aggregate.actions; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.AssertTrue; import java.time.Duration; public class TailSamplerAggregateActionConfig { + @JsonPropertyDescription("Period to wait before considering that a trace event is complete") @JsonProperty("wait_period") @NotNull private Duration waitPeriod; + @JsonPropertyDescription("Percent value to use for sampling non error events. 0.0 < percent < 100.0") @JsonProperty("percent") @NotNull private Integer percent; + @JsonPropertyDescription("A Data Prepper conditional expression (https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the event is an error event or not") @JsonProperty("condition") private String condition; 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 6331ee1f21..6545bbb552 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 @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.anomalydetector; import org.opensearch.dataprepper.model.configuration.PluginModel; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -14,20 +15,25 @@ import java.util.List; public class AnomalyDetectorProcessorConfig { + @JsonPropertyDescription("The ML algorithm (or model) used to detect anomalies. You must provide a mode. See random_cut_forest mode.") @JsonProperty("mode") @NotNull private PluginModel detectorMode; + @JsonPropertyDescription("A non-ordered List that is used as input to the ML algorithm to detect anomalies in the values of the keys in the list. At least one key is required.") @JsonProperty("keys") @NotEmpty private List keys; + @JsonPropertyDescription("If provided, anomalies will be detected within each unique instance of these keys. For example, if you provide the ip field, anomalies will be detected separately for each unique IP address.") @JsonProperty("identification_keys") private List identificationKeys = Collections.emptyList(); + @JsonPropertyDescription("RCF will try to automatically learn and reduce the number of anomalies detected. For example, if latency is consistently between 50 and 100, and then suddenly jumps to around 1000, only the first one or two data points after the transition will be detected (unless there are other spikes/anomalies). Similarly, for repeated spikes to the same level, RCF will likely eliminate many of the spikes after a few initial ones. This is because the default setting is to minimize the number of alerts detected. Setting the verbose setting to true will cause RCF to consistently detect these repeated cases, which may be useful for detecting anomalous behavior that lasts an extended period of time. Default is false.") @JsonProperty("verbose") private Boolean verbose = false; + @JsonPropertyDescription("If using the identification_keys settings, a new ML model will be created for every degree of cardinality. This can cause a large amount of memory usage, so it is helpful to set a limit on the number of models. Default limit is 5000.") @JsonProperty("cardinality_limit") private int cardinalityLimit = 5000; diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java index 60ad080dcd..c477746253 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.anomalydetector.modes; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import java.util.Set; @@ -25,25 +26,31 @@ public class RandomCutForestModeConfig { public static final String VERSION_1_0 = "1.0"; + @JsonPropertyDescription("The algorithm version number. Default is 1.0.") @JsonProperty("version") private String version = VERSION_1_0; public static final Set validVersions = new HashSet<>(Set.of(VERSION_1_0)); + @JsonPropertyDescription("The type of data sent to the algorithm. Default is metrics type") @JsonProperty("type") private String type = RandomCutForestType.METRICS.toString(); public static final Set validTypes = new HashSet<>(Set.of(RandomCutForestType.METRICS.toString())); + @JsonPropertyDescription("The shingle size used in the ML algorithm. Default is 60.") @JsonProperty("shingle_size") private int shingleSize = DEFAULT_SHINGLE_SIZE; + @JsonPropertyDescription("The sample size used in the ML algorithm. Default is 256.") @JsonProperty("sample_size") private int sampleSize = DEFAULT_SAMPLE_SIZE; + @JsonPropertyDescription("The time decay value used in the ML algorithm. Used as the mathematical expression timeDecay divided by SampleSize in the ML algorithm. Default is 0.1") @JsonProperty("time_decay") private double timeDecay = DEFAULT_TIME_DECAY; + @JsonPropertyDescription("Output after indicates the number of events to consume before outputting anamolies. Default is 32.") @JsonProperty("output_after") private int outputAfter = DEFAULT_OUTPUT_AFTER; From 38fe2afb91670d5d8ba5a278dd96de0274ad8f58 Mon Sep 17 00:00:00 2001 From: Ivan Tse <115105835+ivan-tse@users.noreply.github.com> Date: Wed, 14 Aug 2024 09:36:12 -0700 Subject: [PATCH 110/159] PersonalizeSink: add client and configuration classes (#4803) PersonalizeSink: add client and configuration classes Signed-off-by: Ivan Tse --- .../personalize-sink/build.gradle | 48 ++++ .../sink/personalize/ClientFactory.java | 58 +++++ .../sink/personalize/PersonalizeSink.java | 80 +++++++ .../personalize/PersonalizeSinkService.java | 68 ++++++ .../AwsAuthenticationOptions.java | 80 +++++++ .../PersonalizeAdvancedValidation.java | 4 + .../PersonalizeSinkConfiguration.java | 137 ++++++++++++ .../dataset/DatasetTypeOptions.java | 33 +++ .../sink/personalize/ClientFactoryTest.java | 135 ++++++++++++ .../sink/personalize/PersonalizeSinkTest.java | 85 ++++++++ .../AwsAuthenticationOptionsTest.java | 129 +++++++++++ .../PersonalizeSinkConfigurationTest.java | 205 ++++++++++++++++++ .../dataset/DatasetTypeOptionsTest.java | 38 ++++ settings.gradle | 1 + 14 files changed, 1101 insertions(+) create mode 100644 data-prepper-plugins/personalize-sink/build.gradle create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactory.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSink.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkService.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptions.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeAdvancedValidation.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfiguration.java create mode 100644 data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptions.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactoryTest.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkTest.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptionsTest.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfigurationTest.java create mode 100644 data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptionsTest.java diff --git a/data-prepper-plugins/personalize-sink/build.gradle b/data-prepper-plugins/personalize-sink/build.gradle new file mode 100644 index 0000000000..bf408a04b8 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/build.gradle @@ -0,0 +1,48 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +dependencies { + implementation project(':data-prepper-api') + implementation project(path: ':data-prepper-plugins:common') + implementation project(':data-prepper-plugins:aws-plugin-api') + implementation 'io.micrometer:micrometer-core' + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'org.hibernate.validator:hibernate-validator:8.0.1.Final' + implementation 'software.amazon.awssdk:personalizeevents' + implementation 'software.amazon.awssdk:sts' + implementation 'software.amazon.awssdk:arns' + testImplementation project(':data-prepper-test-common') + testImplementation testLibs.slf4j.simple +} + +sourceSets { + integrationTest { + java { + compileClasspath += main.output + test.output + runtimeClasspath += main.output + test.output + srcDir file('src/integrationTest/java') + } + resources.srcDir file('src/integrationTest/resources') + } +} + +configurations { + integrationTestImplementation.extendsFrom testImplementation + integrationTestRuntime.extendsFrom testRuntime +} + +task integrationTest(type: Test) { + group = 'verification' + testClassesDirs = sourceSets.integrationTest.output.classesDirs + + useJUnitPlatform() + + classpath = sourceSets.integrationTest.runtimeClasspath + + filter { + includeTestsMatching '*IT' + } +} diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactory.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactory.java new file mode 100644 index 0000000000..2c93fc991b --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactory.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.personalizeevents.PersonalizeEventsClient; + +final class ClientFactory { + private ClientFactory() { } + + static PersonalizeEventsClient createPersonalizeEventsClient(final PersonalizeSinkConfiguration personalizeSinkConfig, final AwsCredentialsSupplier awsCredentialsSupplier) { + final AwsCredentialsOptions awsCredentialsOptions = convertToCredentialsOptions(personalizeSinkConfig.getAwsAuthenticationOptions()); + final AwsCredentialsProvider awsCredentialsProvider = awsCredentialsSupplier.getProvider(awsCredentialsOptions); + + return PersonalizeEventsClient.builder() + .region(getRegion(personalizeSinkConfig, awsCredentialsSupplier)) + .credentialsProvider(awsCredentialsProvider) + .overrideConfiguration(createOverrideConfiguration(personalizeSinkConfig)).build(); + } + + private static ClientOverrideConfiguration createOverrideConfiguration(final PersonalizeSinkConfiguration personalizeSinkConfig) { + final RetryPolicy retryPolicy = RetryPolicy.builder().numRetries(personalizeSinkConfig.getMaxRetries()).build(); + return ClientOverrideConfiguration.builder() + .retryPolicy(retryPolicy) + .build(); + } + + private static AwsCredentialsOptions convertToCredentialsOptions(final AwsAuthenticationOptions awsAuthenticationOptions) { + if (awsAuthenticationOptions == null) { + return AwsCredentialsOptions.builder().build(); + } + return AwsCredentialsOptions.builder() + .withRegion(awsAuthenticationOptions.getAwsRegion().orElse(null)) + .withStsRoleArn(awsAuthenticationOptions.getAwsStsRoleArn()) + .withStsExternalId(awsAuthenticationOptions.getAwsStsExternalId()) + .withStsHeaderOverrides(awsAuthenticationOptions.getAwsStsHeaderOverrides()) + .build(); + } + + private static Region getRegion(final PersonalizeSinkConfiguration personalizeSinkConfig, final AwsCredentialsSupplier awsCredentialsSupplier) { + Region defaultRegion = awsCredentialsSupplier.getDefaultRegion().orElse(null); + if (personalizeSinkConfig.getAwsAuthenticationOptions() == null) { + return defaultRegion; + } else { + return personalizeSinkConfig.getAwsAuthenticationOptions().getAwsRegion().orElse(defaultRegion); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSink.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSink.java new file mode 100644 index 0000000000..a93e58875c --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSink.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.AbstractSink; +import org.opensearch.dataprepper.model.sink.Sink; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import software.amazon.awssdk.services.personalizeevents.PersonalizeEventsClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; + +/** + * Implementation class of personalize-sink plugin. It is responsible for receiving the collection of + * {@link Event} and uploading to amazon personalize. + */ +@DataPrepperPlugin(name = "aws_personalize", pluginType = Sink.class, pluginConfigurationType = PersonalizeSinkConfiguration.class) +public class PersonalizeSink extends AbstractSink> { + + private static final Logger LOG = LoggerFactory.getLogger(PersonalizeSink.class); + + private final PersonalizeSinkConfiguration personalizeSinkConfig; + private volatile boolean sinkInitialized; + private final PersonalizeSinkService personalizeSinkService; + private final SinkContext sinkContext; + + /** + * @param pluginSetting dp plugin settings. + * @param personalizeSinkConfig personalize sink configurations. + * @param sinkContext sink context + * @param awsCredentialsSupplier aws credentials + * @param pluginFactory dp plugin factory. + */ + @DataPrepperPluginConstructor + public PersonalizeSink(final PluginSetting pluginSetting, + final PersonalizeSinkConfiguration personalizeSinkConfig, + final PluginFactory pluginFactory, + final SinkContext sinkContext, + final AwsCredentialsSupplier awsCredentialsSupplier) { + super(pluginSetting); + this.personalizeSinkConfig = personalizeSinkConfig; + this.sinkContext = sinkContext; + + sinkInitialized = false; + + final PersonalizeEventsClient personalizeEventsClient = ClientFactory.createPersonalizeEventsClient(personalizeSinkConfig, awsCredentialsSupplier); + + personalizeSinkService = new PersonalizeSinkService(personalizeSinkConfig, pluginMetrics); + } + + @Override + public boolean isReady() { + return sinkInitialized; + } + + @Override + public void doInitialize() { + sinkInitialized = true; + } + + /** + * @param records Records to be output + */ + @Override + public void doOutput(final Collection> records) { + personalizeSinkService.output(records); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkService.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkService.java new file mode 100644 index 0000000000..80ea94bcf1 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkService.java @@ -0,0 +1,68 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize; + +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * Class responsible for creating PersonalizeEventsClient object, check thresholds, + * get new buffer and write records into buffer. + */ +class PersonalizeSinkService { + + private static final Logger LOG = LoggerFactory.getLogger(PersonalizeSinkService.class); + public static final String RECORDS_SUCCEEDED = "personalizeRecordsSucceeded"; + public static final String RECORDS_FAILED = "personalizeRecordsFailed"; + public static final String RECORDS_INVALID = "personalizeRecordsInvalid"; + public static final String REQUESTS_THROTTLED = "personalizeRequestsThrottled"; + public static final String REQUEST_LATENCY = "personalizeRequestLatency"; + + private final PersonalizeSinkConfiguration personalizeSinkConfig; + private final Lock reentrantLock; + private final int maxRetries; + private final Counter recordsSucceededCounter; + private final Counter recordsFailedCounter; + private final Counter recordsInvalidCounter; + private final Counter requestsThrottledCounter; + private final Timer requestLatencyTimer; + + /** + * @param personalizeSinkConfig personalize sink related configuration. + * @param pluginMetrics metrics. + */ + public PersonalizeSinkService(final PersonalizeSinkConfiguration personalizeSinkConfig, + final PluginMetrics pluginMetrics) { + this.personalizeSinkConfig = personalizeSinkConfig; + reentrantLock = new ReentrantLock(); + + maxRetries = personalizeSinkConfig.getMaxRetries(); + + recordsSucceededCounter = pluginMetrics.counter(RECORDS_SUCCEEDED); + recordsFailedCounter = pluginMetrics.counter(RECORDS_FAILED); + recordsInvalidCounter = pluginMetrics.counter(RECORDS_INVALID); + requestsThrottledCounter = pluginMetrics.counter(REQUESTS_THROTTLED); + requestLatencyTimer = pluginMetrics.timer(REQUEST_LATENCY); + } + + /** + * @param records received records and add into buffer. + */ + void output(Collection> records) { + LOG.trace("{} records received", records.size()); + return; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptions.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptions.java new file mode 100644 index 0000000000..ba7e96d43d --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptions.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.GroupSequence; +import jakarta.validation.constraints.AssertTrue; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.arns.Arn; + +import java.util.Map; +import java.util.Optional; + +@GroupSequence({AwsAuthenticationOptions.class, PersonalizeAdvancedValidation.class}) +public class AwsAuthenticationOptions { + private static final String AWS_IAM_ROLE = "role"; + private static final String AWS_IAM = "iam"; + + @JsonProperty("region") + @Size(min = 1, message = "Region cannot be empty string") + private String awsRegion; + + @JsonProperty("sts_role_arn") + @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") + private String awsStsRoleArn; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + @AssertTrue(message = "sts_role_arn must be an IAM Role", groups = PersonalizeAdvancedValidation.class) + boolean isValidStsRoleArn() { + if (awsStsRoleArn == null) { + return true; + } + final Arn arn = getArn(); + boolean status = true; + if (!AWS_IAM.equals(arn.service())) { + status = false; + } + final Optional resourceType = arn.resource().resourceType(); + if (resourceType.isEmpty() || !resourceType.get().equals(AWS_IAM_ROLE)) { + status = false; + } + return status; + } + + private Arn getArn() { + try { + return Arn.fromString(awsStsRoleArn); + } catch (final Exception e) { + throw new IllegalArgumentException(String.format("Invalid ARN format for awsStsRoleArn. Check the format of %s", awsStsRoleArn)); + } + } + + public Optional getAwsRegion() { + Region region = awsRegion != null ? Region.of(awsRegion) : null; + return Optional.ofNullable(region); + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeAdvancedValidation.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeAdvancedValidation.java new file mode 100644 index 0000000000..f48c1d9466 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeAdvancedValidation.java @@ -0,0 +1,4 @@ +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +interface PersonalizeAdvancedValidation { +} diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfiguration.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfiguration.java new file mode 100644 index 0000000000..95c9f1d5c9 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfiguration.java @@ -0,0 +1,137 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +import jakarta.validation.GroupSequence; +import org.opensearch.dataprepper.plugins.sink.personalize.dataset.DatasetTypeOptions; +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.AssertTrue; +import software.amazon.awssdk.arns.Arn; + +import java.util.List; +import java.util.Optional; + +/** + * personalize sink configuration class contains properties, used to read yaml configuration. + */ +@GroupSequence({PersonalizeSinkConfiguration.class, PersonalizeAdvancedValidation.class}) +public class PersonalizeSinkConfiguration { + private static final int DEFAULT_RETRIES = 10; + private static final String AWS_PERSONALIZE = "personalize"; + private static final String AWS_PERSONALIZE_DATASET = "dataset"; + private static final List DATASET_ARN_REQUIRED_LIST = List.of(DatasetTypeOptions.USERS, DatasetTypeOptions.ITEMS); + + @JsonProperty("aws") + @Valid + private AwsAuthenticationOptions awsAuthenticationOptions; + + @JsonProperty("dataset_type") + @NotNull + @Valid + private DatasetTypeOptions datasetType; + + @JsonProperty("dataset_arn") + private String datasetArn; + + @JsonProperty("tracking_id") + private String trackingId; + + @JsonProperty("document_root_key") + private String documentRootKey; + + @JsonProperty("max_retries") + private int maxRetries = DEFAULT_RETRIES; + + @AssertTrue(message = "A dataset arn is required for items and users datasets.", groups = PersonalizeAdvancedValidation.class) + boolean isDatasetArnProvidedWhenNeeded() { + if (DATASET_ARN_REQUIRED_LIST.contains(datasetType)) { + return datasetArn != null; + } + return true; + } + + @AssertTrue(message = "dataset_arn must be a Personalize Dataset arn", groups = PersonalizeAdvancedValidation.class) + boolean isValidDatasetArn() { + if (datasetArn == null) { + return true; + } + final Arn arn = getArn(); + boolean status = true; + if (!AWS_PERSONALIZE.equals(arn.service())) { + status = false; + } + final Optional resourceType = arn.resource().resourceType(); + if (resourceType.isEmpty() || !resourceType.get().equals(AWS_PERSONALIZE_DATASET)) { + status = false; + } + return status; + } + + private Arn getArn() { + try { + return Arn.fromString(datasetArn); + } catch (final Exception e) { + throw new IllegalArgumentException(String.format("Invalid ARN format for datasetArn. Check the format of %s", datasetArn), e); + } + } + + @AssertTrue(message = "A tracking id is required for interactions dataset.", groups = PersonalizeAdvancedValidation.class) + boolean isTrackingIdProvidedWhenNeeded() { + if (DatasetTypeOptions.INTERACTIONS.equals(datasetType)) { + return trackingId != null; + } + return true; + } + + /** + * Aws Authentication configuration Options. + * @return aws authentication options. + */ + public AwsAuthenticationOptions getAwsAuthenticationOptions() { + return awsAuthenticationOptions; + } + + /** + * Dataset type configuration Options. + * @return dataset type option object. + */ + public DatasetTypeOptions getDatasetType() { + return datasetType; + } + + /** + * Dataset arn for Personalize Dataset. + * @return dataset arn string. + */ + public String getDatasetArn() { + return datasetArn; + } + + /** + * Tracking id for Personalize Event Tracker. + * @return tracking id string. + */ + public String getTrackingId() { + return trackingId; + } + + /** + * Tracking id for Personalize Event Tracker. + * @return document root key string. + */ + public String getDocumentRootKey() { + return documentRootKey; + } + + /** + * Personalize client retries configuration Options. + * @return maximum retries value. + */ + public int getMaxRetries() { + return maxRetries; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptions.java b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptions.java new file mode 100644 index 0000000000..cc6791f0a6 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptions.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.personalize.dataset; + +import com.fasterxml.jackson.annotation.JsonCreator; +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +/** + * Defines all the dataset types enumerations. + */ +public enum DatasetTypeOptions { + USERS("users"), + ITEMS("items"), + INTERACTIONS("interactions"); + + private final String option; + private static final Map OPTIONS_MAP = Arrays.stream(DatasetTypeOptions.values()) + .collect(Collectors.toMap(value -> value.option, value -> value)); + + DatasetTypeOptions(final String option) { + this.option = option.toLowerCase(); + } + + @JsonCreator + static DatasetTypeOptions fromOptionValue(final String option) { + return OPTIONS_MAP.get(option); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactoryTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactoryTest.java new file mode 100644 index 0000000000..6b1ad7f80a --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/ClientFactoryTest.java @@ -0,0 +1,135 @@ +package org.opensearch.dataprepper.plugins.sink.personalize; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.AwsAuthenticationOptions; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.personalizeevents.PersonalizeEventsClient; +import software.amazon.awssdk.services.personalizeevents.PersonalizeEventsClientBuilder; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ClientFactoryTest { + @Mock + private PersonalizeSinkConfiguration personalizeSinkConfig; + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + + @Mock + private AwsAuthenticationOptions awsAuthenticationOptions; + + @BeforeEach + void setUp() { + when(personalizeSinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + } + + @Test + void createPersonalizeEventsClient_with_real_PersonalizeEventsClient() { + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Optional.of(Region.US_EAST_1)); + final PersonalizeEventsClient personalizeEventsClient = ClientFactory.createPersonalizeEventsClient(personalizeSinkConfig, awsCredentialsSupplier); + + assertThat(personalizeEventsClient, notNullValue()); + } + + @Test + void createPersonalizeEventsClient_provides_correct_inputs_for_null_awsAuthenticationOptions() { + when(personalizeSinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + final AwsCredentialsProvider expectedCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(any())).thenReturn(expectedCredentialsProvider); + + final PersonalizeEventsClientBuilder personalizeEventsClientBuilder = mock(PersonalizeEventsClientBuilder.class); + when(personalizeEventsClientBuilder.region(any())).thenReturn(personalizeEventsClientBuilder); + when(personalizeEventsClientBuilder.credentialsProvider(any())).thenReturn(personalizeEventsClientBuilder); + when(personalizeEventsClientBuilder.overrideConfiguration(any(ClientOverrideConfiguration.class))).thenReturn(personalizeEventsClientBuilder); + try(final MockedStatic personalizeEventsClientMockedStatic = mockStatic(PersonalizeEventsClient.class)) { + personalizeEventsClientMockedStatic.when(PersonalizeEventsClient::builder) + .thenReturn(personalizeEventsClientBuilder); + ClientFactory.createPersonalizeEventsClient(personalizeSinkConfig, awsCredentialsSupplier); + } + + final ArgumentCaptor credentialsProviderArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsProvider.class); + verify(personalizeEventsClientBuilder).credentialsProvider(credentialsProviderArgumentCaptor.capture()); + + final AwsCredentialsProvider actualCredentialsProvider = credentialsProviderArgumentCaptor.getValue(); + + assertThat(actualCredentialsProvider, equalTo(expectedCredentialsProvider)); + + final ArgumentCaptor optionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); + verify(awsCredentialsSupplier).getProvider(optionsArgumentCaptor.capture()); + + final AwsCredentialsOptions actualCredentialsOptions = optionsArgumentCaptor.getValue(); + assertThat(actualCredentialsOptions, is(notNullValue())); + assertThat(actualCredentialsOptions.getRegion(), equalTo(null)); + assertThat(actualCredentialsOptions.getStsRoleArn(), equalTo(null)); + assertThat(actualCredentialsOptions.getStsExternalId(), equalTo(null)); + assertThat(actualCredentialsOptions.getStsHeaderOverrides(), equalTo(Collections.emptyMap())); + } + + @ParameterizedTest + @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) + void createPersonalizeEventsClient_provides_correct_inputs(final String regionString) { + final Region region = Region.of(regionString); + final String stsRoleArn = UUID.randomUUID().toString(); + final String externalId = UUID.randomUUID().toString(); + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Optional.of(region)); + when(awsAuthenticationOptions.getAwsStsRoleArn()).thenReturn(stsRoleArn); + when(awsAuthenticationOptions.getAwsStsExternalId()).thenReturn(externalId); + when(awsAuthenticationOptions.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); + + final AwsCredentialsProvider expectedCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(any())).thenReturn(expectedCredentialsProvider); + + final PersonalizeEventsClientBuilder personalizeEventsClientBuilder = mock(PersonalizeEventsClientBuilder.class); + when(personalizeEventsClientBuilder.region(region)).thenReturn(personalizeEventsClientBuilder); + when(personalizeEventsClientBuilder.credentialsProvider(any())).thenReturn(personalizeEventsClientBuilder); + when(personalizeEventsClientBuilder.overrideConfiguration(any(ClientOverrideConfiguration.class))).thenReturn(personalizeEventsClientBuilder); + try(final MockedStatic personalizeEventsClientMockedStatic = mockStatic(PersonalizeEventsClient.class)) { + personalizeEventsClientMockedStatic.when(PersonalizeEventsClient::builder) + .thenReturn(personalizeEventsClientBuilder); + ClientFactory.createPersonalizeEventsClient(personalizeSinkConfig, awsCredentialsSupplier); + } + + final ArgumentCaptor credentialsProviderArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsProvider.class); + verify(personalizeEventsClientBuilder).credentialsProvider(credentialsProviderArgumentCaptor.capture()); + + final AwsCredentialsProvider actualCredentialsProvider = credentialsProviderArgumentCaptor.getValue(); + + assertThat(actualCredentialsProvider, equalTo(expectedCredentialsProvider)); + + final ArgumentCaptor optionsArgumentCaptor = ArgumentCaptor.forClass(AwsCredentialsOptions.class); + verify(awsCredentialsSupplier).getProvider(optionsArgumentCaptor.capture()); + + final AwsCredentialsOptions actualCredentialsOptions = optionsArgumentCaptor.getValue(); + assertThat(actualCredentialsOptions.getRegion(), equalTo(region)); + assertThat(actualCredentialsOptions.getStsRoleArn(), equalTo(stsRoleArn)); + assertThat(actualCredentialsOptions.getStsExternalId(), equalTo(externalId)); + assertThat(actualCredentialsOptions.getStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkTest.java new file mode 100644 index 0000000000..852e75630f --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/PersonalizeSinkTest.java @@ -0,0 +1,85 @@ +package org.opensearch.dataprepper.plugins.sink.personalize; + +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.SinkContext; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.AwsAuthenticationOptions; +import org.opensearch.dataprepper.plugins.sink.personalize.configuration.PersonalizeSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.personalize.dataset.DatasetTypeOptions; +import software.amazon.awssdk.regions.Region; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class PersonalizeSinkTest { + public static final int MAX_RETRIES = 10; + public static final String REGION = "us-east-1"; + public static final String SINK_PLUGIN_NAME = "personalize"; + public static final String SINK_PIPELINE_NAME = "personalize-sink-pipeline"; + public static final String DATASET_ARN = "arn:aws:iam::123456789012:dataset/test"; + public static final String TRACKING_ID = "1233513241"; + private PersonalizeSinkConfiguration personalizeSinkConfig; + private PersonalizeSink personalizeSink; + private PluginSetting pluginSetting; + private PluginFactory pluginFactory; + private AwsCredentialsSupplier awsCredentialsSupplier; + private SinkContext sinkContext; + + @BeforeEach + void setup() { + personalizeSinkConfig = mock(PersonalizeSinkConfiguration.class); + sinkContext = mock(SinkContext.class); + AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + pluginSetting = mock(PluginSetting.class); + pluginFactory = mock(PluginFactory.class); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + + when(personalizeSinkConfig.getMaxRetries()).thenReturn(MAX_RETRIES); + when(personalizeSinkConfig.getDatasetArn()).thenReturn(DATASET_ARN); + when(personalizeSinkConfig.getDatasetType()).thenReturn(DatasetTypeOptions.USERS); + when(personalizeSinkConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Optional.of(Region.of(REGION))); + when(pluginSetting.getName()).thenReturn(SINK_PLUGIN_NAME); + when(pluginSetting.getPipelineName()).thenReturn(SINK_PIPELINE_NAME); + } + + private PersonalizeSink createObjectUnderTest() { + return new PersonalizeSink(pluginSetting, personalizeSinkConfig, pluginFactory, sinkContext, awsCredentialsSupplier); + } + + @Test + void test_personalize_sink_plugin_isReady_positive() { + personalizeSink = createObjectUnderTest(); + Assertions.assertNotNull(personalizeSink); + personalizeSink.doInitialize(); + assertTrue(personalizeSink.isReady(), "Expected the personalize sink to be ready, but it is reporting it is not ready."); + } + + @Test + void test_personalize_Sink_plugin_isReady_negative() { + personalizeSink = createObjectUnderTest(); + Assertions.assertNotNull(personalizeSink); + assertFalse(personalizeSink.isReady(), "Expected the personalize sink to report that it is not ready, but it is reporting it is ready."); + } + + @Test + void test_doOutput_with_empty_records() { + personalizeSink = createObjectUnderTest(); + Assertions.assertNotNull(personalizeSink); + personalizeSink.doInitialize(); + Collection> records = new ArrayList<>(); + personalizeSink.doOutput(records); + } +} diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptionsTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptionsTest.java new file mode 100644 index 0000000000..29be309622 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/AwsAuthenticationOptionsTest.java @@ -0,0 +1,129 @@ +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import software.amazon.awssdk.regions.Region; + +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.assertFalse; + +class AwsAuthenticationOptionsTest { + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @ParameterizedTest + @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) + void getAwsRegion_returns_Region_of(final String regionString) { + final Optional expectedRegionObject = Optional.of(Region.of(regionString)); + final Map jsonMap = Map.of("region", regionString); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsRegion(), equalTo(expectedRegionObject)); + } + + @Test + void getAwsRegion_returns_null_when_region_is_null() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsRegion(), equalTo(Optional.empty())); + } + + @Test + void getAwsStsRoleArn_returns_value_from_deserialized_JSON() { + final String stsRoleArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), equalTo(stsRoleArn)); + } + + @Test + void getAwsStsRoleArn_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), nullValue()); + } + + @Test + void isValidStsRoleArn_returns_true_for_valid_IAM_role() { + final String stsRoleArn = "arn:aws:iam::123456789012:role/test"; + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertTrue(objectUnderTest.isValidStsRoleArn()); + } + + @Test + void isValidStsRoleArn_returns_true_for_null() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertTrue(objectUnderTest.isValidStsRoleArn()); + } + + @Test + void isValidStsRoleArn_returns_false_when_arn_service_is_not_IAM() { + final String stsRoleArn = "arn:aws:personalize::123456789012:role/test"; + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertFalse(objectUnderTest.isValidStsRoleArn()); + } + + @Test + void isValidStsRoleArn_returns_false_when_arn_resource_is_not_role() { + final String stsRoleArn = "arn:aws:iam::123456789012:dataset/test"; + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertFalse(objectUnderTest.isValidStsRoleArn()); + } + + @Test + void isValidStsRoleArn_invalid_arn_throws_IllegalArgumentException() { + final String stsRoleArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThrows(IllegalArgumentException.class, () -> objectUnderTest.isValidStsRoleArn()); + } + + @Test + void getAwsStsExternalId_returns_value_from_deserialized_JSON() { + final String stsExternalId = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_external_id", stsExternalId); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsExternalId(), equalTo(stsExternalId)); + } + + @Test + void getAwsStsExternalId_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsExternalId(), nullValue()); + } + + @Test + void getAwsStsHeaderOverrides_returns_value_from_deserialized_JSON() { + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + final Map jsonMap = Map.of("sts_header_overrides", stsHeaderOverrides); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } + + @Test + void getAwsStsHeaderOverrides_returns_null_if_not_in_JSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationOptions objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationOptions.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), nullValue()); + } +} diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfigurationTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfigurationTest.java new file mode 100644 index 0000000000..67bc690623 --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/configuration/PersonalizeSinkConfigurationTest.java @@ -0,0 +1,205 @@ +package org.opensearch.dataprepper.plugins.sink.personalize.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.sink.personalize.dataset.DatasetTypeOptions; + +import java.util.Collections; +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class PersonalizeSinkConfigurationTest { + private static final int DEFAULT_RETRIES = 10; + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + objectMapper = new ObjectMapper(); + } + + @Test + void getDatasetType_returns_value_from_deserialized_JSON() { + final String datasetType = "users"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDatasetType(), equalTo(DatasetTypeOptions.USERS)); + } + + @Test + void getDatasetArn_returns_null_when_datasetArn_is_null() { + final Map jsonMap = Collections.emptyMap(); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDatasetArn(), nullValue()); + } + + @Test + void getDatasetArn_returns_value_from_deserialized_JSON() { + final String datasetArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDatasetArn(), equalTo(datasetArn)); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_true_when_datasetType_is_interactions_and_datasetArn_is_null() { + final String datasetType = "interactions"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_true_when_datasetType_is_users_and_datasetArn_is_provided() { + final String datasetType = "users"; + final String datasetArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_type", datasetType, "dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_false_when_datasetType_is_users_and_datasetArn_is_not_provided() { + final String datasetType = "users"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_true_when_datasetType_is_items_and_datasetArn_is_provided() { + final String datasetType = "items"; + final String datasetArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_type", datasetType, "dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isDatasetArnProvidedWhenNeeded_returns_false_when_datasetType_is_items_and_datasetArn_is_not_provided() { + final String datasetType = "items"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isDatasetArnProvidedWhenNeeded()); + } + + @Test + void isValidDatasetArn_returns_true_for_valid_dataset_arn() { + final String datasetArn = "arn:aws:personalize::123456789012:dataset/test"; + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isValidDatasetArn()); + } + + @Test + void isValidDatasetArn_returns_false_when_arn_service_is_not_personalize() { + final String datasetArn = "arn:aws:iam::123456789012:dataset/test"; + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isValidDatasetArn()); + } + + @Test + void isValidDatasetArn_returns_false_when_arn_resource_is_not_dataset() { + final String datasetArn = "arn:aws:personalize::123456789012:role/test"; + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isValidDatasetArn()); + } + + @Test + void isValidStsRoleArn_invalid_arn_throws_IllegalArgumentException() { + final String datasetArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_arn", datasetArn); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThrows(IllegalArgumentException.class, () -> objectUnderTest.isValidDatasetArn()); + } + + + + @Test + void getTrackingId_returns_null_when_trackingId_is_null() { + final Map jsonMap = Collections.emptyMap(); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getTrackingId(), nullValue()); + } + + @Test + void getTrackingId_returns_value_from_deserialized_JSON() { + final String trackingId = UUID.randomUUID().toString();; + final Map jsonMap = Map.of("tracking_id", trackingId); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getTrackingId(), equalTo(trackingId)); + } + + @Test + void isTrackingIdProvidedWhenNeeded_returns_false_when_datasetType_is_interactions_and_trackingId_is_not_provided() { + final String datasetType = "interactions"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertFalse(objectUnderTest.isTrackingIdProvidedWhenNeeded()); + } + + @Test + void isTrackingIdProvidedWhenNeeded_returns_true_when_datasetType_is_interactions_and_trackingId_is_provided() { + final String datasetType = "interactions"; + final String trackingId = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("dataset_type", datasetType, "tracking_id", trackingId); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isTrackingIdProvidedWhenNeeded()); + } + + @Test + void isTrackingIdProvidedWhenNeeded_returns_true_when_datasetType_is_users_and_trackingId_is_not_provided() { + final String datasetType = "users"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isTrackingIdProvidedWhenNeeded()); + } + + @Test + void isTrackingIdProvidedWhenNeeded_returns_true_when_datasetType_is_items_and_trackingId_is_not_provided() { + final String datasetType = "items"; + final Map jsonMap = Map.of("dataset_type", datasetType); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertTrue(objectUnderTest.isTrackingIdProvidedWhenNeeded()); + } + + + @Test + void getDocumentRootKey_returns_null_when_documentRootKey_is_null() { + final Map jsonMap = Collections.emptyMap(); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDocumentRootKey(), nullValue()); + } + + @Test + void getDocumentRootKey_returns_value_from_deserialized_JSON() { + final String documentRootKey = UUID.randomUUID().toString();; + final Map jsonMap = Map.of("document_root_key", documentRootKey); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getDocumentRootKey(), equalTo(documentRootKey)); + } + + @Test + void getMaxRetries_returns_default_when_maxRetries_is_null() { + final Map jsonMap = Collections.emptyMap(); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getMaxRetries(), equalTo(DEFAULT_RETRIES)); + } + + @Test + void getMaxRetries_returns_value_from_deserialized_JSON() { + final int maxRetries = 3; + final Map jsonMap = Map.of("max_retries", maxRetries); + final PersonalizeSinkConfiguration objectUnderTest = objectMapper.convertValue(jsonMap, PersonalizeSinkConfiguration.class); + assertThat(objectUnderTest.getMaxRetries(), equalTo(maxRetries)); + } +} diff --git a/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptionsTest.java b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptionsTest.java new file mode 100644 index 0000000000..40b1821d0a --- /dev/null +++ b/data-prepper-plugins/personalize-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/personalize/dataset/DatasetTypeOptionsTest.java @@ -0,0 +1,38 @@ +package org.opensearch.dataprepper.plugins.sink.personalize.dataset; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +@ExtendWith(MockitoExtension.class) +class DatasetTypeOptionsTest { + @Test + void notNull_test() { + assertNotNull(DatasetTypeOptions.ITEMS); + } + + @Test + void fromOptionValue_users_test() { + DatasetTypeOptions datasetTypeOptions = DatasetTypeOptions.fromOptionValue("users"); + assertNotNull(datasetTypeOptions); + assertThat(datasetTypeOptions.toString(), equalTo("USERS")); + } + + @Test + void fromOptionValue_items_test() { + DatasetTypeOptions datasetTypeOptions = DatasetTypeOptions.fromOptionValue("items"); + assertNotNull(datasetTypeOptions); + assertThat(datasetTypeOptions.toString(), equalTo("ITEMS")); + } + + @Test + void fromOptionValue_interactions_test() { + DatasetTypeOptions datasetTypeOptions = DatasetTypeOptions.fromOptionValue("interactions"); + assertNotNull(datasetTypeOptions); + assertThat(datasetTypeOptions.toString(), equalTo("INTERACTIONS")); + } +} diff --git a/settings.gradle b/settings.gradle index 18ccd4dc7b..63e7ad2a9f 100644 --- a/settings.gradle +++ b/settings.gradle @@ -170,6 +170,7 @@ include 'data-prepper-plugins:buffer-common' //include 'data-prepper-plugins:http-sink' //include 'data-prepper-plugins:sns-sink' //include 'data-prepper-plugins:prometheus-sink' +include 'data-prepper-plugins:personalize-sink' include 'data-prepper-plugins:dissect-processor' include 'data-prepper-plugins:dynamodb-source' include 'data-prepper-plugins:decompress-processor' From 00cc2a576ecb7bb0e624e2ee77d7eaa28644c7c6 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Wed, 14 Aug 2024 11:38:44 -0500 Subject: [PATCH 111/159] FIX: include schema cli into release (#4833) MAINT: include schema cli into release Signed-off-by: George Chen --- build-resources.gradle | 5 +++-- data-prepper-plugin-schema-cli/build.gradle | 11 +++++++++++ 2 files changed, 14 insertions(+), 2 deletions(-) diff --git a/build-resources.gradle b/build-resources.gradle index b8a7bd5879..446f1d97b2 100644 --- a/build-resources.gradle +++ b/build-resources.gradle @@ -14,5 +14,6 @@ ext.coreProjects = [ project(':data-prepper-plugins'), project(':data-prepper-test-common'), project(':data-prepper-test-event'), - project(':data-prepper-plugin-framework') -] \ No newline at end of file + project(':data-prepper-plugin-framework'), + project(':data-prepper-plugin-schema-cli') +] diff --git a/data-prepper-plugin-schema-cli/build.gradle b/data-prepper-plugin-schema-cli/build.gradle index 2c2db93ee6..2108fad681 100644 --- a/data-prepper-plugin-schema-cli/build.gradle +++ b/data-prepper-plugin-schema-cli/build.gradle @@ -26,4 +26,15 @@ dependencies { } testImplementation(platform("org.junit:junit-bom:5.9.1")) testImplementation("org.junit.jupiter:junit-jupiter") +} + +jacocoTestCoverageVerification { + afterEvaluate { + classDirectories.from = files(classDirectories.files.collect { + fileTree(dir: it, exclude: [ + // Exclude main class + 'org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.class' + ]) + }) + } } \ No newline at end of file From 1bfed0d11d8d0b436ed96867d715dcb6b3aa17f8 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 14 Aug 2024 11:49:12 -0700 Subject: [PATCH 112/159] Http chunking fixes (#4823) * dplive1.yaml Signed-off-by: Krishna Kondaka * Delete .github/workflows/static.yml Signed-off-by: Krishna Kondaka * Fix http message chunking bug Signed-off-by: Krishna Kondaka * Modified tests to test for chunks correctly Signed-off-by: Krishna Kondaka * Added comments Signed-off-by: Krishna Kondaka * Addressed offline review comments Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka * Added tests Signed-off-by: Krishna Kondaka * Added tests Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../dataprepper/model/buffer/Buffer.java | 14 ++++ .../dataprepper/model/buffer/BufferTest.java | 6 ++ .../parser/MultiBufferDecorator.java | 6 ++ .../parser/MultiBufferDecoratorTest.java | 11 ++- .../dataprepper/http/codec/JsonCodec.java | 12 ++- .../dataprepper/http/codec/JsonCodecTest.java | 77 ++++++++++++++---- .../source/loghttp/LogHTTPService.java | 23 +++++- .../source/loghttp/LogHTTPServiceTest.java | 81 ++++++++++++++++++- .../plugins/kafka/buffer/KafkaBuffer.java | 6 ++ 9 files changed, 210 insertions(+), 26 deletions(-) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/buffer/Buffer.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/buffer/Buffer.java index eaaa978230..874a9d350e 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/buffer/Buffer.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/buffer/Buffer.java @@ -84,10 +84,24 @@ default boolean isByteBuffer() { return false; } + /** + * returns max request size of an entry in the buffer + * + * @return Optional value of the buffer's max request size + */ default Optional getMaxRequestSize() { return Optional.empty(); } + /** + * returns optimal request size of an entry in the buffer + * + * @return Optional value of the buffer's optimal request size + */ + default Optional getOptimalRequestSize() { + return Optional.empty(); + } + /** * Checks if the buffer enables acknowledgements for the pipeline * diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/buffer/BufferTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/buffer/BufferTest.java index 2236f0ba33..0d9aa51296 100644 --- a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/buffer/BufferTest.java +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/buffer/BufferTest.java @@ -37,6 +37,12 @@ void testMaxRequestSize() { assertEquals(buffer.getMaxRequestSize(), Optional.empty()); } + @Test + void testOptimalRequestSize() { + final Buffer> buffer = createObjectUnderTest(); + assertEquals(buffer.getOptimalRequestSize(), Optional.empty()); + } + @Test void testShutdown() { final Buffer> buffer = createObjectUnderTest(); diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/MultiBufferDecorator.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/MultiBufferDecorator.java index eaa6c09491..76440c0c56 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/MultiBufferDecorator.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/MultiBufferDecorator.java @@ -50,6 +50,12 @@ public Optional getMaxRequestSize() { return maxRequestSize.isPresent() ? Optional.of(maxRequestSize.getAsInt()) : Optional.empty(); } + @Override + public Optional getOptimalRequestSize() { + OptionalInt optimalRequestSize = allBuffers.stream().filter(b -> b.getOptimalRequestSize().isPresent()).mapToInt(b -> (Integer)b.getOptimalRequestSize().get()).min(); + return optimalRequestSize.isPresent() ? Optional.of(optimalRequestSize.getAsInt()) : Optional.empty(); + } + @Override public void shutdown() { allBuffers.forEach(Buffer::shutdown); diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/MultiBufferDecoratorTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/MultiBufferDecoratorTest.java index bae4cb763b..896268fcf4 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/MultiBufferDecoratorTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/MultiBufferDecoratorTest.java @@ -231,6 +231,15 @@ void test_getMaxRequestSize() { assertThat(multiBufferDecorator.getMaxRequestSize(), equalTo(Optional.empty())); } + @Test + void test_getOptimalRequestSize() { + when(primaryBuffer.getOptimalRequestSize()).thenReturn(Optional.empty()); + when(secondaryBuffer.getOptimalRequestSize()).thenReturn(Optional.empty()); + + final MultiBufferDecorator multiBufferDecorator = createObjectUnderTest(2); + assertThat(multiBufferDecorator.getOptimalRequestSize(), equalTo(Optional.empty())); + } + private MultiBufferDecorator createObjectUnderTest(final int secondaryBufferCount) { final List secondaryBuffers = IntStream.range(0, secondaryBufferCount) .mapToObj(i -> secondaryBuffer) @@ -238,4 +247,4 @@ private MultiBufferDecorator createObjectUnderTest(final int secondaryBufferCoun return new MultiBufferDecorator(primaryBuffer, secondaryBuffers); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java index 4c0020a83e..9a74cdb767 100644 --- a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java @@ -47,17 +47,21 @@ public List> parse(HttpData httpData, int maxSize) throws IOExcepti List> jsonList = new ArrayList<>(); final List> logList = mapper.readValue(httpData.toInputStream(), LIST_OF_MAP_TYPE_REFERENCE); - int size = OVERHEAD_CHARACTERS.length(); List innerJsonList = new ArrayList<>(); - for (final Map log: logList) { + int size = OVERHEAD_CHARACTERS.length(); + for (Map log: logList) { final String recordString = mapper.writeValueAsString(log); - if (size + recordString.length() > maxSize) { + final int nextRecordLength = recordString.getBytes(Charset.defaultCharset()).length; + // It is possible that the first record is larger than maxSize, then + // innerJsonList size would be zero. + if (size + nextRecordLength > maxSize && !innerJsonList.isEmpty()) { jsonList.add(innerJsonList); innerJsonList = new ArrayList<>(); size = OVERHEAD_CHARACTERS.length(); } + // The following may result in a innerJsonList with larger than "maxSize" length recordString innerJsonList.add(recordString); - size += recordString.getBytes(Charset.defaultCharset()).length + COMMA_OVERHEAD_LENGTH; + size += nextRecordLength + COMMA_OVERHEAD_LENGTH; } if (size > OVERHEAD_CHARACTERS.length()) { jsonList.add(innerJsonList); diff --git a/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java index 8843d8d6e7..ec095e04e2 100644 --- a/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java +++ b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java @@ -24,6 +24,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.params.provider.Arguments.arguments; @@ -69,20 +70,28 @@ public void testParseSuccessWithMaxSize() throws IOException { @ParameterizedTest @ArgumentsSource(JsonArrayWithKnownFirstArgumentsProvider.class) public void parse_should_return_lists_smaller_than_provided_length( - final String inputJsonArray, final String knownFirstPart) throws IOException { + final String inputJsonArray, final String knownFirstPart, final int maxSize, final List> expectedChunks, final List exceedsMaxSize) throws IOException { final int knownSingleBodySize = knownFirstPart.getBytes(Charset.defaultCharset()).length; - final int maxSize = (knownSingleBodySize * 2) + 3; final List> chunkedBodies = objectUnderTest.parse(HttpData.ofUtf8(inputJsonArray), maxSize); assertThat(chunkedBodies, notNullValue()); - assertThat(chunkedBodies.size(), greaterThanOrEqualTo(1)); - final String firstReconstructed = chunkedBodies.get(0).stream().collect(Collectors.joining(",", "[", "]")); - assertThat(firstReconstructed.getBytes(Charset.defaultCharset()).length, - lessThanOrEqualTo(maxSize)); - - assertThat(chunkedBodies.get(0).size(), greaterThanOrEqualTo(1)); - assertThat(chunkedBodies.get(0).get(0), equalTo(knownFirstPart)); + assertThat(chunkedBodies.size(), equalTo(expectedChunks.size())); + + for (int i = 0; i < expectedChunks.size(); i++) { + final String reconstructed = chunkedBodies.get(i).stream().collect(Collectors.joining(",", "[", "]")); + if (exceedsMaxSize.get(i)) { + assertThat(reconstructed.getBytes(Charset.defaultCharset()).length, + greaterThanOrEqualTo(maxSize)); + } else { + assertThat(reconstructed.getBytes(Charset.defaultCharset()).length, + lessThanOrEqualTo(maxSize)); + } + + for (int j = 0; j < expectedChunks.get(i).size(); j++) { + assertThat(chunkedBodies.get(i).get(j), equalTo(expectedChunks.get(i).get(j))); + } + } } @Test @@ -103,14 +112,50 @@ public void testParseNonJsonFailure() { static class JsonArrayWithKnownFirstArgumentsProvider implements ArgumentsProvider { @Override public Stream provideArguments(ExtensionContext extensionContext) throws Exception { + // First test, all chunks smaller than maxSize, output has 3 lists, all smaller than maxSize + String chunk11 = "{\"ὊὊὊ1\":\"ὊὊὊ1\"}"; + String chunk12 = "{\"ὊὊὊ2\":\"ὊὊὊO2\"}"; + String chunk13 = "{\"a3\":\"b3\"}"; + String chunk14 = "{\"ὊὊὊ4\":\"ὊὊὊ4\"}"; + // Second test, all chunks smaller than maxSize, output has 2 lists, all smaller than maxSize + String chunk21 = "{\"aaa1\":\"aaa1\"}"; + String chunk22 = "{\"aaa2\":\"aaa2\"}"; + String chunk23 = "{\"a3\":\"b3\"}"; + String chunk24 = "{\"bbb4\":\"bbb4\"}"; + // Third test, all chunks larger than maxSize, output has 4 lists, all larger than maxSize + String chunk31 = "{\"ὊὊὊ1\":\"ὊὊὊ01\"}"; + String chunk32 = "{\"ὊὊὊ2\":\"ὊὊὊO2\"}"; + String chunk33 = "{\"ὊὊὊ3\":\"ὊὊὊO3\"}"; + String chunk34 = "{\"ὊὊὊ4\":\"ὊὊὊO4\"}"; + // Fourth test, only first chunk larger than maxSize, output has 3 lists, with first chunk larger than maxSize and others smaller + String chunk41 = "{\"aaaaaaaaaaa1\":\"aaaaaaaaaaa1\"}"; + String chunk42 = "{\"aaa2\":\"aaa2\"}"; + String chunk43 = "{\"a3\":\"b3\"}"; + String chunk44 = "{\"bbb4\":\"bbb4\"}"; + // Fifth test, only second chunk larger than maxSize, output has 3 lists, with second chunk larger than maxSize and others smaller + String chunk51 = "{\"aaa2\":\"aaa2\"}"; + String chunk52 = "{\"aaaaaaaaaaa1\":\"aaaaaaaaaaa1\"}"; + String chunk53 = "{\"a3\":\"b3\"}"; + String chunk54 = "{\"bb4\":\"bb4\"}"; + // Sixth test, only last chunk larger than maxSize, output has 3 lists, with last chunk larger than maxSize and others smaller + String chunk61 = "{\"aaa2\":\"aaa2\"}"; + String chunk62 = "{\"a3\":\"b3\"}"; + String chunk63 = "{\"bbb4\":\"bbb4\"}"; + String chunk64 = "{\"aaaaaaaaaaa1\":\"aaaaaaaaaaa1\"}"; + final int maxSize1 = chunk11.getBytes(Charset.defaultCharset()).length * 2 + 3; + final int maxSize2 = chunk21.getBytes(Charset.defaultCharset()).length * 2 + 3; + final int maxSize3 = chunk31.getBytes(Charset.defaultCharset()).length - 1; + final int maxSize4 = chunk42.getBytes(Charset.defaultCharset()).length * 2 + 3; + final int maxSize5 = chunk51.getBytes(Charset.defaultCharset()).length * 2 + 3; + final int maxSize6 = chunk61.getBytes(Charset.defaultCharset()).length * 2 + 3; return Stream.of( - arguments( - "[{\"ὊὊὊ1\":\"ὊὊὊ1\"}, {\"ὊὊὊ2\":\"ὊὊὊ2\"}, {\"a3\":\"b3\"}, {\"ὊὊὊ4\":\"ὊὊὊ4\"}]", - "{\"ὊὊὊ1\":\"ὊὊὊ1\"}"), - arguments( - "[{\"aaa1\":\"aaa1\"}, {\"aaa2\":\"aaa2\"}, {\"a3\":\"b3\"}, {\"bbb4\":\"bbb4\"}]", - "{\"aaa1\":\"aaa1\"}") + arguments("["+chunk11+","+chunk12+","+chunk13+","+chunk14+"]", chunk11, maxSize1, List.of(List.of(chunk11), List.of(chunk12, chunk13), List.of(chunk14)), List.of(false, false, false)), + arguments("["+chunk21+","+chunk22+","+chunk23+","+chunk24+"]", chunk21, maxSize2, List.of(List.of(chunk21, chunk22), List.of(chunk23, chunk24)), List.of(false, false)), + arguments("["+chunk31+","+chunk32+","+chunk33+","+chunk34+"]", chunk31, maxSize3, List.of(List.of(chunk31), List.of(chunk32), List.of(chunk33), List.of(chunk34)), List.of(true, true, true, true)), + arguments("["+chunk41+","+chunk42+","+chunk43+","+chunk44+"]", chunk41, maxSize4, List.of(List.of(chunk41), List.of(chunk42, chunk43), List.of(chunk44)), List.of(true, false, false)), + arguments("["+chunk51+","+chunk52+","+chunk53+","+chunk54+"]", chunk51, maxSize5, List.of(List.of(chunk51), List.of(chunk52), List.of(chunk53,chunk54)), List.of(false, true, false)), + arguments("["+chunk61+","+chunk62+","+chunk63+","+chunk64+"]", chunk61, maxSize6, List.of(List.of(chunk61,chunk62), List.of(chunk63), List.of(chunk64)), List.of(false, false, true)) ); } } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java index 8384315aa4..1eac99ed92 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java +++ b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java @@ -39,6 +39,8 @@ public class LogHTTPService { private static final int SERIALIZATION_OVERHEAD = 1024; public static final String REQUESTS_RECEIVED = "requestsReceived"; public static final String SUCCESS_REQUESTS = "successRequests"; + public static final String REQUESTS_OVER_OPTIMAL_SIZE = "requestsOverOptimalSize"; + public static final String REQUESTS_OVER_MAXIMUM_SIZE = "requestsOverMaximumSize"; public static final String PAYLOAD_SIZE = "payloadSize"; public static final String REQUEST_PROCESS_DURATION = "requestProcessDuration"; @@ -50,9 +52,12 @@ public class LogHTTPService { private final int bufferWriteTimeoutInMillis; private final Counter requestsReceivedCounter; private final Counter successRequestsCounter; + private final Counter requestsOverOptimalSizeCounter; + private final Counter requestsOverMaximumSizeCounter; private final DistributionSummary payloadSizeSummary; private final Timer requestProcessDuration; private Integer maxRequestLength; + private Integer optimalRequestLength; public LogHTTPService(final int bufferWriteTimeoutInMillis, final Buffer> buffer, @@ -61,8 +66,11 @@ public LogHTTPService(final int bufferWriteTimeoutInMillis, this.buffer = buffer; this.bufferWriteTimeoutInMillis = bufferWriteTimeoutInMillis; this.maxRequestLength = buffer.getMaxRequestSize().isPresent() ? buffer.getMaxRequestSize().get(): null; + this.optimalRequestLength = buffer.getOptimalRequestSize().isPresent() ? buffer.getOptimalRequestSize().get(): null; requestsReceivedCounter = pluginMetrics.counter(REQUESTS_RECEIVED); successRequestsCounter = pluginMetrics.counter(SUCCESS_REQUESTS); + requestsOverOptimalSizeCounter = pluginMetrics.counter(REQUESTS_OVER_OPTIMAL_SIZE); + requestsOverMaximumSizeCounter = pluginMetrics.counter(REQUESTS_OVER_MAXIMUM_SIZE); payloadSizeSummary = pluginMetrics.summary(PAYLOAD_SIZE); requestProcessDuration = pluginMetrics.timer(REQUEST_PROCESS_DURATION); } @@ -91,24 +99,33 @@ private void sendJsonList(List jsonList) throws Exception { } sb.append("]"); if (sb.toString().getBytes().length > maxRequestLength) { + requestsOverMaximumSizeCounter.increment(); throw new RuntimeException("Request length "+ sb.toString().getBytes().length + " exceeds maxRequestLength "+ maxRequestLength); + } else if (sb.toString().getBytes().length > optimalRequestLength) { + requestsOverOptimalSizeCounter.increment(); } buffer.writeBytes(sb.toString().getBytes(), key, bufferWriteTimeoutInMillis); } - private HttpResponse processRequest(final AggregatedHttpRequest aggregatedHttpRequest) throws Exception { + HttpResponse processRequest(final AggregatedHttpRequest aggregatedHttpRequest) throws Exception { final HttpData content = aggregatedHttpRequest.content(); List> jsonList; + boolean isJsonListSplit = false; try { - jsonList = (maxRequestLength == null) ? jsonCodec.parse(content) : jsonCodec.parse(content, maxRequestLength - SERIALIZATION_OVERHEAD); + if (buffer.isByteBuffer() && maxRequestLength != null && optimalRequestLength != null) { + jsonList = jsonCodec.parse(content, optimalRequestLength - SERIALIZATION_OVERHEAD); + isJsonListSplit = true; + } else { + jsonList = jsonCodec.parse(content); + } } catch (IOException e) { LOG.error("Failed to parse the request of size {} due to: {}", content.length(), e.getMessage()); throw new IOException("Bad request data format. Needs to be json array.", e.getCause()); } try { if (buffer.isByteBuffer()) { - if (maxRequestLength != null && content.array().length > maxRequestLength) { + if (isJsonListSplit && content.array().length > optimalRequestLength) { for (final List innerJsonList: jsonList) { sendJsonList(innerJsonList); } diff --git a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPServiceTest.java b/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPServiceTest.java index bd289de769..2e9b802f32 100644 --- a/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPServiceTest.java +++ b/data-prepper-plugins/http-source/src/test/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPServiceTest.java @@ -40,10 +40,14 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; +import java.nio.charset.StandardCharsets; +import java.io.ByteArrayInputStream; +import java.io.InputStream; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -51,6 +55,11 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.isNull; @ExtendWith(MockitoExtension.class) class LogHTTPServiceTest { @@ -78,14 +87,17 @@ class LogHTTPServiceTest { private LogHTTPService logHTTPService; + @Mock + private Buffer> byteBuffer; + @BeforeEach public void setUp() throws Exception { when(pluginMetrics.counter(LogHTTPService.REQUESTS_RECEIVED)).thenReturn(requestsReceivedCounter); when(pluginMetrics.counter(LogHTTPService.SUCCESS_REQUESTS)).thenReturn(successRequestsCounter); when(pluginMetrics.summary(LogHTTPService.PAYLOAD_SIZE)).thenReturn(payloadSizeSummary); when(pluginMetrics.timer(LogHTTPService.REQUEST_PROCESS_DURATION)).thenReturn(requestProcessDuration); - when(serviceRequestContext.isTimedOut()).thenReturn(false); - when(requestProcessDuration.recordCallable(ArgumentMatchers.>any())).thenAnswer( + lenient().when(serviceRequestContext.isTimedOut()).thenReturn(false); + lenient().when(requestProcessDuration.recordCallable(ArgumentMatchers.>any())).thenAnswer( (Answer) invocation -> { final Object[] args = invocation.getArguments(); @SuppressWarnings("unchecked") @@ -170,6 +182,71 @@ public void testHTTPRequestTimeout() throws Exception { verify(requestProcessDuration, times(2)).recordCallable(ArgumentMatchers.>any()); } + @Test + public void testChunking() throws Exception { + byteBuffer = mock(Buffer.class); + when(byteBuffer.isByteBuffer()).thenReturn(true); + when(byteBuffer.getMaxRequestSize()).thenReturn(Optional.of(4*1024*1024)); + when(byteBuffer.getOptimalRequestSize()).thenReturn(Optional.of(1024*1024)); + + logHTTPService = new LogHTTPService(TEST_TIMEOUT_IN_MILLIS, byteBuffer, null, pluginMetrics); + AggregatedHttpRequest aggregatedHttpRequest = mock(AggregatedHttpRequest.class); + HttpData httpData = mock(HttpData.class); + // Test small json data + String testString ="{\"key1\":\"value1\"},{\"key2\":\"value2\"},{\"key3\":\"value3\"},{\"key4\":\"value4\"},{\"key5\":\"value5\"}"; + String exampleString = "[ " + testString + "]"; + when(httpData.array()).thenReturn(exampleString.getBytes()); + InputStream stream = new ByteArrayInputStream(exampleString.getBytes(StandardCharsets.UTF_8)); + when(httpData.toInputStream()).thenReturn(stream); + + when(aggregatedHttpRequest.content()).thenReturn(httpData); + logHTTPService.processRequest(aggregatedHttpRequest); + verify(byteBuffer, times(1)).writeBytes(any(), (String)isNull(), any(Integer.class)); + + // Test more than 1MB json data + StringBuilder sb = new StringBuilder(1024*1024+10240); + for (int i =0; i < 12500; i++) { + sb.append(testString); + if (i+1 != 12500) + sb.append(","); + } + String largeTestString = sb.toString(); + exampleString = "[" + largeTestString + "]"; + when(httpData.array()).thenReturn(exampleString.getBytes()); + stream = new ByteArrayInputStream(exampleString.getBytes(StandardCharsets.UTF_8)); + when(httpData.toInputStream()).thenReturn(stream); + + when(aggregatedHttpRequest.content()).thenReturn(httpData); + logHTTPService.processRequest(aggregatedHttpRequest); + verify(byteBuffer, times(2)).writeBytes(any(), anyString(), any(Integer.class)); + // Test more than 4MB json data + exampleString = "[" + largeTestString + "," + largeTestString + ","+largeTestString +","+largeTestString+"]"; + when(httpData.array()).thenReturn(exampleString.getBytes()); + stream = new ByteArrayInputStream(exampleString.getBytes(StandardCharsets.UTF_8)); + when(httpData.toInputStream()).thenReturn(stream); + + when(aggregatedHttpRequest.content()).thenReturn(httpData); + logHTTPService.processRequest(aggregatedHttpRequest); + verify(byteBuffer, times(7)).writeBytes(any(), anyString(), any(Integer.class)); + + // Test more than 4MB single json object, should throw exception + int length = 3*1024*1024; + sb = new StringBuilder(length); + for (int i = 0; i < length; i++) { + sb.append('A'); + } + String value = sb.toString(); + exampleString = "[{\"key\":\""+value+"\"}]"; + + when(httpData.array()).thenReturn(exampleString.getBytes()); + stream = new ByteArrayInputStream(exampleString.getBytes(StandardCharsets.UTF_8)); + when(httpData.toInputStream()).thenReturn(stream); + + when(aggregatedHttpRequest.content()).thenReturn(httpData); + assertThrows(RuntimeException.class, () -> logHTTPService.processRequest(aggregatedHttpRequest)); + + } + private AggregatedHttpRequest generateRandomValidHTTPRequest(int numJson) throws JsonProcessingException, ExecutionException, InterruptedException { RequestHeaders requestHeaders = RequestHeaders.builder() diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java index 02e24709eb..f8ec9c4d91 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java @@ -32,6 +32,7 @@ import org.opensearch.dataprepper.plugins.kafka.consumer.KafkaCustomConsumerFactory; import org.opensearch.dataprepper.plugins.kafka.producer.KafkaCustomProducer; import org.opensearch.dataprepper.plugins.kafka.producer.KafkaCustomProducerFactory; +import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaProducerProperties; import org.opensearch.dataprepper.plugins.kafka.service.TopicServiceFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -96,6 +97,11 @@ public Optional getMaxRequestSize() { return Optional.of(producer.getMaxRequestSize()); } + @Override + public Optional getOptimalRequestSize() { + return Optional.of(KafkaProducerProperties.DEFAULT_MAX_REQUEST_SIZE); + } + @Override public void writeBytes(final byte[] bytes, final String key, int timeoutInMillis) throws Exception { try { From 91b6666512805bf502186bd683ae800b0943ba10 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 14 Aug 2024 14:05:26 -0700 Subject: [PATCH 113/159] Data Prepper expressions - Set operator fix (#4818) * dplive1.yaml Signed-off-by: Krishna Kondaka * Delete .github/workflows/static.yml Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka * Fixed check style errors Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../src/main/antlr/DataPrepperExpression.g4 | 6 ++- .../expression/ParseTreeCoercionService.java | 3 ++ .../ParseTreeEvaluatorListener.java | 38 ++++++++++++++++++- ...ericExpressionEvaluator_ConditionalIT.java | 27 +++++++++++++ 4 files changed, 72 insertions(+), 2 deletions(-) diff --git a/data-prepper-expression/src/main/antlr/DataPrepperExpression.g4 b/data-prepper-expression/src/main/antlr/DataPrepperExpression.g4 index e0ac5502b2..a0c5375d37 100644 --- a/data-prepper-expression/src/main/antlr/DataPrepperExpression.g4 +++ b/data-prepper-expression/src/main/antlr/DataPrepperExpression.g4 @@ -122,7 +122,11 @@ regexPattern ; setInitializer - : LBRACE primary (SET_DELIMITER primary)* RBRACE + : LBRACE setMembers RBRACE + ; + +setMembers + : literal (SPACE* SET_DELIMITER SPACE* literal)* ; unaryOperator diff --git a/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/ParseTreeCoercionService.java b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/ParseTreeCoercionService.java index 9459dd7845..692a8c561a 100644 --- a/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/ParseTreeCoercionService.java +++ b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/ParseTreeCoercionService.java @@ -80,6 +80,9 @@ public Object coercePrimaryTerminalNode(final TerminalNode node, final Event eve return Float.valueOf(nodeStringValue); case DataPrepperExpressionParser.Boolean: return Boolean.valueOf(nodeStringValue); + case DataPrepperExpressionParser.COMMA: + case DataPrepperExpressionParser.SET_DELIMITER: + return nodeType; case DataPrepperExpressionParser.Null: return null; case DataPrepperExpressionParser.DataTypes: diff --git a/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/ParseTreeEvaluatorListener.java b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/ParseTreeEvaluatorListener.java index 07f5f7407e..f3c5286b09 100644 --- a/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/ParseTreeEvaluatorListener.java +++ b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/ParseTreeEvaluatorListener.java @@ -14,6 +14,8 @@ import org.opensearch.dataprepper.expression.antlr.DataPrepperExpressionListener; import org.opensearch.dataprepper.expression.antlr.DataPrepperExpressionParser; +import java.util.HashSet; +import java.util.Set; import java.util.Stack; /** @@ -35,6 +37,8 @@ class ParseTreeEvaluatorListener extends DataPrepperExpressionBaseListener { private final Stack operatorSymbolStack; private final Stack operandStack; private final Event event; + private boolean listStart; + private Set setMembers; public ParseTreeEvaluatorListener(final OperatorProvider operatorProvider, final ParseTreeCoercionService coercionService, @@ -42,6 +46,7 @@ public ParseTreeEvaluatorListener(final OperatorProvider operatorProvider, this.coercionService = coercionService; this.operatorProvider = operatorProvider; this.event = event; + this.listStart = false; operatorSymbolStack = new Stack<>(); operandStack = new Stack<>(); } @@ -54,6 +59,24 @@ public Object getResult() { return operandStack.peek(); } + private void validateSetMembers(Set setMembers) { + int numbers = 0; + int strings = 0; + int booleans = 0; + for (Object member: setMembers) { + if (member instanceof Number) { + numbers++; + } else if (member instanceof String) { + strings++; + } else if (member instanceof Boolean) { + booleans++; + } + } + if (numbers != setMembers.size() && strings != setMembers.size() && booleans != setMembers.size()) { + throw new RuntimeException("All set members should be of same type"); + } + } + @Override public void visitTerminal(TerminalNode node) { final int nodeType = node.getSymbol().getType(); @@ -62,12 +85,25 @@ public void visitTerminal(TerminalNode node) { } if (operatorProvider.containsOperator(nodeType) || nodeType == DataPrepperExpressionParser.LPAREN) { operatorSymbolStack.push(nodeType); + } else if (nodeType == DataPrepperExpressionParser.LBRACE) { + listStart = true; + setMembers = new HashSet<>(); + } else if (nodeType == DataPrepperExpressionParser.RBRACE) { + listStart = false; + validateSetMembers(setMembers); + operandStack.push(setMembers); } else if (nodeType == DataPrepperExpressionParser.RPAREN) { // pop LPAREN at operatorSymbolStack top operatorSymbolStack.pop(); } else { final Object arg = coercionService.coercePrimaryTerminalNode(node, event); - operandStack.push(arg); + if (listStart) { + if (!(arg instanceof Integer) || (((int)arg) != DataPrepperExpressionParser.COMMA && ((int)arg) != DataPrepperExpressionParser.SET_DELIMITER)) { + setMembers.add(arg); + } + } else { + operandStack.push(arg); + } } } diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java index a8fc7971f3..49ba051d92 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java @@ -216,6 +216,21 @@ private static Stream validExpressionArguments() { arguments("/sourceIp == null", event("{\"sourceIp\": [\"test\", \"test_two\"]}"), false), arguments("/sourceIp == null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), false), arguments("/sourceIp != null", event("{\"sourceIp\": {\"test\": \"test_two\"}}"), true), + arguments("/value in {200.222, 300.333, 400}", event("{\"value\": 400}"), true), + arguments("/value in {200.222, 300.333, 400}", event("{\"value\": 400.222}"), false), + arguments("/value not in {200.222, 300.333, 400}", event("{\"value\": 400}"), false), + arguments("/value not in {200.222, 300.333, 400}", event("{\"value\": 800.222}"), true), + arguments("/color in {\"blue\", \"red\", \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}"), true), + arguments("/color in {\"blue\", \"red\", \"yellow\", \"green\"}", event("{\"color\": \"gray\"}"), false), + arguments("/color not in {\"blue\", \"red\", \"yellow\", \"green\"}", event("{\"color\": \"gray\"}"), true), + arguments("/color not in {\"blue\", \"red\", \"yellow\", \"green\"}", event("{\"color\": \"blue\"}"), false), + arguments("/color in {\"blue\", \"\", \"red\", \"yellow\", \"green\"}", event("{\"color\": \"\"}"), true), + arguments("/status_code in {200 , 300}", event("{\"status_code\": 200}"), true), + arguments("/status_code in {2 , 3}", event("{\"status_code\": 2}"), true), + arguments("/status_code not in {200 , 300}", event("{\"status_code\": 400}"), true), + arguments("/status_code in {200 , 300}", event("{\"status_code\": 500}"), false), + arguments("/flag in {true , false}", event("{\"flag\": false}"), true), + arguments("/flag in {true , false}", event("{\"flag\": true}"), true), arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-0\"}"), true), arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-212\"}"), true), arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-abc\"}"), false), @@ -273,6 +288,18 @@ private static Stream invalidExpressionArguments() { arguments("contains(1234, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), arguments("contains(str, /strField)", event("{\"intField\":1234,\"strField\":\"string\"}")), arguments("contains(/strField, 1234)", event("{\"intField\":1234,\"strField\":\"string\"}")), + arguments("/color in {\"blue\", 222.0, \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/color in {\"blue, \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/color in {\"blue\", yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/color in {\", \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/color in { \", \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/color in {, \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/color in { , \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/color in {blue, \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/color in { blue, \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/color in {\"\",blue, \"yellow\", \"green\"}", event("{\"color\": \"yellow\"}")), + arguments("/value in {22a2.0, 100}", event("{\"value\": 100}")), + arguments("/value in {222, 10a0}", event("{\"value\": 100}")), arguments("getMetadata(10)", tagEvent), arguments("getMetadata("+ testMetadataKey+ ")", tagEvent), arguments("getMetadata(\""+ testMetadataKey+")", tagEvent), From 72e2db74b0b19d3582972ce03e26892d9eb5279b Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Thu, 15 Aug 2024 10:25:43 -0700 Subject: [PATCH 114/159] Translate proc optimizations (#4824) * dplive1.yaml Signed-off-by: Krishna Kondaka * Delete .github/workflows/static.yml Signed-off-by: Krishna Kondaka * Optimize translateSource in translate processor Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../plugins/processor/translate/TranslateProcessor.java | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java index d720208d86..d4c69bb331 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessor.java @@ -89,9 +89,8 @@ private List getSourceKeys(Object sourceObject){ } private void translateSource(Object sourceObject, Event recordEvent, TargetsParameterConfig targetConfig) { - Map recordObject = recordEvent.toMap(); List sourceKeysPaths = getSourceKeys(sourceObject); - if(Objects.isNull(recordObject) || sourceKeysPaths.isEmpty()){ + if(sourceKeysPaths.isEmpty()){ return; } @@ -107,14 +106,14 @@ private void translateSource(Object sourceObject, Event recordEvent, TargetsPara } String rootField = jsonExtractor.getRootField(commonPath); - if(!recordObject.containsKey(rootField)){ + if (!recordEvent.containsKey(rootField)) { return; } - + Map recordObject = recordEvent.toMap(); List targetObjects = jsonExtractor.getObjectFromPath(commonPath, recordObject); if(!targetObjects.isEmpty()) { targetObjects.forEach(targetObj -> performMappings(targetObj, sourceKeys, sourceObject, targetConfig)); - recordEvent.put(rootField, recordObject.get(rootField)); + recordEvent.put(rootField,recordObject.get(rootField)); } } From 79db359dac9010e77d3c7a7e02a64d6d7d01ceb3 Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 16 Aug 2024 07:52:46 -0500 Subject: [PATCH 115/159] Logging improvements when failing to parse JSON/XML/ION. Do not include the stack trace since it doesn't provide any value with these exceptions which are expected when the JSON is invalid. Log the input string rather than the Event which was not readable. (#4839) Signed-off-by: David Venable --- .../parse-json-processor/build.gradle | 1 + .../processor/parse/ion/ParseIonProcessor.java | 6 +++--- .../processor/parse/json/ParseJsonProcessor.java | 12 ++++++------ .../processor/parse/xml/ParseXmlProcessor.java | 6 +++--- 4 files changed, 13 insertions(+), 12 deletions(-) diff --git a/data-prepper-plugins/parse-json-processor/build.gradle b/data-prepper-plugins/parse-json-processor/build.gradle index 488dbf7d86..5125409731 100644 --- a/data-prepper-plugins/parse-json-processor/build.gradle +++ b/data-prepper-plugins/parse-json-processor/build.gradle @@ -16,6 +16,7 @@ dependencies { implementation libs.parquet.common testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-test-event') + testImplementation testLibs.slf4j.simple } test { diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java index 39c06165d9..9a58594da7 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java @@ -21,7 +21,7 @@ import java.util.HashMap; import java.util.Optional; -import static org.opensearch.dataprepper.logging.DataPrepperMarkers.EVENT; +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.SENSITIVE; @DataPrepperPlugin(name = "parse_ion", pluginType = Processor.class, pluginConfigurationType = ParseIonProcessorConfig.class) public class ParseIonProcessor extends AbstractParseProcessor { @@ -45,10 +45,10 @@ protected Optional> readValue(String message, Event cont // We need to do a two-step process here, read the value in, then convert away any Ion types like Timestamp return Optional.of(objectMapper.convertValue(objectMapper.readValue(message, new TypeReference<>() {}), new TypeReference<>() {})); } catch (JsonProcessingException e) { - LOG.error(EVENT, "An exception occurred due to invalid Ion while reading event [{}]", context, e); + LOG.error(SENSITIVE, "An exception occurred due to invalid Ion while parsing [{}] due to {}", message, e.getMessage()); return Optional.empty(); } catch (Exception e) { - LOG.error(EVENT, "An exception occurred while using the parse_ion processor on Event [{}]", context, e); + LOG.error(SENSITIVE, "An exception occurred while using the parse_ion processor while parsing [{}]", message, e); return Optional.empty(); } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java index c62d2af9c2..dd7b471b33 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java @@ -5,15 +5,15 @@ package org.opensearch.dataprepper.plugins.processor.parse.json; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.processor.Processor; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -21,7 +21,7 @@ import java.util.HashMap; import java.util.Optional; -import static org.opensearch.dataprepper.logging.DataPrepperMarkers.EVENT; +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.SENSITIVE; @DataPrepperPlugin(name = "parse_json", pluginType = Processor.class, pluginConfigurationType = ParseJsonProcessorConfig.class) public class ParseJsonProcessor extends AbstractParseProcessor { @@ -41,10 +41,10 @@ protected Optional> readValue(String message, Event cont try { return Optional.of(objectMapper.readValue(message, new TypeReference<>() {})); } catch (JsonProcessingException e) { - LOG.error(EVENT, "An exception occurred due to invalid JSON while reading event [{}]", context, e); + LOG.error(SENSITIVE, "An exception occurred due to invalid JSON while parsing [{}] due to {}", message, e.getMessage()); return Optional.empty(); } catch (Exception e) { - LOG.error(EVENT, "An exception occurred while using the parse_json processor on Event [{}]", context, e); + LOG.error(SENSITIVE, "An exception occurred while using the parse_json processor while parsing [{}]", message, e); return Optional.empty(); } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java index 840515afb2..debacad49c 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java @@ -16,7 +16,7 @@ import java.util.HashMap; import java.util.Optional; -import static org.opensearch.dataprepper.logging.DataPrepperMarkers.EVENT; +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.SENSITIVE; @DataPrepperPlugin(name = "parse_xml", pluginType =Processor.class, pluginConfigurationType =ParseXmlProcessorConfig.class) public class ParseXmlProcessor extends AbstractParseProcessor { @@ -36,10 +36,10 @@ protected Optional> readValue(final String message, fina try { return Optional.of(xmlMapper.readValue(message, new TypeReference<>() {})); } catch (JsonProcessingException e) { - LOG.error(EVENT, "An exception occurred due to invalid XML while reading event [{}]", context, e); + LOG.error(SENSITIVE, "An exception occurred due to invalid XML while parsing [{}] due to {}", message, e.getMessage()); return Optional.empty(); } catch (Exception e) { - LOG.error(EVENT, "An exception occurred while using the parse_xml processor on Event [{}]", context, e); + LOG.error(SENSITIVE, "An exception occurred while using the parse_xml processor while parsing [{}]", message, e); return Optional.empty(); } } From 1ac6df2e07e369f0414b93ed2e151ce92f8d514b Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Fri, 16 Aug 2024 16:37:44 -0500 Subject: [PATCH 116/159] Add startsWith expression function (#4840) Signed-off-by: Taylor Gray --- .../StartsWithExpressionFunction.java | 53 +++++++++ .../ContainsExpressionFunctionTest.java | 3 +- ...ericExpressionEvaluator_ConditionalIT.java | 7 +- .../StartsWithExpressionFunctionTest.java | 111 ++++++++++++++++++ 4 files changed, 170 insertions(+), 4 deletions(-) create mode 100644 data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/StartsWithExpressionFunction.java create mode 100644 data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/StartsWithExpressionFunctionTest.java diff --git a/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/StartsWithExpressionFunction.java b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/StartsWithExpressionFunction.java new file mode 100644 index 0000000000..c8e4b77187 --- /dev/null +++ b/data-prepper-expression/src/main/java/org/opensearch/dataprepper/expression/StartsWithExpressionFunction.java @@ -0,0 +1,53 @@ +package org.opensearch.dataprepper.expression; + +import org.opensearch.dataprepper.model.event.Event; + +import javax.inject.Named; +import java.util.List; +import java.util.function.Function; + +@Named +public class StartsWithExpressionFunction implements ExpressionFunction { + private static final int NUMBER_OF_ARGS = 2; + + static final String STARTS_WITH_FUNCTION_NAME = "startsWith"; + @Override + public String getFunctionName() { + return STARTS_WITH_FUNCTION_NAME; + } + + @Override + public Object evaluate( + final List args, + final Event event, + final Function convertLiteralType) { + + if (args.size() != NUMBER_OF_ARGS) { + throw new RuntimeException("startsWith() takes exactly two arguments"); + } + + String[] strArgs = new String[NUMBER_OF_ARGS]; + for (int i = 0; i < NUMBER_OF_ARGS; i++) { + Object arg = args.get(i); + if (!(arg instanceof String)) { + throw new RuntimeException(String.format("startsWith() takes only string type arguments. \"%s\" is not of type string", arg)); + } + String stringOrKey = (String) arg; + if (stringOrKey.charAt(0) == '"') { + strArgs[i] = stringOrKey.substring(1, stringOrKey.length()-1); + } else if (stringOrKey.charAt(0) == '/') { + Object obj = event.get(stringOrKey, Object.class); + if (obj == null) { + return false; + } + if (!(obj instanceof String)) { + throw new RuntimeException(String.format("startsWith() only operates on string types. The value at \"%s\" is \"%s\" which is not a string type.", stringOrKey, obj)); + } + strArgs[i] = (String)obj; + } else { + throw new RuntimeException(String.format("Arguments to startsWith() must be a literal string or a Json Pointer. \"%s\" is not string literal or json pointer", stringOrKey)); + } + } + return strArgs[0].startsWith(strArgs[1]); + } +} diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/ContainsExpressionFunctionTest.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/ContainsExpressionFunctionTest.java index b24086b231..3497caa10d 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/ContainsExpressionFunctionTest.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/ContainsExpressionFunctionTest.java @@ -56,8 +56,7 @@ public ContainsExpressionFunction createObjectUnderTest() { } @Test - void testContainsBasic() { - containsExpressionFunction = createObjectUnderTest(); + void testContainsBasic() {containsExpressionFunction = createObjectUnderTest(); assertThat(containsExpressionFunction.evaluate(List.of("\"abcde\"", "\"abcd\""), testEvent, testFunction), equalTo(true)); assertThat(containsExpressionFunction.evaluate(List.of("/"+testKey, "/"+testKey2), testEvent, testFunction), equalTo(true)); assertThat(containsExpressionFunction.evaluate(List.of("\""+testValue+"\"", "/"+testKey2), testEvent, testFunction), equalTo(true)); diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java index 49ba051d92..d4741c539a 100644 --- a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/GenericExpressionEvaluator_ConditionalIT.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; import java.util.Random; +import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -234,8 +235,10 @@ private static Stream validExpressionArguments() { arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-0\"}"), true), arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-212\"}"), true), arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"name\": \"dataprepper-abc\"}"), false), - arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"other\": \"dataprepper-abc\"}"), false) - ); + arguments("/name =~ \".*dataprepper-[0-9]+\"", event("{\"other\": \"dataprepper-abc\"}"), false), + arguments("startsWith(\""+strValue+ UUID.randomUUID() + "\",/status)", event("{\"status\":\""+strValue+"\"}"), true), + arguments("startsWith(\""+ UUID.randomUUID() +strValue+ "\",/status)", event("{\"status\":\""+strValue+"\"}"), false) + ); } private static Stream invalidExpressionArguments() { diff --git a/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/StartsWithExpressionFunctionTest.java b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/StartsWithExpressionFunctionTest.java new file mode 100644 index 0000000000..52e03da193 --- /dev/null +++ b/data-prepper-expression/src/test/java/org/opensearch/dataprepper/expression/StartsWithExpressionFunctionTest.java @@ -0,0 +1,111 @@ +package org.opensearch.dataprepper.expression; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Stream; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; +import static org.opensearch.dataprepper.expression.StartsWithExpressionFunction.STARTS_WITH_FUNCTION_NAME; + +public class StartsWithExpressionFunctionTest { + + private Event testEvent; + + private Event createTestEvent(final Object data) { + return JacksonEvent.builder().withEventType("event").withData(data).build(); + } + + private ExpressionFunction createObjectUnderTest() { + return new StartsWithExpressionFunction(); + } + + @ParameterizedTest + @MethodSource("validStartsWithProvider") + void startsWith_returns_expected_result_when_evaluated( + final String value, final String prefix, final boolean expectedResult) { + final String key = "test_key"; + testEvent = createTestEvent(Map.of(key, value)); + + final ExpressionFunction objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.getFunctionName(), equalTo(STARTS_WITH_FUNCTION_NAME)); + + final Object result = objectUnderTest.evaluate(List.of("/" + key, "\"" + prefix + "\""), testEvent, mock(Function.class)); + + assertThat(result, equalTo(expectedResult)); + } + + @Test + void startsWith_with_a_key_as_the_prefix_returns_expected_result() { + + final String prefixKey = "prefix"; + final String prefixValue = "te"; + + final String key = "test_key"; + final String value = "test"; + testEvent = createTestEvent(Map.of(key, value, prefixKey, prefixValue)); + + final ExpressionFunction objectUnderTest = createObjectUnderTest(); + assertThat(objectUnderTest.getFunctionName(), equalTo(STARTS_WITH_FUNCTION_NAME)); + + final Object result = objectUnderTest.evaluate(List.of("/" + key, "/" + prefixKey), testEvent, mock(Function.class)); + + assertThat(result, equalTo(true)); + } + + @Test + void startsWith_returns_false_when_key_does_not_exist_in_Event() { + final String key = "test_key"; + testEvent = createTestEvent(Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString())); + + final ExpressionFunction startsWithExpressionFunction = createObjectUnderTest(); + final Object result = startsWithExpressionFunction.evaluate(List.of("/" + key, "\"abcd\""), testEvent, mock(Function.class)); + + assertThat(result, equalTo(false)); + } + + @Test + void startsWith_without_2_arguments_throws_RuntimeException() { + final ExpressionFunction startsWithExpressionFunction = createObjectUnderTest(); + assertThrows(RuntimeException.class, () -> startsWithExpressionFunction.evaluate(List.of("abcd"), testEvent, mock(Function.class))); + } + + @ParameterizedTest + @MethodSource("invalidStartsWithProvider") + void invalid_startsWith_arguments_throws_RuntimeException(final String firstArg, final Object secondArg, final Object value) { + final ExpressionFunction startsWithExpressionFunction = createObjectUnderTest(); + final String testKey = "test_key"; + + assertThrows(RuntimeException.class, () -> startsWithExpressionFunction.evaluate(List.of(firstArg, secondArg), createTestEvent(Map.of(testKey, value)), mock(Function.class))); + } + + private static Stream validStartsWithProvider() { + return Stream.of( + Arguments.of("{test", "{te", true), + Arguments.of("{test", "{", true), + Arguments.of("test", "{", false), + Arguments.of("MyPrefix", "My", true), + Arguments.of("MyPrefix", "Prefix", false) + ); + } + + private static Stream invalidStartsWithProvider() { + return Stream.of( + Arguments.of("\"abc\"", "/test_key", 1234), + Arguments.of("abcd", "/test_key", "value"), + Arguments.of("\"abcd\"", "/test_key", 1234), + Arguments.of("\"/test_key\"", 1234, "value") + ); + } +} From 38f807924d508c7b3a6bad681e27007bdb3774ee Mon Sep 17 00:00:00 2001 From: Srikanth Govindarajan Date: Fri, 16 Aug 2024 18:10:52 -0700 Subject: [PATCH 117/159] Add support for dynamic rule detection for pipeline config transformation (#4601) * Add support for dynamic rule detection for pipeline config transformation Signed-off-by: Srikanth Govindarajan * Address comments Signed-off-by: Srikanth Govindarajan * Move rules and templates to plugin level Signed-off-by: Srikanth Govindarajan * Add dummy plugin for testing dynamic rule detection Signed-off-by: Srikanth Govindarajan * Address comments Signed-off-by: Srikanth Govindarajan --------- Signed-off-by: Srikanth Govindarajan Signed-off-by: Srikanth Govindarajan --- .../PipelineTransformationConfiguration.java | 31 +--- .../pipeline/parser/rule/RuleEvaluator.java | 124 +++++++------- .../parser/rule/RuleFileEvaluation.java | 18 +++ .../pipeline/parser/rule/RuleStream.java | 26 +++ .../parser/rule/RuleTransformerModel.java | 21 +-- .../pipeline/parser/rule/TemplateStream.java | 20 +++ .../transformer/TransformersFactory.java | 148 ++++++++++------- .../templates/documentdb-template.yaml | 81 ---------- .../parser/rule/RuleEvaluatorTest.java | 69 ++++---- .../parser/rule/RuleTransformerModelTest.java | 6 +- .../DynamicConfigTransformerTest.java | 152 ++++++++++-------- .../transformer/TransformersFactoryTest.java | 96 ++++++----- .../transforms/rules/test-plugin-rule.yaml | 3 + .../templates/test-plugin-template.yaml | 4 + .../transformation/rules/documentdb-rule.yaml | 3 +- .../rules/documentdb1-rule.yaml | 1 + data-prepper-plugins/dummy-plugin/README.md | 63 ++++++++ .../dummy-plugin/build.gradle | 0 .../transforms/rules/dummy-plugin-rule.yaml | 3 + .../templates/dummy-plugin-template.yaml | 9 ++ .../transforms}/rules/documentdb-rule.yaml | 1 + .../templates/documentdb-template.yaml | 81 ++++++++++ settings.gradle | 3 +- 23 files changed, 574 insertions(+), 389 deletions(-) create mode 100644 data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleFileEvaluation.java create mode 100644 data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleStream.java create mode 100644 data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/TemplateStream.java delete mode 100644 data-prepper-pipeline-parser/src/main/resources/templates/documentdb-template.yaml create mode 100644 data-prepper-pipeline-parser/src/test/resources/org/opensearch/dataprepper/transforms/rules/test-plugin-rule.yaml create mode 100644 data-prepper-pipeline-parser/src/test/resources/org/opensearch/dataprepper/transforms/templates/test-plugin-template.yaml create mode 100644 data-prepper-plugins/dummy-plugin/README.md create mode 100644 data-prepper-plugins/dummy-plugin/build.gradle create mode 100644 data-prepper-plugins/dummy-plugin/src/main/resources/org/opensearch/dataprepper/transforms/rules/dummy-plugin-rule.yaml create mode 100644 data-prepper-plugins/dummy-plugin/src/main/resources/org/opensearch/dataprepper/transforms/templates/dummy-plugin-template.yaml rename {data-prepper-pipeline-parser/src/main/resources => data-prepper-plugins/mongodb/src/main/resources/org/opensearch/dataprepper/transforms}/rules/documentdb-rule.yaml (74%) create mode 100644 data-prepper-plugins/mongodb/src/main/resources/org/opensearch/dataprepper/transforms/templates/documentdb-template.yaml diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/PipelineTransformationConfiguration.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/PipelineTransformationConfiguration.java index 9cf83b11d7..aa7d593bfb 100644 --- a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/PipelineTransformationConfiguration.java +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/PipelineTransformationConfiguration.java @@ -9,39 +9,12 @@ import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.Configuration; -import javax.inject.Named; -import java.nio.file.Path; -import java.nio.file.Paths; - @Configuration public class PipelineTransformationConfiguration { - public static final String TEMPLATES_DIRECTORY_PATH = "TEMPLATES_DIRECTORY_PATH"; - public static final String RULES_DIRECTORY_PATH = "RULES_DIRECTORY_PATH"; - private static final Path currentDir = Paths.get(System.getProperty("user.dir")); -// private static final String parserRelativePath = "/data-prepper-pipeline-parser/src"; - - @Bean - @Named(RULES_DIRECTORY_PATH) - static String provideRulesDirectoryPath() { - ClassLoader classLoader = PipelineTransformationConfiguration.class.getClassLoader(); - String filePath = classLoader.getResource("rules").getFile(); - return filePath; - } - - @Bean - @Named(TEMPLATES_DIRECTORY_PATH) - static String provideTemplateDirectoryPath() { - ClassLoader classLoader = PipelineTransformationConfiguration.class.getClassLoader(); - String filePath = classLoader.getResource("templates").getFile(); - return filePath; - } @Bean - TransformersFactory transformersFactory( - @Named(RULES_DIRECTORY_PATH) String rulesDirectoryPath, - @Named(TEMPLATES_DIRECTORY_PATH) String templatesDirectoryPath - ) { - return new TransformersFactory(rulesDirectoryPath, templatesDirectoryPath); + TransformersFactory transformersFactory() { + return new TransformersFactory(); } @Bean diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleEvaluator.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleEvaluator.java index e546d5fd10..5e2c50c4ff 100644 --- a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleEvaluator.java +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleEvaluator.java @@ -5,15 +5,14 @@ package org.opensearch.dataprepper.pipeline.parser.rule; import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.jayway.jsonpath.Configuration; import com.jayway.jsonpath.JsonPath; import com.jayway.jsonpath.Option; -import com.jayway.jsonpath.ParseContext; import com.jayway.jsonpath.PathNotFoundException; -import com.jayway.jsonpath.ReadContext; -import com.jayway.jsonpath.spi.json.JacksonJsonProvider; +import com.jayway.jsonpath.spi.json.JacksonJsonNodeJsonProvider; import com.jayway.jsonpath.spi.mapper.JacksonMappingProvider; import org.opensearch.dataprepper.model.configuration.PipelineModel; import org.opensearch.dataprepper.model.configuration.PipelinesDataFlowModel; @@ -25,6 +24,7 @@ import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; +import java.util.Collection; import java.util.List; import java.util.Map; @@ -34,40 +34,29 @@ public class RuleEvaluator { private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final ObjectMapper yamlMapper = new ObjectMapper(new YAMLFactory()); private final TransformersFactory transformersFactory; - private String PLUGIN_NAME = null; + public RuleEvaluator(TransformersFactory transformersFactory) { this.transformersFactory = transformersFactory; } public RuleEvaluatorResult isTransformationNeeded(PipelinesDataFlowModel pipelineModel) { - //TODO - Dynamically scan the rules folder and get the corresponding template. - return isDocDBSource(pipelineModel); - } - /** - * Evaluates model based on pre defined rules and - * result contains the name of the pipeline that will need transformation, - * evaluated boolean result and the corresponding template model - * Assumption: only one pipeline can have transformation. - * - * @param pipelinesModel - * @return RuleEvaluatorResult - */ - private RuleEvaluatorResult isDocDBSource(PipelinesDataFlowModel pipelinesModel) { - PLUGIN_NAME = "documentdb"; - - Map pipelines = pipelinesModel.getPipelines(); + Map pipelines = pipelineModel.getPipelines(); for (Map.Entry entry : pipelines.entrySet()) { try { String pipelineJson = OBJECT_MAPPER.writeValueAsString(entry); - if (evaluate(pipelineJson, PLUGIN_NAME)) { - LOG.info("Rule for {} is evaluated true for pipelineJson {}", PLUGIN_NAME, pipelineJson); + RuleFileEvaluation ruleFileEvaluation = evaluate(pipelineJson); + + if (ruleFileEvaluation.getResult()) { + String pluginName = ruleFileEvaluation.getPluginName(); + LOG.info("Applying rule {}",ruleFileEvaluation.getRuleFileName().toString()); + LOG.info("Rule for {} is evaluated true for pipelineJson {}", pluginName, pipelineJson); - InputStream templateStream = transformersFactory.getPluginTemplateFileStream(PLUGIN_NAME); + InputStream templateStream = transformersFactory.getPluginTemplateFileStream(pluginName); PipelineTemplateModel templateModel = yamlMapper.readValue(templateStream, PipelineTemplateModel.class); - LOG.info("Template is chosen for {}", PLUGIN_NAME); + LOG.info("Template is chosen for {}", pluginName); return RuleEvaluatorResult.builder() .withEvaluatedResult(true) @@ -75,11 +64,7 @@ private RuleEvaluatorResult isDocDBSource(PipelinesDataFlowModel pipelinesModel) .withPipelineName(entry.getKey()) .build(); } - } catch (FileNotFoundException e){ - LOG.error("Template File Not Found for {}", PLUGIN_NAME); - throw new RuntimeException(e); - } - catch (JsonProcessingException e) { + } catch (JsonProcessingException e) { LOG.error("Error processing json"); throw new RuntimeException(e); } catch (IOException e) { @@ -94,47 +79,68 @@ private RuleEvaluatorResult isDocDBSource(PipelinesDataFlowModel pipelinesModel) .build(); } - private Boolean evaluate(String pipelinesJson, - String pluginName) { - + private RuleFileEvaluation evaluate(String pipelinesJson) { Configuration parseConfig = Configuration.builder() - .jsonProvider(new JacksonJsonProvider()) + .jsonProvider(new JacksonJsonNodeJsonProvider()) .mappingProvider(new JacksonMappingProvider()) - .options(Option.AS_PATH_LIST) + .options(Option.SUPPRESS_EXCEPTIONS) .build(); - ParseContext parseContext = JsonPath.using(parseConfig); - ReadContext readPathContext = parseContext.parse(pipelinesJson); RuleTransformerModel rulesModel = null; - InputStream ruleStream = null; + try { - ruleStream = transformersFactory.getPluginRuleFileStream(pluginName); + Collection ruleStreams = transformersFactory.loadRules(); - rulesModel = yamlMapper.readValue(ruleStream, RuleTransformerModel.class); - List rules = rulesModel.getApplyWhen(); - for (String rule : rules) { - try { - Object result = readPathContext.read(rule); - } catch (PathNotFoundException e) { - LOG.warn("Json Path not found for {}", pluginName); - return false; - } - } - } catch (FileNotFoundException e){ - LOG.warn("Rule File Not Found for {}", pluginName); - return false; - } catch(IOException e){ - throw new RuntimeException(e); - }finally { - if (ruleStream != null) { + //walk through all rules and return first valid + for (RuleStream ruleStream : ruleStreams) { try { + rulesModel = yamlMapper.readValue(ruleStream.getRuleStream(), RuleTransformerModel.class); + List rules = rulesModel.getApplyWhen(); + String pluginName = rulesModel.getPluginName(); + boolean allRulesValid = true; + + for (String rule : rules) { + try { + JsonNode result = JsonPath.using(parseConfig).parse(pipelinesJson).read(rule); + if (result == null || result.size() == 0) { + allRulesValid = false; + break; + } + } catch (PathNotFoundException e) { + LOG.debug("Json Path not found for {}", ruleStream.getName()); + allRulesValid = false; + break; + } + } + + if (allRulesValid) { + return RuleFileEvaluation.builder() + .withPluginName(pluginName) + .withRuleFileName(ruleStream.getName()) + .withResult(true) + .build(); + } + } finally { ruleStream.close(); - } catch (IOException e) { - throw new RuntimeException(e); } } + + } catch (FileNotFoundException e) { + LOG.debug("Rule File Not Found", e); + return RuleFileEvaluation.builder() + .withPluginName(null) + .withRuleFileName(null) + .withResult(false) + .build(); + } catch (IOException e) { + throw new RuntimeException(e); } - return true; + + return RuleFileEvaluation.builder() + .withPluginName(null) + .withRuleFileName(null) + .withResult(false) + .build(); } -} +} \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleFileEvaluation.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleFileEvaluation.java new file mode 100644 index 0000000000..88ee0f8d6c --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleFileEvaluation.java @@ -0,0 +1,18 @@ +package org.opensearch.dataprepper.pipeline.parser.rule; + +import lombok.AllArgsConstructor; +import lombok.Builder; +import lombok.Data; + +@Builder(setterPrefix = "with") +@AllArgsConstructor +@Data +public class RuleFileEvaluation { + private Boolean result; + private String ruleFileName; + private String pluginName; + + public RuleFileEvaluation() { + + } +} diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleStream.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleStream.java new file mode 100644 index 0000000000..6d467787fa --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleStream.java @@ -0,0 +1,26 @@ +package org.opensearch.dataprepper.pipeline.parser.rule; + +import lombok.AllArgsConstructor; +import lombok.Data; + +import java.io.IOException; +import java.io.InputStream; + +@Data +@AllArgsConstructor +public class RuleStream { + private String name; + private InputStream ruleStream; + + + public void close() { + if (ruleStream != null) { + try { + ruleStream.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } +} + diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleTransformerModel.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleTransformerModel.java index 0ad9e45b72..aaf757cedb 100644 --- a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleTransformerModel.java +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleTransformerModel.java @@ -4,36 +4,31 @@ */ package org.opensearch.dataprepper.pipeline.parser.rule; +import com.fasterxml.jackson.annotation.JsonInclude; import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.AllArgsConstructor; import lombok.Data; import java.util.List; @Data +@AllArgsConstructor public class RuleTransformerModel { + @JsonInclude(JsonInclude.Include.NON_EMPTY) @JsonProperty("apply_when") private List applyWhen; - public RuleTransformerModel() { - } - - public RuleTransformerModel(List applyWhen) { - this.applyWhen = applyWhen; - } + @JsonProperty("plugin_name") + private String pluginName; - public List getApplyWhen() { - return applyWhen; - } - - public void setApplyWhen(List applyWhen) { - this.applyWhen = applyWhen; + public RuleTransformerModel() { } @Override public String toString() { return "RuleConfiguration{" + "applyWhen=" + applyWhen + - '}'; + "\npluginName="+ pluginName +'}'; } } \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/TemplateStream.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/TemplateStream.java new file mode 100644 index 0000000000..2fc238d796 --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/rule/TemplateStream.java @@ -0,0 +1,20 @@ +package org.opensearch.dataprepper.pipeline.parser.rule; + +import java.io.IOException; +import java.io.InputStream; + +public class TemplateStream { + private String name; + private InputStream templateStream; + + + public void close() { + if (templateStream != null) { + try { + templateStream.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } +} diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/transformer/TransformersFactory.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/transformer/TransformersFactory.java index 95741e9cf6..e903888a49 100644 --- a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/transformer/TransformersFactory.java +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/transformer/TransformersFactory.java @@ -4,84 +4,116 @@ */ package org.opensearch.dataprepper.pipeline.parser.transformer; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; -import static org.opensearch.dataprepper.pipeline.parser.PipelineTransformationConfiguration.RULES_DIRECTORY_PATH; -import static org.opensearch.dataprepper.pipeline.parser.PipelineTransformationConfiguration.TEMPLATES_DIRECTORY_PATH; - -import javax.inject.Named; -import java.io.File; -import java.io.FileNotFoundException; +import org.opensearch.dataprepper.pipeline.parser.rule.RuleStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.io.IOException; import java.io.InputStream; +import java.net.URISyntaxException; +import java.net.URL; +import java.nio.file.FileSystem; +import java.nio.file.FileSystemNotFoundException; +import java.nio.file.FileSystems; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Enumeration; +import java.util.List; +import java.util.stream.Stream; -public class TransformersFactory implements PipelineTransformationPathProvider { - - private static final ObjectMapper YAML_MAPPER = new ObjectMapper(new YAMLFactory()); +public class TransformersFactory { + private static final Logger LOG = LoggerFactory.getLogger(TransformersFactory.class); + private static final String TEMPLATES_PATH = "org/opensearch/dataprepper/transforms/templates/"; + private static final String RULES_PATH = "org/opensearch/dataprepper/transforms/rules/"; private final String TEMPLATE_FILE_NAME_PATTERN = "-template.yaml"; private final String RULE_FILE_NAME_PATTERN = "-rule.yaml"; - private final String templatesDirectoryPath; - private final String rulesDirectoryPath; - - public TransformersFactory(@Named(RULES_DIRECTORY_PATH) final String rulesDirectoryPath, - @Named(TEMPLATES_DIRECTORY_PATH) final String templatesDirectoryPath) { - this.templatesDirectoryPath = templatesDirectoryPath; - this.rulesDirectoryPath = rulesDirectoryPath; - } - @Override - public String getTransformationTemplateDirectoryLocation() { - return templatesDirectoryPath; + public TransformersFactory(){ } - @Override - public String getTransformationRulesDirectoryLocation() { - return rulesDirectoryPath; - } - public String getPluginTemplateFileLocation(String pluginName) { - if(pluginName == null || pluginName.isEmpty()){ - throw new RuntimeException("Transformation plugin not found"); + public InputStream getPluginTemplateFileStream(String pluginName) { + if (pluginName == null || pluginName.isEmpty()) { + throw new RuntimeException("Transformation plugin not found"); } - return templatesDirectoryPath + "/" + pluginName + TEMPLATE_FILE_NAME_PATTERN; - } - public String getPluginRuleFileLocation(String pluginName) { - if(pluginName == null || pluginName.isEmpty()){ - throw new RuntimeException("Transformation plugin not found"); - } - return rulesDirectoryPath + "/" + pluginName + RULE_FILE_NAME_PATTERN; - } + // Construct the expected file name + String templateFileName = pluginName + TEMPLATE_FILE_NAME_PATTERN; - public InputStream getPluginRuleFileStream(String pluginName) { - if(pluginName == null || pluginName.isEmpty()){ - throw new RuntimeException("Transformation plugin not found"); + // Use the ClassLoader to find the template file on the classpath + ClassLoader classLoader = getClass().getClassLoader(); + URL templateURL = classLoader.getResource(TEMPLATES_PATH + templateFileName); + + if (templateURL == null) { + throw new RuntimeException("Template file not found for plugin: " + pluginName); } - ClassLoader classLoader = TransformersFactory.class.getClassLoader(); - InputStream filestream = classLoader.getResourceAsStream("rules" + "/" + pluginName + RULE_FILE_NAME_PATTERN); - return filestream; - } - public InputStream getPluginTemplateFileStream(String pluginName) { - if(pluginName == null || pluginName.isEmpty()){ - throw new RuntimeException("Transformation plugin not found"); + try { + // Convert the URL to a URI, then to a Path to read the file + Path templatePath; + try { + templatePath = Paths.get(templateURL.toURI()); + } catch (FileSystemNotFoundException e) { + // Handle the case where the file system is not accessible (e.g., in a JAR) + FileSystem fileSystem = FileSystems.newFileSystem(templateURL.toURI(), Collections.emptyMap()); + templatePath = fileSystem.getPath(TEMPLATES_PATH + templateFileName); + } + + // Return an InputStream for the found file + return Files.newInputStream(templatePath); + + } catch (IOException | URISyntaxException e) { + throw new RuntimeException("Failed to load template file for plugin: " + pluginName, e); } - ClassLoader classLoader = TransformersFactory.class.getClassLoader(); - InputStream filestream = classLoader.getResourceAsStream("templates" + "/" + pluginName + TEMPLATE_FILE_NAME_PATTERN); - return filestream; } - - public PipelineTemplateModel getTemplateModel(String pluginName) { - String templatePath = getPluginTemplateFileLocation(pluginName); + public Collection loadRules() { + List ruleStreams = new ArrayList<>(); + ClassLoader classLoader = getClass().getClassLoader(); try { - PipelineTemplateModel pipelineTemplateModel = YAML_MAPPER.readValue(new File(templatePath), PipelineTemplateModel.class); - return pipelineTemplateModel; - } catch (FileNotFoundException e) { - throw new RuntimeException(e); + // Use ClassLoader to find all resources that match the RULES_PATH pattern + Enumeration rulesURLs = classLoader.getResources(RULES_PATH); + + while (rulesURLs.hasMoreElements()) { + URL rulesURL = rulesURLs.nextElement(); + + try { + // Convert the URL to a URI, then to a Path to read the directory contents + Path rulesPath; + try { + rulesPath = Paths.get(rulesURL.toURI()); + } catch (FileSystemNotFoundException e) { + // Handle the case where the file system is not accessible (e.g., in a JAR) + FileSystem fileSystem = FileSystems.newFileSystem(rulesURL.toURI(), Collections.emptyMap()); + rulesPath = fileSystem.getPath(RULES_PATH); + } + + // Scan the directory for rule files + try (Stream paths = Files.walk(rulesPath)) { + paths.filter(Files::isRegularFile) + .forEach(rulePath -> { + try { + InputStream ruleInputStream = Files.newInputStream(rulePath); + ruleStreams.add(new RuleStream(rulePath.getFileName().toString(), ruleInputStream)); + } catch (IOException e) { + throw new RuntimeException("Failed to load rule: " + rulePath, e); + } + }); + } + } catch (IOException | URISyntaxException e) { + throw new RuntimeException("Failed to scan rules directory on classpath: " + rulesURL, e); + } + } } catch (IOException e) { - throw new RuntimeException(e); + throw new RuntimeException("Failed to load rules from classpath.", e); } + + return ruleStreams; } + } diff --git a/data-prepper-pipeline-parser/src/main/resources/templates/documentdb-template.yaml b/data-prepper-pipeline-parser/src/main/resources/templates/documentdb-template.yaml deleted file mode 100644 index 0e0e6d5325..0000000000 --- a/data-prepper-pipeline-parser/src/main/resources/templates/documentdb-template.yaml +++ /dev/null @@ -1,81 +0,0 @@ -"<>": - workers: "<<$.<>.workers>>" - delay: "<<$.<>.delay>>" - buffer: "<<$.<>.buffer>>" - source: - documentdb: "<<$.<>.source.documentdb>>" - routes: - - initial_load: 'getMetadata("ingestion_type") == "EXPORT"' - - stream_load: 'getMetadata("ingestion_type") == "STREAM"' - sink: - - s3: - routes: - - initial_load - aws: - region: "<<$.<>.source.documentdb.s3_region>>" - sts_role_arn: "<<$.<>.source.documentdb.aws.sts_role_arn>>" - sts_external_id: "<<$.<>.source.documentdb.aws.sts_external_id>>" - sts_header_overrides: "<<$.<>.source.documentdb.aws.sts_header_overrides>>" - bucket: "<<$.<>.source.documentdb.s3_bucket>>" - threshold: - event_collect_timeout: "120s" - maximum_size: "2mb" - aggregate_threshold: - maximum_size: "128mb" - flush_capacity_ratio: 0 - object_key: - path_prefix: "${getMetadata(\"s3_partition_key\")}" - codec: - event_json: - default_bucket_owner: "<>.source.documentdb.aws.sts_role_arn>>" - - s3: - routes: - - stream_load - aws: - region: "<<$.<>.source.documentdb.s3_region>>" - sts_role_arn: "<<$.<>.source.documentdb.aws.sts_role_arn>>" - sts_external_id: "<<$.<>.source.documentdb.aws.sts_external_id>>" - sts_header_overrides: "<<$.<>.source.documentdb.aws.sts_header_overrides>>" - bucket: "<<$.<>.source.documentdb.s3_bucket>>" - threshold: - event_collect_timeout: "15s" - maximum_size: "1mb" - aggregate_threshold: - maximum_size: "128mb" - flush_capacity_ratio: 0 - object_key: - path_prefix: "${getMetadata(\"s3_partition_key\")}" - codec: - event_json: - default_bucket_owner: "<>.source.documentdb.aws.sts_role_arn>>" -"<>-s3": - workers: "<<$.<>.workers>>" - delay: "<<$.<>.delay>>" - buffer: "<<$.<>.buffer>>" - source: - s3: - codec: - event_json: - compression: "none" - aws: - region: "<<$.<>.source.documentdb.s3_region>>" - sts_role_arn: "<<$.<>.source.documentdb.aws.sts_role_arn>>" - sts_external_id: "<<$.<>.source.documentdb.aws.sts_external_id>>" - sts_header_overrides: "<<$.<>.source.documentdb.aws.sts_header_overrides>>" - acknowledgments: true - delete_s3_objects_on_read: true - disable_s3_metadata_in_event: true - scan: - folder_partitions: - depth: "<>.source.documentdb.s3_prefix>>" - max_objects_per_ownership: 50 - buckets: - - bucket: - name: "<<$.<>.source.documentdb.s3_bucket>>" - filter: - include_prefix: ["<>.source.documentdb.s3_prefix>>"] - scheduling: - interval: "60s" - processor: "<<$.<>.processor>>" - sink: "<<$.<>.sink>>" - routes: "<<$.<>.routes>>" # In placeholder, routes or route (defined as alias) will be transformed to route in json as route will be primarily picked in pipelineModel. \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleEvaluatorTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleEvaluatorTest.java index 361feb3558..a7bf0f5e6c 100644 --- a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleEvaluatorTest.java +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleEvaluatorTest.java @@ -5,10 +5,10 @@ package org.opensearch.dataprepper.pipeline.parser.rule; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import org.junit.jupiter.api.Test; -import org.mockito.Mockito; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import org.opensearch.dataprepper.model.configuration.PipelineExtensions; @@ -20,9 +20,9 @@ import org.opensearch.dataprepper.pipeline.parser.transformer.TransformersFactory; import java.io.FileInputStream; -import java.io.FileNotFoundException; +import java.io.IOException; import java.io.InputStream; -import java.util.ArrayList; +import java.nio.file.Paths; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -31,18 +31,16 @@ class RuleEvaluatorTest { @Test - void test_isTransformationNeeded_ForDocDBSource_ShouldReturn_True() throws FileNotFoundException { + void test_isTransformationNeeded_ForDocDBSource_ShouldReturn_True() throws IOException { - // Set up String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCUMENTDB_CONFIG_FILE; + String ruleDocDBTemplatePath = TestConfigurationProvider.TEMPLATE_TRANSFORMATION_DOCDB1_CONFIG_FILE; String pluginName = "documentdb"; String pipelineName = "test-pipeline"; - Map sourceOptions = new HashMap(); - Map s3_bucket = new HashMap<>(); - s3_bucket.put("s3_bucket", "bucket-name"); - List collections = new ArrayList(); - collections.add(s3_bucket); - sourceOptions.put("collections", collections); + Map sourceOptions = new HashMap<>(); + Map s3Bucket = new HashMap<>(); + s3Bucket.put("s3_bucket", "bucket-name"); + sourceOptions.put("s3_bucket", s3Bucket); final PluginModel source = new PluginModel(pluginName, sourceOptions); final List processors = Collections.singletonList(new PluginModel("testProcessor", null)); final List sinks = Collections.singletonList(new SinkModel("testSink", Collections.emptyList(), null, Collections.emptyList(), Collections.emptyList(), null)); @@ -51,54 +49,55 @@ void test_isTransformationNeeded_ForDocDBSource_ShouldReturn_True() throws FileN final PipelinesDataFlowModel pipelinesDataFlowModel = new PipelinesDataFlowModel( (PipelineExtensions) null, Collections.singletonMap(pipelineName, pipelineModel)); - TransformersFactory transformersFactory = Mockito.spy(new TransformersFactory( - TestConfigurationProvider.RULES_TRANSFORMATION_DIRECTORY, - TestConfigurationProvider.TEMPLATES_SOURCE_TRANSFORMATION_DIRECTORY - )); - RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); +// TransformersFactory transformersFactory = spy(new TransformersFactory("", "")); InputStream ruleStream = new FileInputStream(ruleDocDBFilePath); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream); + InputStream templateStream = new FileInputStream(ruleDocDBTemplatePath); + RuleStream ruleInputStream = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream); + + List ruleStreams = Collections.singletonList(ruleInputStream); + when(transformersFactory.loadRules()).thenReturn(ruleStreams); + when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); + + RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); RuleEvaluatorResult result = ruleEvaluator.isTransformationNeeded(pipelinesDataFlowModel); - // Assert assertTrue(result.isEvaluatedResult()); assertEquals(result.getPipelineName(), pipelineName); } @Test - void test_isTransformationNeeded_ForOtherSource_ShouldReturn_False() { - // Set up + void test_isTransformationNeeded_ForOtherSource_ShouldReturn_False() throws IOException { + + String pluginName = "http"; String pipelineName = "test-pipeline"; - Map sourceOptions = new HashMap(); - sourceOptions.put("option1", "1"); - sourceOptions.put("option2", null); - final PluginModel source = new PluginModel("http", sourceOptions); + Map sourceOptions = new HashMap<>(); + Map s3Bucket = new HashMap<>(); + s3Bucket.put("s3_bucket", "bucket-name"); + sourceOptions.put("s3_bucket", s3Bucket); + final PluginModel source = new PluginModel(pluginName, sourceOptions); final List processors = Collections.singletonList(new PluginModel("testProcessor", null)); final List sinks = Collections.singletonList(new SinkModel("testSink", Collections.emptyList(), null, Collections.emptyList(), Collections.emptyList(), null)); final PipelineModel pipelineModel = new PipelineModel(source, null, processors, null, sinks, 8, 50); + TransformersFactory transformersFactory = mock(TransformersFactory.class); + final PipelinesDataFlowModel pipelinesDataFlowModel = new PipelinesDataFlowModel( (PipelineExtensions) null, Collections.singletonMap(pipelineName, pipelineModel)); - TransformersFactory transformersFactory = Mockito.spy(new TransformersFactory( - TestConfigurationProvider.RULES_TRANSFORMATION_DIRECTORY, - TestConfigurationProvider.TEMPLATES_SOURCE_TRANSFORMATION_DIRECTORY - )); RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); RuleEvaluatorResult result = ruleEvaluator.isTransformationNeeded(pipelinesDataFlowModel); - // Assert - assertEquals(result.isEvaluatedResult(), false); + assertFalse(result.isEvaluatedResult()); } @Test void testThrowsExceptionOnFileError() { TransformersFactory transformersFactory = mock(TransformersFactory.class); String pipelineName = "test-pipeline"; - Map sourceOptions = new HashMap(); + Map sourceOptions = new HashMap<>(); sourceOptions.put("option1", "1"); sourceOptions.put("option2", null); final PluginModel source = new PluginModel("http", sourceOptions); @@ -109,16 +108,14 @@ void testThrowsExceptionOnFileError() { final PipelinesDataFlowModel pipelinesDataFlowModel = new PipelinesDataFlowModel( (PipelineExtensions) null, Collections.singletonMap(pipelineName, pipelineModel)); - // Setup mock to throw an exception when file path is incorrect - when(transformersFactory.getPluginRuleFileLocation("documentdb")).thenThrow(new RuntimeException("File not found")); - when(transformersFactory.getPluginRuleFileStream("documentdb")).thenThrow(new RuntimeException("File not found")); + when(transformersFactory.loadRules()).thenThrow(new RuntimeException("File not found")); RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); - // Execute and Assert Exception exception = assertThrows(RuntimeException.class, () -> { ruleEvaluator.isTransformationNeeded(pipelinesDataFlowModel); }); + assertEquals("File not found", exception.getMessage()); } } diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleTransformerModelTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleTransformerModelTest.java index 2cd6f0fad4..02a6370229 100644 --- a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleTransformerModelTest.java +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/rule/RuleTransformerModelTest.java @@ -19,7 +19,8 @@ class RuleTransformerModelTest { @Test void testSerialization() throws Exception { List applyWhen = Arrays.asList("condition1", "condition2"); - RuleTransformerModel model = new RuleTransformerModel(applyWhen); + String pluginName = "testPlugin"; + RuleTransformerModel model = new RuleTransformerModel(applyWhen, pluginName); String json = objectMapper.writeValueAsString(model); assertNotNull(json, "Serialized JSON should not be null"); @@ -27,12 +28,13 @@ void testSerialization() throws Exception { @Test void testDeserialization() throws Exception { - String json = "{\"apply_when\":[\"condition1\",\"condition2\"]}"; + String json = "{\"plugin_name\": \"testPlugin\", \"apply_when\": [\"condition1\", \"condition2\"]}"; RuleTransformerModel model = objectMapper.readValue(json, RuleTransformerModel.class); assertNotNull(model, "Deserialized model should not be null"); assertEquals(2, model.getApplyWhen().size(), "ApplyWhen should contain two conditions"); assertEquals("condition1", model.getApplyWhen().get(0), "The first condition should be 'condition1'"); assertEquals("condition2", model.getApplyWhen().get(1), "The second condition should be 'condition2'"); + assertEquals("testPlugin", model.getPluginName(), "plugin Name"); } } diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/transformer/DynamicConfigTransformerTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/transformer/DynamicConfigTransformerTest.java index c5e0aea18d..a1f417054c 100644 --- a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/transformer/DynamicConfigTransformerTest.java +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/transformer/DynamicConfigTransformerTest.java @@ -11,7 +11,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import org.junit.jupiter.api.RepeatedTest; import org.junit.jupiter.api.Test; -import org.mockito.Mockito; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; import org.opensearch.dataprepper.model.configuration.PipelinesDataFlowModel; import org.opensearch.dataprepper.pipeline.parser.PipelineConfigurationFileReader; @@ -19,26 +19,28 @@ import org.opensearch.dataprepper.pipeline.parser.PipelinesDataflowModelParser; import org.opensearch.dataprepper.pipeline.parser.TestConfigurationProvider; import org.opensearch.dataprepper.pipeline.parser.rule.RuleEvaluator; +import org.opensearch.dataprepper.pipeline.parser.rule.RuleStream; import java.io.File; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.List; import java.util.Map; class DynamicConfigTransformerTest { private final ObjectMapper yamlMapper = new ObjectMapper(new YAMLFactory() .disable(YAMLGenerator.Feature.WRITE_DOC_START_MARKER)); - private final String RULES_DIRECTORY_PATH = "src/test/resources/transformation/rules"; - private final String TEMPLATES_DIRECTORY_PATH = "src/test/resources/transformation/templates/testSource"; TransformersFactory transformersFactory; RuleEvaluator ruleEvaluator; @Test void test_successful_transformation_with_only_source_and_sink() throws IOException { - String docDBUserConfig = TestConfigurationProvider.USER_CONFIG_TRANSFORMATION_DOCDB1_CONFIG_FILE; String templateDocDBFilePath = TestConfigurationProvider.TEMPLATE_TRANSFORMATION_DOCDB1_CONFIG_FILE; String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCDB1_CONFIG_FILE; @@ -48,15 +50,15 @@ void test_successful_transformation_with_only_source_and_sink() throws IOExcepti final PipelinesDataflowModelParser pipelinesDataflowModelParser = new PipelinesDataflowModelParser(pipelineConfigurationReader); - transformersFactory = Mockito.spy(new TransformersFactory(RULES_DIRECTORY_PATH, - TEMPLATES_DIRECTORY_PATH)); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); - when(transformersFactory.getPluginTemplateFileLocation(pluginName)).thenReturn(templateDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); InputStream ruleStream = new FileInputStream(ruleDocDBFilePath); InputStream templateStream = new FileInputStream(templateDocDBFilePath); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream); + RuleStream ruleInputStream = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream); + List ruleStreams = Collections.singletonList(ruleInputStream); + when(transformersFactory.loadRules()).thenReturn(ruleStreams); when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); - ruleEvaluator = new RuleEvaluator(transformersFactory); + + RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); // Load the original and template YAML files from the test resources directory PipelinesDataFlowModel pipelinesDataFlowModel = pipelinesDataflowModelParser.parseConfiguration(); @@ -74,20 +76,21 @@ void test_successful_transformation_with_documentdb() throws IOException { String docDBUserConfig = TestConfigurationProvider.USER_CONFIG_TRANSFORMATION_DOCUMENTDB_CONFIG_FILE; String templateDocDBFilePath = TestConfigurationProvider.TEMPLATE_TRANSFORMATION_DOCUMENTDB_CONFIG_FILE; - String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCUMENTDB_CONFIG_FILE; + String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCDB1_CONFIG_FILE; String expectedDocDBFilePath = TestConfigurationProvider.EXPECTED_TRANSFORMATION_DOCUMENTDB_CONFIG_FILE; String pluginName = "documentdb"; PipelineConfigurationReader pipelineConfigurationReader = new PipelineConfigurationFileReader(docDBUserConfig); final PipelinesDataflowModelParser pipelinesDataflowModelParser = new PipelinesDataflowModelParser(pipelineConfigurationReader); - transformersFactory = Mockito.spy(new TransformersFactory(RULES_DIRECTORY_PATH, - TEMPLATES_DIRECTORY_PATH)); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); - when(transformersFactory.getPluginTemplateFileLocation(pluginName)).thenReturn(templateDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); + Path ruleFile = mock(Path.class); + when(ruleFile.getFileName()).thenReturn(Paths.get(ruleDocDBFilePath).getFileName()); InputStream ruleStream = new FileInputStream(ruleDocDBFilePath); InputStream templateStream = new FileInputStream(templateDocDBFilePath); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream); + RuleStream ruleInputStream = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream); + List ruleStreams = Collections.singletonList(ruleInputStream); + when(transformersFactory.loadRules()).thenReturn(ruleStreams); when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); ruleEvaluator = new RuleEvaluator(transformersFactory); @@ -103,28 +106,31 @@ void test_successful_transformation_with_documentdb() throws IOException { } @RepeatedTest(5) - @Test void test_successful_transformation_with_subpipelines() throws IOException { String docDBUserConfig = TestConfigurationProvider.USER_CONFIG_TRANSFORMATION_DOCUMENTDB_SUBPIPELINES_CONFIG_FILE; String templateDocDBFilePath = TestConfigurationProvider.TEMPLATE_TRANSFORMATION_DOCUMENTDB_SUBPIPELINES_CONFIG_FILE; - String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCUMENTDB_CONFIG_FILE; + String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCDB1_CONFIG_FILE; String expectedDocDBFilePath = TestConfigurationProvider.EXPECTED_TRANSFORMATION_DOCUMENTDB_SUBPIPLINES_CONFIG_FILE; String pluginName = "documentdb"; PipelineConfigurationReader pipelineConfigurationReader = new PipelineConfigurationFileReader(docDBUserConfig); final PipelinesDataflowModelParser pipelinesDataflowModelParser = new PipelinesDataflowModelParser(pipelineConfigurationReader); - transformersFactory = Mockito.spy(new TransformersFactory(RULES_DIRECTORY_PATH, - TEMPLATES_DIRECTORY_PATH)); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); - when(transformersFactory.getPluginTemplateFileLocation(pluginName)).thenReturn(templateDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); + InputStream ruleStream1 = new FileInputStream(ruleDocDBFilePath); InputStream ruleStream2 = new FileInputStream(ruleDocDBFilePath); InputStream templateStream = new FileInputStream(templateDocDBFilePath); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream1).thenReturn(ruleStream2); + RuleStream ruleInputStream1 = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream1); + RuleStream ruleInputStream2 = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream2); + + List ruleStreams1 = Collections.singletonList(ruleInputStream1); + List ruleStreams2 = Collections.singletonList(ruleInputStream2); + when(transformersFactory.loadRules()).thenReturn(ruleStreams1).thenReturn(ruleStreams2); when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); - ruleEvaluator = new RuleEvaluator(transformersFactory); + + RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); // Load the original and template YAML files from the test resources directory PipelinesDataFlowModel pipelinesDataFlowModel = pipelinesDataflowModelParser.parseConfiguration(); @@ -139,7 +145,6 @@ void test_successful_transformation_with_subpipelines() throws IOException { @Test void test_successful_transformation_with_functionPlaceholder() throws IOException { - String docDBUserConfig = TestConfigurationProvider.USER_CONFIG_TRANSFORMATION_DOCUMENTDB_FUNCTION_CONFIG_FILE; String templateDocDBFilePath = TestConfigurationProvider.TEMPLATE_TRANSFORMATION_DOCUMENTDB_FUNCTION_CONFIG_FILE; String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCDB1_CONFIG_FILE; @@ -149,15 +154,17 @@ void test_successful_transformation_with_functionPlaceholder() throws IOExceptio final PipelinesDataflowModelParser pipelinesDataflowModelParser = new PipelinesDataflowModelParser(pipelineConfigurationReader); - transformersFactory = Mockito.spy(new TransformersFactory(RULES_DIRECTORY_PATH, - TEMPLATES_DIRECTORY_PATH)); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); - when(transformersFactory.getPluginTemplateFileLocation(pluginName)).thenReturn(templateDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); + InputStream ruleStream = new FileInputStream(ruleDocDBFilePath); InputStream templateStream = new FileInputStream(templateDocDBFilePath); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream); + RuleStream ruleInputStream = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream); + + List ruleStreams = Collections.singletonList(ruleInputStream); + when(transformersFactory.loadRules()).thenReturn(ruleStreams); when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); - ruleEvaluator = new RuleEvaluator(transformersFactory); + + RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); // Load the original and template YAML files from the test resources directory PipelinesDataFlowModel pipelinesDataFlowModel = pipelinesDataflowModelParser.parseConfiguration(); @@ -170,10 +177,8 @@ void test_successful_transformation_with_functionPlaceholder() throws IOExceptio assertThat(expectedYamlasMap).usingRecursiveComparison().isEqualTo(transformedYamlasMap); } - @Test void test_successful_transformation_with_complete_template() throws IOException { - String docDBUserConfig = TestConfigurationProvider.USER_CONFIG_TRANSFORMATION_DOCDB2_CONFIG_FILE; String templateDocDBFilePath = TestConfigurationProvider.TEMPLATE_TRANSFORMATION_DOCDB2_CONFIG_FILE; String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCDB1_CONFIG_FILE; @@ -183,15 +188,17 @@ void test_successful_transformation_with_complete_template() throws IOException final PipelinesDataflowModelParser pipelinesDataflowModelParser = new PipelinesDataflowModelParser(pipelineConfigurationReader); - transformersFactory = Mockito.spy(new TransformersFactory(RULES_DIRECTORY_PATH, - TEMPLATES_DIRECTORY_PATH)); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); - when(transformersFactory.getPluginTemplateFileLocation(pluginName)).thenReturn(templateDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); + InputStream ruleStream = new FileInputStream(ruleDocDBFilePath); InputStream templateStream = new FileInputStream(templateDocDBFilePath); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream); + RuleStream ruleInputStream = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream); + + List ruleStreams = Collections.singletonList(ruleInputStream); + when(transformersFactory.loadRules()).thenReturn(ruleStreams); when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); - ruleEvaluator = new RuleEvaluator(transformersFactory); + + RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); // Load the original and template YAML files from the test resources directory PipelinesDataFlowModel pipelinesDataFlowModel = pipelinesDataflowModelParser.parseConfiguration(); @@ -205,9 +212,9 @@ void test_successful_transformation_with_complete_template() throws IOException assertThat(expectedYamlasMap).usingRecursiveComparison().isEqualTo(transformedYamlasMap); } + @Test void test_successful_transformation_with_routes_keyword() throws IOException { - String docDBUserConfig = TestConfigurationProvider.USER_CONFIG_TRANSFORMATION_DOCUMENTDB_ROUTES_CONFIG_FILE; String templateDocDBFilePath = TestConfigurationProvider.TEMPLATE_TRANSFORMATION_DOCUMENTDB_FINAL_CONFIG_FILE; String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCDB1_CONFIG_FILE; @@ -217,15 +224,17 @@ void test_successful_transformation_with_routes_keyword() throws IOException { final PipelinesDataflowModelParser pipelinesDataflowModelParser = new PipelinesDataflowModelParser(pipelineConfigurationReader); - transformersFactory = Mockito.spy(new TransformersFactory(RULES_DIRECTORY_PATH, - TEMPLATES_DIRECTORY_PATH)); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); - when(transformersFactory.getPluginTemplateFileLocation(pluginName)).thenReturn(templateDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); + InputStream ruleStream = new FileInputStream(ruleDocDBFilePath); InputStream templateStream = new FileInputStream(templateDocDBFilePath); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream); + RuleStream ruleInputStream = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream); + + List ruleStreams = Collections.singletonList(ruleInputStream); + when(transformersFactory.loadRules()).thenReturn(ruleStreams); when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); - ruleEvaluator = new RuleEvaluator(transformersFactory); + + RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); // Load the original and template YAML files from the test resources directory PipelinesDataFlowModel pipelinesDataFlowModel = pipelinesDataflowModelParser.parseConfiguration(); @@ -239,11 +248,8 @@ void test_successful_transformation_with_routes_keyword() throws IOException { assertThat(expectedYamlasMap).usingRecursiveComparison().isEqualTo(transformedYamlasMap); } - - @Test void test_successful_transformation_with_route_keyword() throws IOException { - String docDBUserConfig = TestConfigurationProvider.USER_CONFIG_TRANSFORMATION_DOCUMENTDB_ROUTE_CONFIG_FILE; String templateDocDBFilePath = TestConfigurationProvider.TEMPLATE_TRANSFORMATION_DOCUMENTDB_FINAL_CONFIG_FILE; String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCDB1_CONFIG_FILE; @@ -254,15 +260,17 @@ void test_successful_transformation_with_route_keyword() throws IOException { final PipelinesDataflowModelParser pipelinesDataflowModelParser = new PipelinesDataflowModelParser(pipelineConfigurationReader); - transformersFactory = Mockito.spy(new TransformersFactory(RULES_DIRECTORY_PATH, - TEMPLATES_DIRECTORY_PATH)); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); - when(transformersFactory.getPluginTemplateFileLocation(pluginName)).thenReturn(templateDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); + InputStream ruleStream = new FileInputStream(ruleDocDBFilePath); InputStream templateStream = new FileInputStream(templateDocDBFilePath); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream); + RuleStream ruleInputStream = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream); + + List ruleStreams = Collections.singletonList(ruleInputStream); + when(transformersFactory.loadRules()).thenReturn(ruleStreams); when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); - ruleEvaluator = new RuleEvaluator(transformersFactory); + + RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); // Load the original and template YAML files from the test resources directory PipelinesDataFlowModel pipelinesDataFlowModel = pipelinesDataflowModelParser.parseConfiguration(); @@ -276,10 +284,8 @@ void test_successful_transformation_with_route_keyword() throws IOException { assertThat(expectedYamlasMap).usingRecursiveComparison().isEqualTo(transformedYamlasMap); } - @Test void test_successful_transformation_with_routes_and_subpipelines() throws IOException { - String docDBUserConfig = TestConfigurationProvider.USER_CONFIG_TRANSFORMATION_DOCUMENTDB_SUBPIPELINES_ROUTES_CONFIG_FILE; String templateDocDBFilePath = TestConfigurationProvider.TEMPLATE_TRANSFORMATION_DOCUMENTDB_FINAL_CONFIG_FILE; String ruleDocDBFilePath = TestConfigurationProvider.RULES_TRANSFORMATION_DOCDB1_CONFIG_FILE; @@ -289,18 +295,24 @@ void test_successful_transformation_with_routes_and_subpipelines() throws IOExce final PipelinesDataflowModelParser pipelinesDataflowModelParser = new PipelinesDataflowModelParser(pipelineConfigurationReader); - transformersFactory = Mockito.spy(new TransformersFactory(RULES_DIRECTORY_PATH, - TEMPLATES_DIRECTORY_PATH)); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); - when(transformersFactory.getPluginTemplateFileLocation(pluginName)).thenReturn(templateDocDBFilePath); - InputStream ruleStream = new FileInputStream(ruleDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); + + InputStream ruleStream1 = new FileInputStream(ruleDocDBFilePath); InputStream ruleStream2 = new FileInputStream(ruleDocDBFilePath); InputStream ruleStream3 = new FileInputStream(ruleDocDBFilePath); InputStream templateStream = new FileInputStream(templateDocDBFilePath); + RuleStream ruleInputStream1 = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream1); + RuleStream ruleInputStream2 = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream2); + RuleStream ruleInputStream3 = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream3); + + List ruleStreams1 = Collections.singletonList(ruleInputStream1); + List ruleStreams2 = Collections.singletonList(ruleInputStream2); + List ruleStreams3 = Collections.singletonList(ruleInputStream3); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream).thenReturn(ruleStream2).thenReturn(ruleStream3); + when(transformersFactory.loadRules()).thenReturn(ruleStreams1).thenReturn(ruleStreams2).thenReturn(ruleStreams3); when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); - ruleEvaluator = new RuleEvaluator(transformersFactory); + + RuleEvaluator ruleEvaluator = new RuleEvaluator(transformersFactory); // Load the original and template YAML files from the test resources directory PipelinesDataFlowModel pipelinesDataFlowModel = pipelinesDataflowModelParser.parseConfiguration(); @@ -324,14 +336,16 @@ void testInvalidJsonPathThrowsException() throws IOException { final PipelinesDataflowModelParser pipelinesDataflowModelParser = new PipelinesDataflowModelParser(pipelineConfigurationReader); - transformersFactory = Mockito.spy(new TransformersFactory(RULES_DIRECTORY_PATH, - TEMPLATES_DIRECTORY_PATH)); - when(transformersFactory.getPluginRuleFileLocation(pluginName)).thenReturn(ruleDocDBFilePath); - when(transformersFactory.getPluginTemplateFileLocation(pluginName)).thenReturn(templateDocDBFilePath); + TransformersFactory transformersFactory = mock(TransformersFactory.class); + InputStream ruleStream = new FileInputStream(ruleDocDBFilePath); InputStream templateStream = new FileInputStream(templateDocDBFilePath); - when(transformersFactory.getPluginRuleFileStream(pluginName)).thenReturn(ruleStream); + RuleStream ruleInputStream = new RuleStream(Paths.get(ruleDocDBFilePath).getFileName().toString(), ruleStream); + + List ruleStreams = Collections.singletonList(ruleInputStream); + when(transformersFactory.loadRules()).thenReturn(ruleStreams); when(transformersFactory.getPluginTemplateFileStream(pluginName)).thenReturn(templateStream); + ruleEvaluator = new RuleEvaluator(transformersFactory); // Load the original and template YAML files from the test resources directory diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/transformer/TransformersFactoryTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/transformer/TransformersFactoryTest.java index 4b3b8df0e4..ff5b6f21c6 100644 --- a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/transformer/TransformersFactoryTest.java +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/pipeline/parser/transformer/TransformersFactoryTest.java @@ -1,76 +1,92 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ package org.opensearch.dataprepper.pipeline.parser.transformer; -import com.fasterxml.jackson.databind.ObjectMapper; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.mockito.Mockito; +import org.opensearch.dataprepper.pipeline.parser.rule.RuleStream; -import java.io.File; -import java.io.IOException; +import java.io.InputStream; +import java.util.Collection; -public class TransformersFactoryTest { +class TransformersFactoryTest { private TransformersFactory transformersFactory; - private final String templatesDirectoryPath = "src/test/resources/templates"; - private final String rulesDirectoryPath = "src/test/resources/rules"; - private final String validPluginName = "testPlugin"; - private final String invalidPluginName = ""; @BeforeEach - public void setUp() { - transformersFactory = new TransformersFactory(rulesDirectoryPath, templatesDirectoryPath); + void setUp() { + transformersFactory = new TransformersFactory(); } @Test - public void testGetPluginTemplateFileLocation_validPluginName() { - String expectedPath = templatesDirectoryPath + "/" + validPluginName + "-template.yaml"; - assertEquals(expectedPath, transformersFactory.getPluginTemplateFileLocation(validPluginName)); + void testGetPluginTemplateFileStream_whenTemplateExists_shouldReturnInputStream() throws Exception { + String pluginName = "test-plugin"; + + // Load the actual resource + InputStream inputStream = transformersFactory.getPluginTemplateFileStream(pluginName); + + assertNotNull(inputStream); + inputStream.close(); } @Test - public void testGetPluginTemplateFileLocation_invalidPluginName() { + void testGetPluginTemplateFileStream_whenTemplateDoesNotExist_shouldThrowException() { + String pluginName = "non-existent-plugin"; + Exception exception = assertThrows(RuntimeException.class, () -> { - transformersFactory.getPluginTemplateFileLocation(invalidPluginName); + transformersFactory.getPluginTemplateFileStream(pluginName); }); - assertEquals("Transformation plugin not found", exception.getMessage()); + + assertEquals("Template file not found for plugin: " + pluginName, exception.getMessage()); + } + + @Test + void testLoadRules_whenRulesExist_shouldReturnRuleStreams() throws Exception { + Collection ruleStreams = transformersFactory.loadRules(); + + assertNotNull(ruleStreams); + assertFalse(ruleStreams.isEmpty()); + + for (RuleStream ruleStream : ruleStreams) { + assertNotNull(ruleStream.getRuleStream()); + assertNotNull(ruleStream.getName()); + } } @Test - public void testGetPluginRuleFileLocation_validPluginName() { - String expectedPath = rulesDirectoryPath + "/" + validPluginName + "-rule.yaml"; - assertEquals(expectedPath, transformersFactory.getPluginRuleFileLocation(validPluginName)); + void testLoadRules_whenFilesExist_shouldReturnRuleStreams() throws Exception { + // Ensure the rules directory has at least one file + Collection ruleStreams = transformersFactory.loadRules(); + + assertNotNull(ruleStreams); + assertFalse(ruleStreams.isEmpty()); + + for (RuleStream ruleStream : ruleStreams) { + assertNotNull(ruleStream.getRuleStream()); + assertNotNull(ruleStream.getName()); + assertTrue(ruleStream.getName().endsWith("-rule.yaml")); + } } @Test - public void testGetPluginRuleFileLocation_invalidPluginName() { + void testGetPluginTemplateFileStream_whenPluginNameIsNull_shouldThrowException() { Exception exception = assertThrows(RuntimeException.class, () -> { - transformersFactory.getPluginRuleFileLocation(invalidPluginName); + transformersFactory.getPluginTemplateFileStream(null); }); + assertEquals("Transformation plugin not found", exception.getMessage()); } @Test - public void testGetTemplateModel_throwsRuntimeExceptionOnIOException() throws IOException { - ObjectMapper mockedYamlMapper = Mockito.mock(ObjectMapper.class); - String templatePath = templatesDirectoryPath + "/" + validPluginName + "-template.yaml"; - File expectedFile = new File(templatePath); - - Mockito.when(mockedYamlMapper.readValue(Mockito.eq(expectedFile), Mockito.eq(PipelineTemplateModel.class))) - .thenThrow(new IOException("Test exception")); + void testGetPluginTemplateFileStream_whenPluginNameIsEmpty_shouldThrowException() { + Exception exception = assertThrows(RuntimeException.class, () -> { + transformersFactory.getPluginTemplateFileStream(""); + }); - assertThrows(RuntimeException.class, () -> transformersFactory.getTemplateModel(validPluginName)); + assertEquals("Transformation plugin not found", exception.getMessage()); } - @Test - public void testGetTemplateModel_invalidPluginNameThrowsRuntimeException() { - assertThrows(RuntimeException.class, () -> transformersFactory.getTemplateModel(invalidPluginName), - "Should throw a RuntimeException for empty plugin name."); - } } - diff --git a/data-prepper-pipeline-parser/src/test/resources/org/opensearch/dataprepper/transforms/rules/test-plugin-rule.yaml b/data-prepper-pipeline-parser/src/test/resources/org/opensearch/dataprepper/transforms/rules/test-plugin-rule.yaml new file mode 100644 index 0000000000..bdee2aaacf --- /dev/null +++ b/data-prepper-pipeline-parser/src/test/resources/org/opensearch/dataprepper/transforms/rules/test-plugin-rule.yaml @@ -0,0 +1,3 @@ +plugin_name: "test-plugin" +apply_when: + - "$..source.documentdb" \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/test/resources/org/opensearch/dataprepper/transforms/templates/test-plugin-template.yaml b/data-prepper-pipeline-parser/src/test/resources/org/opensearch/dataprepper/transforms/templates/test-plugin-template.yaml new file mode 100644 index 0000000000..e9e95b4ff0 --- /dev/null +++ b/data-prepper-pipeline-parser/src/test/resources/org/opensearch/dataprepper/transforms/templates/test-plugin-template.yaml @@ -0,0 +1,4 @@ +"<>-transformed": + source: "<<$.<>.source>>" + sink: + - noop: diff --git a/data-prepper-pipeline-parser/src/test/resources/transformation/rules/documentdb-rule.yaml b/data-prepper-pipeline-parser/src/test/resources/transformation/rules/documentdb-rule.yaml index 28da1406f8..b120d1531c 100644 --- a/data-prepper-pipeline-parser/src/test/resources/transformation/rules/documentdb-rule.yaml +++ b/data-prepper-pipeline-parser/src/test/resources/transformation/rules/documentdb-rule.yaml @@ -1,3 +1,4 @@ +plugin_name: "documentdb" apply_when: - "$..source.documentdb" - - "$..source.documentdb.collections[0].s3_bucket" \ No newline at end of file + - "$..source.documentdb.s3_bucket" diff --git a/data-prepper-pipeline-parser/src/test/resources/transformation/rules/documentdb1-rule.yaml b/data-prepper-pipeline-parser/src/test/resources/transformation/rules/documentdb1-rule.yaml index cb10684065..ed3c4b8b57 100644 --- a/data-prepper-pipeline-parser/src/test/resources/transformation/rules/documentdb1-rule.yaml +++ b/data-prepper-pipeline-parser/src/test/resources/transformation/rules/documentdb1-rule.yaml @@ -1,2 +1,3 @@ +plugin_name: "documentdb" apply_when: - "$..source.documentdb" \ No newline at end of file diff --git a/data-prepper-plugins/dummy-plugin/README.md b/data-prepper-plugins/dummy-plugin/README.md new file mode 100644 index 0000000000..b741ffe5ba --- /dev/null +++ b/data-prepper-plugins/dummy-plugin/README.md @@ -0,0 +1,63 @@ +A dummy plugin is used specifically when we want to use pipeline transformation for a +plugin that does not exist. + +We can define a rule and template for this plugin by creating a plugin folder and +place rule for which the pipeline configuration would be valid and a corresponding +template to transform to when the rule is valid. + +For further details on transformation refer: +/docs/pipeline_configuration_transformation.md + + +For Example: + +User Config: +```yaml +test-pipeline: + source: + file: + path: "/full/path/to/logs_json.log" + record_type: "event" + format: "json" + processor: + - dummy_plugin: + sink: + - stdout: +``` + +Here dummy_plugin is not really a plugin that is defined in dataprepper, but we can +use the pipeline transformation to convert the user-config to different config based +on the template. + +Rule: +```yaml +plugin_name: "dummy-plugin" +apply_when: + - "$.test-pipeline.processor[?(@.dummy_plugin)]" +``` + +Template: +```yaml +test-pipeline: + source: + file: + path: "/tmp/input-file.log" + processor: + - string_converter: + upper_case: true + sink: + - noop: +``` + +Output: +```yaml +test-pipeline: + source: + file: + path: "/tmp/input-file.log" + processor: + - string_converter: + upper_case: true + sink: + - noop: +``` \ No newline at end of file diff --git a/data-prepper-plugins/dummy-plugin/build.gradle b/data-prepper-plugins/dummy-plugin/build.gradle new file mode 100644 index 0000000000..e69de29bb2 diff --git a/data-prepper-plugins/dummy-plugin/src/main/resources/org/opensearch/dataprepper/transforms/rules/dummy-plugin-rule.yaml b/data-prepper-plugins/dummy-plugin/src/main/resources/org/opensearch/dataprepper/transforms/rules/dummy-plugin-rule.yaml new file mode 100644 index 0000000000..6c26f86b2a --- /dev/null +++ b/data-prepper-plugins/dummy-plugin/src/main/resources/org/opensearch/dataprepper/transforms/rules/dummy-plugin-rule.yaml @@ -0,0 +1,3 @@ +plugin_name: "dummy-plugin" +apply_when: + - "$.test-pipeline.processor[?(@.dummy_plugin)]" diff --git a/data-prepper-plugins/dummy-plugin/src/main/resources/org/opensearch/dataprepper/transforms/templates/dummy-plugin-template.yaml b/data-prepper-plugins/dummy-plugin/src/main/resources/org/opensearch/dataprepper/transforms/templates/dummy-plugin-template.yaml new file mode 100644 index 0000000000..bbf00a285e --- /dev/null +++ b/data-prepper-plugins/dummy-plugin/src/main/resources/org/opensearch/dataprepper/transforms/templates/dummy-plugin-template.yaml @@ -0,0 +1,9 @@ +test-pipeline: + source: + file: + path: "/tmp/input-file.log" + processor: + - string_converter: + upper_case: true + sink: + - noop: \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/main/resources/rules/documentdb-rule.yaml b/data-prepper-plugins/mongodb/src/main/resources/org/opensearch/dataprepper/transforms/rules/documentdb-rule.yaml similarity index 74% rename from data-prepper-pipeline-parser/src/main/resources/rules/documentdb-rule.yaml rename to data-prepper-plugins/mongodb/src/main/resources/org/opensearch/dataprepper/transforms/rules/documentdb-rule.yaml index e7279ee733..60aa428d8a 100644 --- a/data-prepper-pipeline-parser/src/main/resources/rules/documentdb-rule.yaml +++ b/data-prepper-plugins/mongodb/src/main/resources/org/opensearch/dataprepper/transforms/rules/documentdb-rule.yaml @@ -1,3 +1,4 @@ +plugin_name: "documentdb" apply_when: - "$..source.documentdb" - "$..source.documentdb.s3_bucket" \ No newline at end of file diff --git a/data-prepper-plugins/mongodb/src/main/resources/org/opensearch/dataprepper/transforms/templates/documentdb-template.yaml b/data-prepper-plugins/mongodb/src/main/resources/org/opensearch/dataprepper/transforms/templates/documentdb-template.yaml new file mode 100644 index 0000000000..38bb70d8ca --- /dev/null +++ b/data-prepper-plugins/mongodb/src/main/resources/org/opensearch/dataprepper/transforms/templates/documentdb-template.yaml @@ -0,0 +1,81 @@ +"<>": + workers: "<<$.<>.workers>>" + delay: "<<$.<>.delay>>" + buffer: "<<$.<>.buffer>>" + source: + documentdb: "<<$.<>.source.documentdb>>" + routes: + - initial_load: 'getMetadata("ingestion_type") == "EXPORT"' + - stream_load: 'getMetadata("ingestion_type") == "STREAM"' + sink: + - s3: + routes: + - initial_load + aws: + region: "<<$.<>.source.documentdb.s3_region>>" + sts_role_arn: "<<$.<>.source.documentdb.aws.sts_role_arn>>" + sts_external_id: "<<$.<>.source.documentdb.aws.sts_external_id>>" + sts_header_overrides: "<<$.<>.source.documentdb.aws.sts_header_overrides>>" + bucket: "<<$.<>.source.documentdb.s3_bucket>>" + threshold: + event_collect_timeout: "120s" + maximum_size: "2mb" + aggregate_threshold: + maximum_size: "128mb" + flush_capacity_ratio: 0 + object_key: + path_prefix: "${getMetadata(\"s3_partition_key\")}" + codec: + event_json: + default_bucket_owner: "<>.source.documentdb.aws.sts_role_arn>>" + - s3: + routes: + - stream_load + aws: + region: "<<$.<>.source.documentdb.s3_region>>" + sts_role_arn: "<<$.<>.source.documentdb.aws.sts_role_arn>>" + sts_external_id: "<<$.<>.source.documentdb.aws.sts_external_id>>" + sts_header_overrides: "<<$.<>.source.documentdb.aws.sts_header_overrides>>" + bucket: "<<$.<>.source.documentdb.s3_bucket>>" + threshold: + event_collect_timeout: "15s" + maximum_size: "1mb" + aggregate_threshold: + maximum_size: "128mb" + flush_capacity_ratio: 0 + object_key: + path_prefix: "${getMetadata(\"s3_partition_key\")}" + codec: + event_json: + default_bucket_owner: "<>.source.documentdb.aws.sts_role_arn>>" +"<>-s3": + workers: "<<$.<>.workers>>" + delay: "<<$.<>.delay>>" + buffer: "<<$.<>.buffer>>" + source: + s3: + codec: + event_json: + compression: "none" + aws: + region: "<<$.<>.source.documentdb.s3_region>>" + sts_role_arn: "<<$.<>.source.documentdb.aws.sts_role_arn>>" + sts_external_id: "<<$.<>.source.documentdb.aws.sts_external_id>>" + sts_header_overrides: "<<$.<>.source.documentdb.aws.sts_header_overrides>>" + acknowledgments: true + delete_s3_objects_on_read: true + disable_s3_metadata_in_event: true + scan: + folder_partitions: + depth: "<>.source.documentdb.s3_prefix>>" + max_objects_per_ownership: 50 + buckets: + - bucket: + name: "<<$.<>.source.documentdb.s3_bucket>>" + filter: + include_prefix: ["<>.source.documentdb.s3_prefix>>"] + scheduling: + interval: "60s" + processor: "<<$.<>.processor>>" + sink: "<<$.<>.sink>>" + routes: "<<$.<>.routes>>" # In placeholder, routes or route (defined as alias) will be transformed to route in json as route will be primarily picked in pipelineModel. \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index 63e7ad2a9f..20edb79287 100644 --- a/settings.gradle +++ b/settings.gradle @@ -180,4 +180,5 @@ include 'data-prepper-plugins:mongodb' include 'data-prepper-plugins:rds-source' include 'data-prepper-plugins:http-source-common' include 'data-prepper-plugins:http-common' -include 'data-prepper-plugins:aws-lambda' \ No newline at end of file +include 'data-prepper-plugins:aws-lambda' +//include 'data-prepper-plugins:dummy-plugin' \ No newline at end of file From 04098b3287cc796b8962f91c765996ebc1932250 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 19 Aug 2024 13:12:27 -0500 Subject: [PATCH 118/159] Update the parse JSON/XML/ION processors to use EventKey. (#4842) Signed-off-by: David Venable --- .../parse/AbstractParseProcessor.java | 30 +++++++++++-------- .../parse/ion/ParseIonProcessor.java | 6 ++-- .../parse/json/ParseJsonProcessor.java | 6 ++-- .../parse/xml/ParseXmlProcessor.java | 6 ++-- .../parse/ion/ParseIonProcessorTest.java | 4 +-- .../parse/json/ParseJsonProcessorTest.java | 25 +++++++++------- .../parse/xml/ParseXmlProcessorTest.java | 15 ++++++---- 7 files changed, 55 insertions(+), 37 deletions(-) diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java index 878316c183..ffb0855590 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java @@ -9,6 +9,8 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.record.Record; @@ -30,8 +32,8 @@ public abstract class AbstractParseProcessor extends AbstractProcessor, Record> { private static final Logger LOG = LoggerFactory.getLogger(AbstractParseProcessor.class); - private final String source; - private final String destination; + private final EventKey source; + private final EventKey destination; private final String pointer; private final String parseWhen; private final List tagsOnFailure; @@ -39,20 +41,23 @@ public abstract class AbstractParseProcessor extends AbstractProcessor> doExecute(final Collection> recor if(deleteSourceRequested) { event.delete(this.source); } - } catch (Exception e) { + } catch (final Exception e) { LOG.error(EVENT, "An exception occurred while using the {} processor on Event [{}]", getProcessorName(), record.getData(), e); } } @@ -128,7 +133,8 @@ private String getProcessorName() { private Map parseUsingPointer(final Event event, final Map parsedJson, final String pointer, final boolean doWriteToRoot) { final Event temporaryEvent = JacksonEvent.builder().withEventType("event").build(); - temporaryEvent.put(source, parsedJson); + final EventKey temporaryPutKey = eventKeyFactory.createEventKey(source.getKey(), EventKeyFactory.EventAction.PUT); + temporaryEvent.put(temporaryPutKey, parsedJson); final String trimmedPointer = trimPointer(pointer); final String actualPointer = source + "/" + trimmedPointer; @@ -170,15 +176,15 @@ private String normalizePointerStructure(final String pointer) { return pointer.replace('/','.'); } - private String trimPointer(String pointer) { + private String trimPointer(final String pointer) { final String trimmedLeadingSlash = pointer.startsWith("/") ? pointer.substring(1) : pointer; return trimmedLeadingSlash.endsWith("/") ? trimmedLeadingSlash.substring(0, trimmedLeadingSlash.length() - 1) : trimmedLeadingSlash; } private void writeToRoot(final Event event, final Map parsedJson) { - for (Map.Entry entry : parsedJson.entrySet()) { + for (final Map.Entry entry : parsedJson.entrySet()) { if (overwriteIfDestinationExists || !event.containsKey(entry.getKey())) { - event.put(entry.getKey(), entry.getValue()); + event.put(eventKeyFactory.createEventKey(entry.getKey(), EventKeyFactory.EventAction.PUT), entry.getValue()); } } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java index 9a58594da7..9d2677e0be 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java @@ -13,6 +13,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.slf4j.Logger; @@ -32,8 +33,9 @@ public class ParseIonProcessor extends AbstractParseProcessor { @DataPrepperPluginConstructor public ParseIonProcessor(final PluginMetrics pluginMetrics, final ParseIonProcessorConfig parseIonProcessorConfig, - final ExpressionEvaluator expressionEvaluator) { - super(pluginMetrics, parseIonProcessorConfig, expressionEvaluator); + final ExpressionEvaluator expressionEvaluator, + final EventKeyFactory eventKeyFactory) { + super(pluginMetrics, parseIonProcessorConfig, expressionEvaluator, eventKeyFactory); // Convert Timestamps to ISO-8601 Z strings objectMapper.registerModule(new IonTimestampConverterModule()); diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java index dd7b471b33..637cbdea0d 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java @@ -13,6 +13,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.slf4j.Logger; @@ -32,8 +33,9 @@ public class ParseJsonProcessor extends AbstractParseProcessor { @DataPrepperPluginConstructor public ParseJsonProcessor(final PluginMetrics pluginMetrics, final ParseJsonProcessorConfig parseJsonProcessorConfig, - final ExpressionEvaluator expressionEvaluator) { - super(pluginMetrics, parseJsonProcessorConfig, expressionEvaluator); + final ExpressionEvaluator expressionEvaluator, + final EventKeyFactory eventKeyFactory) { + super(pluginMetrics, parseJsonProcessorConfig, expressionEvaluator, eventKeyFactory); } @Override diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java index debacad49c..984a49964a 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java @@ -8,6 +8,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.slf4j.Logger; @@ -27,8 +28,9 @@ public class ParseXmlProcessor extends AbstractParseProcessor { @DataPrepperPluginConstructor public ParseXmlProcessor(final PluginMetrics pluginMetrics, final ParseXmlProcessorConfig parseXmlProcessorConfig, - final ExpressionEvaluator expressionEvaluator) { - super(pluginMetrics, parseXmlProcessorConfig, expressionEvaluator); + final ExpressionEvaluator expressionEvaluator, + final EventKeyFactory eventKeyFactory) { + super(pluginMetrics, parseXmlProcessorConfig, expressionEvaluator, eventKeyFactory); } @Override diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java index c9a8fdf4e5..d7eb14c28a 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java @@ -37,7 +37,7 @@ public void setup() { @Override protected AbstractParseProcessor createObjectUnderTest() { - return new ParseIonProcessor(pluginMetrics, ionProcessorConfig, expressionEvaluator); + return new ParseIonProcessor(pluginMetrics, ionProcessorConfig, expressionEvaluator, testEventKeyFactory); } @Test @@ -58,7 +58,7 @@ void test_when_using_ion_features_then_processorParsesCorrectly() { @Test void test_when_deleteSourceFlagEnabled() { when(processorConfig.isDeleteSourceRequested()).thenReturn(true); - parseJsonProcessor = new ParseIonProcessor(pluginMetrics, ionProcessorConfig, expressionEvaluator); + parseJsonProcessor = createObjectUnderTest(); final String serializedMessage = "{bareKey: 1, symbol: SYMBOL, timestamp: 2023-11-30T21:05:23.383Z, attribute: dollars::100.0 }"; final Event parsedEvent = createAndParseMessageEvent(serializedMessage); diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java index 1416d6cf35..cf71f2251f 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java @@ -5,16 +5,20 @@ package org.opensearch.dataprepper.plugins.processor.parse.json; -import org.opensearch.dataprepper.expression.ExpressionEvaluator; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.record.Record; 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.event.TestEventFactory; +import org.opensearch.dataprepper.event.TestEventKeyFactory; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.opensearch.dataprepper.plugins.processor.parse.CommonParseConfig; @@ -48,6 +52,8 @@ public class ParseJsonProcessorTest { protected ExpressionEvaluator expressionEvaluator; protected AbstractParseProcessor parseJsonProcessor; + private final EventFactory testEventFactory = TestEventFactory.getTestEventFactory(); + protected final EventKeyFactory testEventKeyFactory = TestEventKeyFactory.getTestEventFactory(); @BeforeEach public void setup() { @@ -61,7 +67,7 @@ public void setup() { } protected AbstractParseProcessor createObjectUnderTest() { - return new ParseJsonProcessor(pluginMetrics, jsonProcessorConfig, expressionEvaluator); + return new ParseJsonProcessor(pluginMetrics, jsonProcessorConfig, expressionEvaluator, testEventKeyFactory); } @Test @@ -197,7 +203,7 @@ void test_when_nestedJSONArray_then_parsedIntoArrayAndIndicesAccessible() { @Test void test_when_deleteSourceFlagEnabled() { when(processorConfig.isDeleteSourceRequested()).thenReturn(true); - parseJsonProcessor = new ParseJsonProcessor(pluginMetrics, jsonProcessorConfig, expressionEvaluator); + parseJsonProcessor = createObjectUnderTest(); final String key = "key"; final ArrayList value = new ArrayList<>(List.of("Element0","Element1","Element2")); @@ -434,10 +440,7 @@ private Record createMessageEvent(final String message) { } private Record buildRecordWithEvent(final Map data) { - return new Record<>(JacksonEvent.builder() - .withData(data) - .withEventType("event") - .build()); + return new Record<>(testEventFactory.eventBuilder(EventBuilder.class).withData(data).build()); } private void assertThatKeyEquals(final Event parsedEvent, final String key, final Object value) { diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java index 8d9bc4cde3..5cd9037e5b 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java @@ -5,10 +5,14 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.event.TestEventFactory; +import org.opensearch.dataprepper.event.TestEventKeyFactory; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.event.EventBuilder; +import org.opensearch.dataprepper.model.event.EventFactory; +import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; @@ -37,6 +41,8 @@ public class ParseXmlProcessorTest { private ExpressionEvaluator expressionEvaluator; private AbstractParseProcessor parseXmlProcessor; + private final EventFactory testEventFactory = TestEventFactory.getTestEventFactory(); + private final EventKeyFactory testEventKeyFactory = TestEventKeyFactory.getTestEventFactory(); @BeforeEach public void setup() { @@ -46,7 +52,7 @@ public void setup() { } protected AbstractParseProcessor createObjectUnderTest() { - return new ParseXmlProcessor(pluginMetrics, processorConfig, expressionEvaluator); + return new ParseXmlProcessor(pluginMetrics, processorConfig, expressionEvaluator, testEventKeyFactory); } @Test @@ -104,9 +110,6 @@ private Record createMessageEvent(final String message) { } private Record buildRecordWithEvent(final Map data) { - return new Record<>(JacksonEvent.builder() - .withData(data) - .withEventType("event") - .build()); + return new Record<>(testEventFactory.eventBuilder(EventBuilder.class).withData(data).build()); } } From 9a825909398f1e9064e01c4eb8860d5232f8426a Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Mon, 19 Aug 2024 16:15:05 -0500 Subject: [PATCH 119/159] Fix bug where race condition on ack callback could cause S3 folder partition to not be given up (#4835) Signed-off-by: Taylor Gray --- .../dataprepper/plugins/source/s3/ScanObjectWorker.java | 5 ++--- .../plugins/source/s3/S3ScanObjectWorkerTest.java | 4 ++-- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java index 3cddb1a2e8..471a0efa3d 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java @@ -280,8 +280,6 @@ private void processFolderPartition(final SourcePartition sourceCoordinator.saveProgressStateForPartition(folderPartition.getPartitionKey(), folderPartitionState.get()); processObjectsForFolderPartition(objectsToProcess, folderPartition); - - sourceCoordinator.updatePartitionForAcknowledgmentWait(folderPartition.getPartitionKey(), ACKNOWLEDGEMENT_SET_TIMEOUT); } private List getObjectsForPrefix(final String bucket, final String s3Prefix) { @@ -364,7 +362,8 @@ private void processObjectsForFolderPartition(final List obje objectIndex++; } - // Complete the final acknowledgment set + sourceCoordinator.updatePartitionForAcknowledgmentWait(folderPartition.getPartitionKey(), ACKNOWLEDGEMENT_SET_TIMEOUT); + if (acknowledgementSet != null) { acknowledgementSet.complete(); } diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanObjectWorkerTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanObjectWorkerTest.java index a02fec1af4..fa1645db8d 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanObjectWorkerTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanObjectWorkerTest.java @@ -460,8 +460,8 @@ void processing_with_folder_partition_processes_objects_in_folder_and_deletes_th inOrder.verify(s3ObjectDeleteWorker).buildDeleteObjectRequest(bucket, firstObject.key()); inOrder.verify(acknowledgementSet1).complete(); inOrder.verify(s3ObjectDeleteWorker).buildDeleteObjectRequest(bucket, secondObject.key()); - inOrder.verify(acknowledgementSet2).complete(); inOrder.verify(sourceCoordinator).updatePartitionForAcknowledgmentWait(partitionKey, ACKNOWLEDGEMENT_SET_TIMEOUT); + inOrder.verify(acknowledgementSet2).complete(); final Consumer firstAckCallback = ackCallbacks.get(0); firstAckCallback.accept(true); @@ -532,8 +532,8 @@ void processing_with_folder_partition_processes_objects_in_folder_until_max_obje final InOrder inOrder = inOrder(sourceCoordinator, acknowledgementSet1, s3ObjectDeleteWorker); inOrder.verify(s3ObjectDeleteWorker).buildDeleteObjectRequest(bucket, firstObject.key()); - inOrder.verify(acknowledgementSet1).complete(); inOrder.verify(sourceCoordinator).updatePartitionForAcknowledgmentWait(partitionKey, ACKNOWLEDGEMENT_SET_TIMEOUT); + inOrder.verify(acknowledgementSet1).complete(); final Consumer ackCallback = consumerArgumentCaptor.getValue(); ackCallback.accept(true); From ff2de268f9f111f329552f70c67c969652ab269b Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Mon, 19 Aug 2024 17:12:29 -0500 Subject: [PATCH 120/159] Add handle failed events option to parse json processors (#4844) Signed-off-by: Taylor Gray --- .../event}/HandleFailedEventsOption.java | 25 +++--- .../event/HandleFailedEventsOptionTest.java | 33 +++++++ .../drop/DropEventProcessorConfig.java | 1 + .../drop/DropEventsWhenCondition.java | 8 +- .../drop/DropEventProcessorConfigTest.java | 1 + .../drop/DropEventsProcessorTests.java | 1 + .../DropEventsWhenConditionBuilderTest.java | 1 + .../drop/DropEventsWhenConditionTest.java | 1 + .../drop/HandleFailedEventsOptionTest.java | 20 ----- .../parse-json-processor/build.gradle | 1 + .../parse/AbstractParseProcessor.java | 16 +++- .../processor/parse/CommonParseConfig.java | 7 ++ .../parse/ion/ParseIonProcessor.java | 20 ++++- .../parse/ion/ParseIonProcessorConfig.java | 29 +++++++ .../parse/json/ParseJsonProcessor.java | 18 +++- .../parse/json/ParseJsonProcessorConfig.java | 29 +++++++ .../parse/xml/ParseXmlProcessor.java | 19 +++- .../parse/xml/ParseXmlProcessorConfig.java | 29 +++++++ .../ion/ParseIonProcessorConfigTest.java | 17 ++++ .../parse/ion/ParseIonProcessorTest.java | 17 ++++ .../json/ParseJsonProcessorConfigTest.java | 21 ++++- .../parse/json/ParseJsonProcessorTest.java | 87 +++++++++++++++++++ .../xml/ParseXmlProcessorConfigTest.java | 17 ++++ .../parse/xml/ParseXmlProcessorTest.java | 57 ++++++++++++ 24 files changed, 430 insertions(+), 45 deletions(-) rename {data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop => data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event}/HandleFailedEventsOption.java (61%) create mode 100644 data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/HandleFailedEventsOptionTest.java delete mode 100644 data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/HandleFailedEventsOptionTest.java diff --git a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/HandleFailedEventsOption.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/HandleFailedEventsOption.java similarity index 61% rename from data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/HandleFailedEventsOption.java rename to data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/HandleFailedEventsOption.java index b3f4532e65..6c310eb395 100644 --- a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/HandleFailedEventsOption.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/event/HandleFailedEventsOption.java @@ -3,23 +3,19 @@ * SPDX-License-Identifier: Apache-2.0 */ -package org.opensearch.dataprepper.plugins.processor.drop; +package org.opensearch.dataprepper.model.event; -import org.opensearch.dataprepper.model.event.Event; import com.fasterxml.jackson.annotation.JsonCreator; -import org.slf4j.Logger; import java.util.Arrays; import java.util.Map; import java.util.stream.Collectors; -import static org.opensearch.dataprepper.logging.DataPrepperMarkers.EVENT; - -enum HandleFailedEventsOption { - DROP("drop", true, false), - DROP_SILENTLY("drop_silently", true, true), - SKIP("skip", false, false), - SKIP_SILENTLY("skip_silently", false, true); +public enum HandleFailedEventsOption { + DROP("drop", true, true), + DROP_SILENTLY("drop_silently", true, false), + SKIP("skip", false, true), + SKIP_SILENTLY("skip_silently", false, false); private static final Map OPTIONS_MAP = Arrays.stream(HandleFailedEventsOption.values()) .collect(Collectors.toMap( @@ -37,13 +33,14 @@ enum HandleFailedEventsOption { this.isLogRequired = isLogRequired; } - public boolean isDropEventOption(final Event event, final Throwable cause, final Logger log) { - if (isLogRequired) { - log.warn(EVENT, "An exception occurred while processing when expression for event {}", event, cause); - } + public boolean shouldDropEvent() { return isDropEventOption; } + public boolean shouldLog() { + return isLogRequired; + } + @JsonCreator static HandleFailedEventsOption fromOptionValue(final String option) { return OPTIONS_MAP.get(option.toLowerCase()); diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/HandleFailedEventsOptionTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/HandleFailedEventsOptionTest.java new file mode 100644 index 0000000000..90a319ad24 --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/event/HandleFailedEventsOptionTest.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.model.event; + +import org.hamcrest.CoreMatchers; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; + +class HandleFailedEventsOptionTest { + @ParameterizedTest + @EnumSource(HandleFailedEventsOption.class) + void fromOptionValue(final HandleFailedEventsOption option) { + assertThat(HandleFailedEventsOption.fromOptionValue(option.name()), CoreMatchers.is(option)); + + if (option == HandleFailedEventsOption.SKIP || option == HandleFailedEventsOption.SKIP_SILENTLY) { + assertThat(option.shouldDropEvent(), equalTo(false)); + } else { + assertThat(option.shouldDropEvent(), equalTo(true)); + } + + if (option == HandleFailedEventsOption.SKIP_SILENTLY || option == HandleFailedEventsOption.DROP_SILENTLY) { + assertThat(option.shouldLog(), equalTo(false)); + } else { + assertThat(option.shouldLog(), equalTo(true)); + } + } +} diff --git a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java index 587a482064..ecc2d2d065 100644 --- a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java +++ b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.NotEmpty; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; public class DropEventProcessorConfig { @JsonProperty("drop_when") diff --git a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenCondition.java b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenCondition.java index 7e2887e320..8d74ba6efb 100644 --- a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenCondition.java +++ b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenCondition.java @@ -7,11 +7,14 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Objects; +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.EVENT; + /** * @since 1.3 * @@ -57,7 +60,10 @@ public boolean isStatementFalseWith(final Event event) { try { return !expressionEvaluator.evaluateConditional(dropWhen, event); } catch (final Exception e) { - return handleFailedEventsSetting.isDropEventOption(event, e, LOG); + if (handleFailedEventsSetting.shouldLog()) { + LOG.warn(EVENT, "An exception occurred while processing when expression for event [{}]", event, e); + } + return handleFailedEventsSetting.shouldDropEvent(); } } diff --git a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfigTest.java b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfigTest.java index 5cbdf91aad..84669606a3 100644 --- a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfigTest.java +++ b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfigTest.java @@ -7,6 +7,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.nullValue; diff --git a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessorTests.java b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessorTests.java index 3c5694f632..1af2146139 100644 --- a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessorTests.java +++ b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessorTests.java @@ -7,6 +7,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenConditionBuilderTest.java b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenConditionBuilderTest.java index 860b7042ae..a962e37b39 100644 --- a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenConditionBuilderTest.java +++ b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenConditionBuilderTest.java @@ -10,6 +10,7 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import java.util.UUID; diff --git a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenConditionTest.java b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenConditionTest.java index 98ec1536fc..8b309210db 100644 --- a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenConditionTest.java +++ b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsWhenConditionTest.java @@ -15,6 +15,7 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import java.util.UUID; import java.util.stream.Stream; diff --git a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/HandleFailedEventsOptionTest.java b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/HandleFailedEventsOptionTest.java deleted file mode 100644 index 04b377ad2a..0000000000 --- a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/HandleFailedEventsOptionTest.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Copyright OpenSearch Contributors - * SPDX-License-Identifier: Apache-2.0 - */ - -package org.opensearch.dataprepper.plugins.processor.drop; - -import org.hamcrest.CoreMatchers; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -import static org.hamcrest.MatcherAssert.assertThat; - -class HandleFailedEventsOptionTest { - @ParameterizedTest - @EnumSource(HandleFailedEventsOption.class) - void fromOptionValue(final HandleFailedEventsOption option) { - assertThat(HandleFailedEventsOption.fromOptionValue(option.name()), CoreMatchers.is(option)); - } -} diff --git a/data-prepper-plugins/parse-json-processor/build.gradle b/data-prepper-plugins/parse-json-processor/build.gradle index 5125409731..91275eb799 100644 --- a/data-prepper-plugins/parse-json-processor/build.gradle +++ b/data-prepper-plugins/parse-json-processor/build.gradle @@ -10,6 +10,7 @@ plugins { dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:common') + implementation 'io.micrometer:micrometer-core' implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-ion' implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-xml' diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java index ffb0855590..18acb3dfd8 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java @@ -11,9 +11,11 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.record.Record; +import io.micrometer.core.instrument.Counter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,6 +33,7 @@ public abstract class AbstractParseProcessor extends AbstractProcessor, Record> { private static final Logger LOG = LoggerFactory.getLogger(AbstractParseProcessor.class); + private static final String PROCESSING_FAILURES = "processingFailures"; private final EventKey source; private final EventKey destination; @@ -40,6 +43,10 @@ public abstract class AbstractParseProcessor extends AbstractProcessor> doExecute(final Collection> recor if(deleteSourceRequested) { event.delete(this.source); } - } catch (final Exception e) { - LOG.error(EVENT, "An exception occurred while using the {} processor on Event [{}]", getProcessorName(), record.getData(), e); + } catch (Exception e) { + processingFailuresCounter.increment(); + if (handleFailedEventsOption.shouldLog()) { + LOG.error(EVENT, "An exception occurred while using the {} processor on Event [{}]", getProcessorName(), record.getData(), e); + } } } return records; diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java index 5fd5050b3d..f10537bc7c 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.parse; import java.util.List; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; public interface CommonParseConfig { /** @@ -59,4 +60,10 @@ public interface CommonParseConfig { * Defaults to false. */ boolean isDeleteSourceRequested(); + + /** + * An optional setting used to determine how to handle parsing errors. Default is skip, which includes logging the error + * and passing the failed Event downstream to the next processor. + */ + HandleFailedEventsOption getHandleFailedEventsOption(); } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java index 9d2677e0be..4bfb88ded6 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessor.java @@ -8,12 +8,14 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.dataformat.ion.IonObjectMapper; +import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.slf4j.Logger; @@ -27,9 +29,14 @@ @DataPrepperPlugin(name = "parse_ion", pluginType = Processor.class, pluginConfigurationType = ParseIonProcessorConfig.class) public class ParseIonProcessor extends AbstractParseProcessor { private static final Logger LOG = LoggerFactory.getLogger(ParseIonProcessor.class); + private static final String PARSE_ERRORS = "parseErrors"; private final IonObjectMapper objectMapper = new IonObjectMapper(); + private final Counter parseErrorsCounter; + + private final HandleFailedEventsOption handleFailedEventsOption; + @DataPrepperPluginConstructor public ParseIonProcessor(final PluginMetrics pluginMetrics, final ParseIonProcessorConfig parseIonProcessorConfig, @@ -39,6 +46,9 @@ public ParseIonProcessor(final PluginMetrics pluginMetrics, // Convert Timestamps to ISO-8601 Z strings objectMapper.registerModule(new IonTimestampConverterModule()); + + handleFailedEventsOption = parseIonProcessorConfig.getHandleFailedEventsOption(); + parseErrorsCounter = pluginMetrics.counter(PARSE_ERRORS); } @Override @@ -47,10 +57,16 @@ protected Optional> readValue(String message, Event cont // We need to do a two-step process here, read the value in, then convert away any Ion types like Timestamp return Optional.of(objectMapper.convertValue(objectMapper.readValue(message, new TypeReference<>() {}), new TypeReference<>() {})); } catch (JsonProcessingException e) { - LOG.error(SENSITIVE, "An exception occurred due to invalid Ion while parsing [{}] due to {}", message, e.getMessage()); + if (handleFailedEventsOption.shouldLog()) { + LOG.error(SENSITIVE, "An exception occurred due to invalid Ion while parsing [{}] due to {}", message, e.getMessage()); + } + parseErrorsCounter.increment(); return Optional.empty(); } catch (Exception e) { - LOG.error(SENSITIVE, "An exception occurred while using the parse_ion processor while parsing [{}]", message, e); + if (handleFailedEventsOption.shouldLog()) { + LOG.error(SENSITIVE, "An exception occurred while using the parse_ion processor while parsing [{}]", message, e); + } + processingFailuresCounter.increment(); return Optional.empty(); } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java index fcc2950477..6fad364e17 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java @@ -6,8 +6,11 @@ package org.opensearch.dataprepper.plugins.processor.parse.ion; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotBlank; +import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.plugins.processor.parse.CommonParseConfig; import java.util.List; @@ -38,6 +41,14 @@ public class ParseIonProcessorConfig implements CommonParseConfig { @JsonProperty private boolean deleteSource = false; + @JsonProperty("handle_failed_events") + @JsonPropertyDescription("Determines how to handle events with ION processing errors. Options include 'skip', " + + "which will log the error and send the Event downstream to the next processor, and 'skip_silently', " + + "which will send the Event downstream to the next processor without logging the error. " + + "Default is 'skip'.") + @NotNull + private HandleFailedEventsOption handleFailedEventsOption = HandleFailedEventsOption.SKIP; + @Override public String getSource() { return source; @@ -78,4 +89,22 @@ boolean isValidDestination() { public boolean isDeleteSourceRequested() { return deleteSource; } + + @Override + public HandleFailedEventsOption getHandleFailedEventsOption() { + return handleFailedEventsOption; + } + + @AssertTrue(message = "handled_failed_events must be set to 'skip' or 'skip_silently'.") + boolean isHandleFailedEventsOptionValid() { + if (handleFailedEventsOption == null) { + return true; + } + + if (handleFailedEventsOption.shouldDropEvent()) { + return false; + } + + return true; + } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java index 637cbdea0d..407b59fab1 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessor.java @@ -8,12 +8,14 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.slf4j.Logger; @@ -27,15 +29,21 @@ @DataPrepperPlugin(name = "parse_json", pluginType = Processor.class, pluginConfigurationType = ParseJsonProcessorConfig.class) public class ParseJsonProcessor extends AbstractParseProcessor { private static final Logger LOG = LoggerFactory.getLogger(ParseJsonProcessor.class); + private static final String PARSE_ERRORS = "parseErrors"; private final ObjectMapper objectMapper = new ObjectMapper(); + private final HandleFailedEventsOption handleFailedEventsOption; + private final Counter parseErrorsCounter; + @DataPrepperPluginConstructor public ParseJsonProcessor(final PluginMetrics pluginMetrics, final ParseJsonProcessorConfig parseJsonProcessorConfig, final ExpressionEvaluator expressionEvaluator, final EventKeyFactory eventKeyFactory) { super(pluginMetrics, parseJsonProcessorConfig, expressionEvaluator, eventKeyFactory); + this.handleFailedEventsOption = parseJsonProcessorConfig.getHandleFailedEventsOption(); + parseErrorsCounter = pluginMetrics.counter(PARSE_ERRORS); } @Override @@ -43,10 +51,16 @@ protected Optional> readValue(String message, Event cont try { return Optional.of(objectMapper.readValue(message, new TypeReference<>() {})); } catch (JsonProcessingException e) { - LOG.error(SENSITIVE, "An exception occurred due to invalid JSON while parsing [{}] due to {}", message, e.getMessage()); + if (handleFailedEventsOption.shouldLog()) { + LOG.error(SENSITIVE, "An exception occurred due to invalid JSON while parsing [{}] due to {}", message, e.getMessage()); + } + parseErrorsCounter.increment(); return Optional.empty(); } catch (Exception e) { - LOG.error(SENSITIVE, "An exception occurred while using the parse_json processor while parsing [{}]", message, e); + if (handleFailedEventsOption.shouldLog()) { + LOG.error(SENSITIVE, "An exception occurred while using the parse_json processor while parsing [{}]", message, e); + } + processingFailuresCounter.increment(); return Optional.empty(); } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java index 49ff2a5969..b6a1b14a23 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java @@ -6,8 +6,11 @@ package org.opensearch.dataprepper.plugins.processor.parse.json; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotBlank; +import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.plugins.processor.parse.CommonParseConfig; import java.util.Objects; @@ -38,6 +41,14 @@ public class ParseJsonProcessorConfig implements CommonParseConfig { @JsonProperty private boolean deleteSource = false; + @JsonProperty("handle_failed_events") + @JsonPropertyDescription("Determines how to handle events with JSON processing errors. Options include 'skip', " + + "which will log the error and send the Event downstream to the next processor, and 'skip_silently', " + + "which will send the Event downstream to the next processor without logging the error. " + + "Default is 'skip'.") + @NotNull + private HandleFailedEventsOption handleFailedEventsOption = HandleFailedEventsOption.SKIP; + @Override public String getSource() { return source; @@ -71,6 +82,11 @@ public boolean isDeleteSourceRequested() { return deleteSource; } + @Override + public HandleFailedEventsOption getHandleFailedEventsOption() { + return handleFailedEventsOption; + } + @AssertTrue(message = "destination cannot be empty, whitespace, or a front slash (/)") boolean isValidDestination() { if (Objects.isNull(destination)) return true; @@ -78,4 +94,17 @@ boolean isValidDestination() { final String trimmedDestination = destination.trim(); return !trimmedDestination.isEmpty() && !(trimmedDestination.equals("/")); } + + @AssertTrue(message = "handled_failed_events must be set to 'skip' or 'skip_silently'.") + boolean isHandleFailedEventsOptionValid() { + if (handleFailedEventsOption == null) { + return true; + } + + if (handleFailedEventsOption.shouldDropEvent()) { + return false; + } + + return true; + } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java index 984a49964a..9dced55355 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessor.java @@ -3,12 +3,14 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.dataformat.xml.XmlMapper; +import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.expression.ExpressionEvaluator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.slf4j.Logger; @@ -22,8 +24,12 @@ @DataPrepperPlugin(name = "parse_xml", pluginType =Processor.class, pluginConfigurationType =ParseXmlProcessorConfig.class) public class ParseXmlProcessor extends AbstractParseProcessor { private static final Logger LOG = LoggerFactory.getLogger(ParseXmlProcessor.class); + private static final String PARSE_ERRORS = "parseErrors"; private final XmlMapper xmlMapper = new XmlMapper(); + private final Counter parseErrorsCounter; + + private final HandleFailedEventsOption handleFailedEventsOption; @DataPrepperPluginConstructor public ParseXmlProcessor(final PluginMetrics pluginMetrics, @@ -31,6 +37,9 @@ public ParseXmlProcessor(final PluginMetrics pluginMetrics, final ExpressionEvaluator expressionEvaluator, final EventKeyFactory eventKeyFactory) { super(pluginMetrics, parseXmlProcessorConfig, expressionEvaluator, eventKeyFactory); + + handleFailedEventsOption = parseXmlProcessorConfig.getHandleFailedEventsOption(); + parseErrorsCounter = pluginMetrics.counter(PARSE_ERRORS); } @Override @@ -38,10 +47,16 @@ protected Optional> readValue(final String message, fina try { return Optional.of(xmlMapper.readValue(message, new TypeReference<>() {})); } catch (JsonProcessingException e) { - LOG.error(SENSITIVE, "An exception occurred due to invalid XML while parsing [{}] due to {}", message, e.getMessage()); + if (handleFailedEventsOption.shouldLog()) { + LOG.error(SENSITIVE, "An exception occurred due to invalid XML while parsing [{}] due to {}", message, e.getMessage()); + } + parseErrorsCounter.increment(); return Optional.empty(); } catch (Exception e) { - LOG.error(SENSITIVE, "An exception occurred while using the parse_xml processor while parsing [{}]", message, e); + if (handleFailedEventsOption.shouldLog()) { + LOG.error(SENSITIVE, "An exception occurred while using the parse_xml processor while parsing [{}]", message, e); + } + processingFailuresCounter.increment(); return Optional.empty(); } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java index c90173dc43..f84f2de4b6 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java @@ -1,8 +1,11 @@ package org.opensearch.dataprepper.plugins.processor.parse.xml; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotBlank; +import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.plugins.processor.parse.CommonParseConfig; import java.util.List; @@ -33,6 +36,14 @@ public class ParseXmlProcessorConfig implements CommonParseConfig { @JsonProperty private boolean deleteSource = false; + @JsonProperty("handle_failed_events") + @JsonPropertyDescription("Determines how to handle events with XML processing errors. Options include 'skip', " + + "which will log the error and send the Event downstream to the next processor, and 'skip_silently', " + + "which will send the Event downstream to the next processor without logging the error. " + + "Default is 'skip'.") + @NotNull + private HandleFailedEventsOption handleFailedEventsOption = HandleFailedEventsOption.SKIP; + @Override public String getSource() { return source; @@ -75,4 +86,22 @@ boolean isValidDestination() { public boolean isDeleteSourceRequested() { return deleteSource; } + + @Override + public HandleFailedEventsOption getHandleFailedEventsOption() { + return handleFailedEventsOption; + } + + @AssertTrue(message = "handled_failed_events must be set to 'skip' or 'skip_silently'.") + boolean isHandleFailedEventsOptionValid() { + if (handleFailedEventsOption == null) { + return true; + } + + if (handleFailedEventsOption.shouldDropEvent()) { + return false; + } + + return true; + } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfigTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfigTest.java index 8c47650c05..1768b701bb 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfigTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfigTest.java @@ -7,6 +7,7 @@ import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import java.util.List; @@ -29,6 +30,8 @@ public void test_when_defaultParseIonProcessorConfig_then_returns_default_values assertThat(objectUnderTest.getPointer(), equalTo(null)); assertThat(objectUnderTest.getTagsOnFailure(), equalTo(null)); assertThat(objectUnderTest.getOverwriteIfDestinationExists(), equalTo(true)); + assertThat(objectUnderTest.getHandleFailedEventsOption(), equalTo(HandleFailedEventsOption.SKIP)); + assertThat(objectUnderTest.isHandleFailedEventsOptionValid(), equalTo(true)); } @Nested @@ -61,5 +64,19 @@ void test_when_destinationIsWhiteSpaceOrFrontSlash_then_isValidDestinationFalse( setField(ParseIonProcessorConfig.class, config, "deleteSource", true); assertThat(config.isDeleteSourceRequested(), equalTo(true)); } + + @Test + void isHandleFailedEventsOptionValid_returns_false_with_drop_option() throws NoSuchFieldException, IllegalAccessException { + setField(ParseIonProcessorConfig.class, config, "handleFailedEventsOption", HandleFailedEventsOption.DROP); + + assertThat(config.isHandleFailedEventsOptionValid(), equalTo(false)); + } + + @Test + void isHandleFailedEventsOptionValid_returns_true_with_null_handle_events() throws NoSuchFieldException, IllegalAccessException { + setField(ParseIonProcessorConfig.class, config, "handleFailedEventsOption", null); + + assertThat(config.isHandleFailedEventsOptionValid(), equalTo(true)); + } } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java index d7eb14c28a..8bd63c3eec 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.processor.parse.ion; +import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -16,6 +17,8 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -33,6 +36,12 @@ public void setup() { when(processorConfig.getPointer()).thenReturn(defaultConfig.getPointer()); when(processorConfig.getParseWhen()).thenReturn(null); when(processorConfig.getOverwriteIfDestinationExists()).thenReturn(true); + + when(pluginMetrics.counter("recordsIn")).thenReturn(mock(Counter.class)); + when(pluginMetrics.counter("recordsOut")).thenReturn(mock(Counter.class)); + when(pluginMetrics.counter("processingFailures")).thenReturn(this.processingFailuresCounter); + when(pluginMetrics.counter("parseErrors")).thenReturn(this.parseErrorsCounter); + when(processorConfig.getHandleFailedEventsOption()).thenReturn(handleFailedEventsOption); } @Override @@ -53,6 +62,10 @@ void test_when_using_ion_features_then_processorParsesCorrectly() { assertThat(parsedEvent.get("symbol", String.class), equalTo("SYMBOL")); assertThat(parsedEvent.get("timestamp", String.class), equalTo("2023-11-30T21:05:23.383Z")); assertThat(parsedEvent.get("attribute", Double.class), equalTo(100.0)); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -68,5 +81,9 @@ void test_when_deleteSourceFlagEnabled() { assertThat(parsedEvent.get("symbol", String.class), equalTo("SYMBOL")); assertThat(parsedEvent.get("timestamp", String.class), equalTo("2023-11-30T21:05:23.383Z")); assertThat(parsedEvent.get("attribute", Double.class), equalTo(100.0)); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfigTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfigTest.java index aa138a0e7e..8d27120b36 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfigTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfigTest.java @@ -7,13 +7,14 @@ import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; + +import java.util.List; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; -import java.util.List; - public class ParseJsonProcessorConfigTest { private ParseJsonProcessorConfig createObjectUnderTest() { @@ -30,6 +31,8 @@ public void test_when_defaultParseJsonProcessorConfig_then_returns_default_value assertThat(objectUnderTest.getTagsOnFailure(), equalTo(null)); assertThat(objectUnderTest.getOverwriteIfDestinationExists(), equalTo(true)); assertThat(objectUnderTest.isDeleteSourceRequested(), equalTo(false)); + assertThat(objectUnderTest.getHandleFailedEventsOption(), equalTo(HandleFailedEventsOption.SKIP)); + assertThat(objectUnderTest.isHandleFailedEventsOptionValid(), equalTo(true)); } @Nested @@ -62,5 +65,19 @@ void test_when_destinationIsWhiteSpaceOrFrontSlash_then_isValidDestinationFalse( setField(ParseJsonProcessorConfig.class, config, "deleteSource", true); assertThat(config.isDeleteSourceRequested(), equalTo(true)); } + + @Test + void isHandleFailedEventsOptionValid_returns_false_with_drop_option() throws NoSuchFieldException, IllegalAccessException { + setField(ParseJsonProcessorConfig.class, config, "handleFailedEventsOption", HandleFailedEventsOption.DROP); + + assertThat(config.isHandleFailedEventsOptionValid(), equalTo(false)); + } + + @Test + void isHandleFailedEventsOptionValid_returns_true_with_null_handle_events() throws NoSuchFieldException, IllegalAccessException { + setField(ParseJsonProcessorConfig.class, config, "handleFailedEventsOption", null); + + assertThat(config.isHandleFailedEventsOptionValid(), equalTo(true)); + } } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java index cf71f2251f..9aac54b23f 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.processor.parse.json; +import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -18,6 +19,7 @@ import org.opensearch.dataprepper.model.event.EventBuilder; import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.opensearch.dataprepper.plugins.processor.parse.CommonParseConfig; @@ -34,6 +36,9 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -51,6 +56,15 @@ public class ParseJsonProcessorTest { @Mock protected ExpressionEvaluator expressionEvaluator; + @Mock + protected HandleFailedEventsOption handleFailedEventsOption; + + @Mock + protected Counter processingFailuresCounter; + + @Mock + protected Counter parseErrorsCounter; + protected AbstractParseProcessor parseJsonProcessor; private final EventFactory testEventFactory = TestEventFactory.getTestEventFactory(); protected final EventKeyFactory testEventKeyFactory = TestEventKeyFactory.getTestEventFactory(); @@ -64,6 +78,12 @@ public void setup() { when(processorConfig.getPointer()).thenReturn(defaultConfig.getPointer()); when(processorConfig.getParseWhen()).thenReturn(null); when(processorConfig.getOverwriteIfDestinationExists()).thenReturn(true); + + when(pluginMetrics.counter("recordsIn")).thenReturn(mock(Counter.class)); + when(pluginMetrics.counter("recordsOut")).thenReturn(mock(Counter.class)); + when(pluginMetrics.counter("processingFailures")).thenReturn(processingFailuresCounter); + when(pluginMetrics.counter("parseErrors")).thenReturn(parseErrorsCounter); + when(processorConfig.getHandleFailedEventsOption()).thenReturn(handleFailedEventsOption); } protected AbstractParseProcessor createObjectUnderTest() { @@ -86,6 +106,10 @@ void test_when_differentSourceAndDestination_then_processorParsesCorrectly() { assertThat(parsedEvent.containsKey(destination), equalTo(true)); assertThatFirstMapIsSubsetOfSecondMap(data, parsedEvent.get(destination, Map.class)); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -104,6 +128,10 @@ void test_when_dataFieldEqualToRootField_then_overwritesOriginalFields() { assertThatKeyEquals(parsedEvent, source, "value_that_will_overwrite_source"); assertThatKeyEquals(parsedEvent, "key", "value"); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -119,6 +147,10 @@ void test_when_dataFieldEqualToRootField_then_notOverwritesOriginalFields() { final Event parsedEvent = createAndParseMessageEvent(serializedMessage); assertThatKeyEquals(parsedEvent, source, "{\"root_source\":\"value_that_will_not_be_overwritten\"}"); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -136,6 +168,10 @@ void test_when_dataFieldEqualToDestinationField_then_notOverwritesOriginalFields assertThatKeyEquals(parsedEvent, source, "{\"key\":\"value\"}"); assertThat(parsedEvent.containsKey("key"), equalTo(false)); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -149,6 +185,8 @@ void test_when_valueIsEmpty_then_notParsed() { assertThatKeyEquals(parsedEvent, processorConfig.getSource(), serializedMessage); assertThat(parsedEvent.toMap().size(), equalTo(1)); + + verify(parseErrorsCounter).increment(); } @Test @@ -164,6 +202,10 @@ void test_when_deeplyNestedFieldInRoot_then_canReachDeepestLayer() { assertThatKeyEquals(parsedEvent, DEEPLY_NESTED_KEY_NAME, messageMap.get(DEEPLY_NESTED_KEY_NAME)); final String jsonPointerToValue = constructDeeplyNestedJsonPointer(numberOfLayers); assertThat(parsedEvent.get(jsonPointerToValue, String.class), equalTo("value")); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -183,6 +225,10 @@ void test_when_deeplyNestedFieldInKey_then_canReachDeepestLayer() { final String jsonPointerToValue = destination + constructDeeplyNestedJsonPointer(numberOfLayers); assertThat(parsedEvent.get(jsonPointerToValue, String.class), equalTo("value")); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -198,6 +244,10 @@ void test_when_nestedJSONArray_then_parsedIntoArrayAndIndicesAccessible() { assertThat(parsedEvent.get(key, ArrayList.class), equalTo(value)); final String pointerToFirstElement = key + "/0"; assertThat(parsedEvent.get(pointerToFirstElement, String.class), equalTo(value.get(0))); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -214,6 +264,10 @@ void test_when_deleteSourceFlagEnabled() { assertThat(parsedEvent.get(key, ArrayList.class), equalTo(value)); final String pointerToFirstElement = key + "/0"; assertThat(parsedEvent.get(pointerToFirstElement, String.class), equalTo(value.get(0))); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -232,6 +286,10 @@ void test_when_nestedJSONArrayOfJSON_then_parsedIntoArrayAndIndicesAccessible() final String pointerToInternalValue = key + "/0/key0"; assertThat(parsedEvent.get(pointerToInternalValue, String.class), equalTo("value0")); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -250,6 +308,10 @@ void test_when_nestedJSONArrayOfJSONAndPointer_then_parsedIntoValue() { assertThat(parsedEvent.get("key0", String.class), equalTo("value0")); assertThat(parsedEvent.containsKey("key1"),equalTo(false)); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -263,6 +325,10 @@ void test_when_nestedJSONArrayAndIndexPointer_then_parsedIntoArrayAndIndicesAcce assertThat(parsedEvent.containsKey(processorConfig.getSource()), equalTo(true)); assertThat(parsedEvent.get("key.0", String.class), equalTo(value.get(0))); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -283,6 +349,10 @@ void test_when_pointerKeyAlreadyPresentInEvent_then_usesAbsolutePath() { assertThatKeyEquals(parsedEvent, "s3", data.get("s3")); assertThatKeyEquals(parsedEvent, "log.s3", Collections.singletonMap("data", "sample data")); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -298,6 +368,10 @@ void test_when_nestedDestinationField_then_writesToNestedDestination() { assertThat(parsedEvent.get(location, String.class), equalTo("value")); assertThat(parsedEvent.get(destination, Map.class), equalTo(data)); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -346,10 +420,16 @@ void test_when_condition_skips_processing_when_evaluates_to_false() { assertThat(parsedEvent.toMap(), equalTo(testEvent.getData().toMap())); + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(parseErrorsCounter); + verifyNoInteractions(handleFailedEventsOption); + } @Test void test_tags_when_json_parse_fails() { + when(handleFailedEventsOption.shouldLog()).thenReturn(true); + final String source = "different_source"; final String destination = "destination_key"; when(processorConfig.getSource()).thenReturn(source); @@ -364,10 +444,14 @@ void test_tags_when_json_parse_fails() { final Event parsedEvent = createAndParseMessageEvent(testEvent); assertTrue(parsedEvent.getMetadata().hasTags(testTags)); + + verify(parseErrorsCounter).increment(); } @Test void when_evaluate_conditional_throws_RuntimeException_events_are_not_dropped() { + when(handleFailedEventsOption.shouldLog()).thenReturn(true); + final String source = "different_source"; final String destination = "destination_key"; when(processorConfig.getSource()).thenReturn(source); @@ -383,6 +467,9 @@ void when_evaluate_conditional_throws_RuntimeException_events_are_not_dropped() final Event parsedEvent = createAndParseMessageEvent(testEvent); assertThat(parsedEvent.toMap(), equalTo(testEvent.getData().toMap())); + + verify(processingFailuresCounter).increment(); + verifyNoInteractions(parseErrorsCounter); } private String constructDeeplyNestedJsonPointer(final int numberOfLayers) { diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfigTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfigTest.java index bab6d6e919..a0ef665124 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfigTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfigTest.java @@ -2,6 +2,7 @@ import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import java.util.List; @@ -24,6 +25,8 @@ public void test_when_defaultParseXmlProcessorConfig_then_returns_default_values assertThat(objectUnderTest.getPointer(), equalTo(null)); assertThat(objectUnderTest.getTagsOnFailure(), equalTo(null)); assertThat(objectUnderTest.getOverwriteIfDestinationExists(), equalTo(true)); + assertThat(objectUnderTest.getHandleFailedEventsOption(), equalTo(HandleFailedEventsOption.SKIP)); + assertThat(objectUnderTest.isHandleFailedEventsOptionValid(), equalTo(true)); } @Nested @@ -56,5 +59,19 @@ void test_when_destinationIsWhiteSpaceOrFrontSlash_then_isValidDestinationFalse( setField(ParseXmlProcessorConfig.class, config, "deleteSource", true); assertThat(config.isDeleteSourceRequested(), equalTo(true)); } + + @Test + void isHandleFailedEventsOptionValid_returns_false_with_drop_option() throws NoSuchFieldException, IllegalAccessException { + setField(ParseXmlProcessorConfig.class, config, "handleFailedEventsOption", HandleFailedEventsOption.DROP); + + assertThat(config.isHandleFailedEventsOptionValid(), equalTo(false)); + } + + @Test + void isHandleFailedEventsOptionValid_returns_true_with_null_handle_events() throws NoSuchFieldException, IllegalAccessException { + setField(ParseXmlProcessorConfig.class, config, "handleFailedEventsOption", null); + + assertThat(config.isHandleFailedEventsOptionValid(), equalTo(true)); + } } } diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java index 5cd9037e5b..900a7a7bef 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java @@ -1,5 +1,9 @@ package org.opensearch.dataprepper.plugins.processor.parse.xml; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.dataformat.xml.XmlMapper; +import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -13,8 +17,10 @@ import org.opensearch.dataprepper.model.event.EventBuilder; import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; import java.util.Collections; import java.util.HashMap; @@ -24,6 +30,11 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; import static org.opensearch.dataprepper.plugins.processor.parse.xml.ParseXmlProcessorConfig.DEFAULT_SOURCE; @@ -40,6 +51,15 @@ public class ParseXmlProcessorTest { @Mock private ExpressionEvaluator expressionEvaluator; + @Mock + private Counter processingFailuresCounter; + + @Mock + private Counter parseErrorsCounter; + + @Mock + private HandleFailedEventsOption handleFailedEventsOption; + private AbstractParseProcessor parseXmlProcessor; private final EventFactory testEventFactory = TestEventFactory.getTestEventFactory(); private final EventKeyFactory testEventKeyFactory = TestEventKeyFactory.getTestEventFactory(); @@ -49,6 +69,11 @@ public void setup() { when(processorConfig.getSource()).thenReturn(DEFAULT_SOURCE); when(processorConfig.getParseWhen()).thenReturn(null); when(processorConfig.getOverwriteIfDestinationExists()).thenReturn(true); + when(pluginMetrics.counter("recordsIn")).thenReturn(mock(Counter.class)); + when(pluginMetrics.counter("recordsOut")).thenReturn(mock(Counter.class)); + when(pluginMetrics.counter("processingFailures")).thenReturn(processingFailuresCounter); + when(pluginMetrics.counter("parseErrors")).thenReturn(parseErrorsCounter); + when(processorConfig.getHandleFailedEventsOption()).thenReturn(handleFailedEventsOption); } protected AbstractParseProcessor createObjectUnderTest() { @@ -64,6 +89,9 @@ void test_when_using_xml_features_then_processorParsesCorrectly() { assertThat(parsedEvent.get("name", String.class), equalTo("John Doe")); assertThat(parsedEvent.get("age", String.class), equalTo("30")); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -80,6 +108,9 @@ void test_when_deleteSourceFlagEnabled() { assertThat(parsedEvent.containsKey(processorConfig.getSource()), equalTo(false)); assertThat(parsedEvent.get("name", String.class), equalTo("John Doe")); assertThat(parsedEvent.get("age", String.class), equalTo("30")); + + verifyNoInteractions(processingFailuresCounter); + verifyNoInteractions(handleFailedEventsOption); } @Test @@ -87,6 +118,7 @@ void test_when_using_invalid_xml_tags_correctly() { final String tagOnFailure = UUID.randomUUID().toString(); when(processorConfig.getTagsOnFailure()).thenReturn(List.of(tagOnFailure)); + when(handleFailedEventsOption.shouldLog()).thenReturn(true); parseXmlProcessor = createObjectUnderTest(); @@ -94,6 +126,31 @@ void test_when_using_invalid_xml_tags_correctly() { final Event parsedEvent = createAndParseMessageEvent(serializedMessage); assertThat(parsedEvent.getMetadata().hasTags(List.of(tagOnFailure)), equalTo(true)); + + verify(parseErrorsCounter).increment(); + verifyNoInteractions(processingFailuresCounter); + } + + @Test + void test_when_object_mapper_throws_other_exception_tags_correctly() throws JsonProcessingException, NoSuchFieldException, IllegalAccessException { + + final String tagOnFailure = UUID.randomUUID().toString(); + when(processorConfig.getTagsOnFailure()).thenReturn(List.of(tagOnFailure)); + when(handleFailedEventsOption.shouldLog()).thenReturn(true); + + parseXmlProcessor = createObjectUnderTest(); + + final XmlMapper mockMapper = mock(XmlMapper.class); + when(mockMapper.readValue(anyString(), any(TypeReference.class))).thenThrow(IllegalArgumentException.class); + + ReflectivelySetField.setField(ParseXmlProcessor.class, parseXmlProcessor, "xmlMapper", mockMapper); + + final String serializedMessage = "invalidXml"; + final Event parsedEvent = createAndParseMessageEvent(serializedMessage); + + assertThat(parsedEvent.getMetadata().hasTags(List.of(tagOnFailure)), equalTo(true)); + + verify(processingFailuresCounter).increment(); } private Event createAndParseMessageEvent(final String message) { From 385d43862879485cc387df50cc7cc0dd123c8dcb Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Tue, 20 Aug 2024 13:19:42 -0700 Subject: [PATCH 121/159] Add support for AWS security lake sink as a bucket selector mode in S3 sink (#4846) * dplive1.yaml Signed-off-by: Kondaka * Delete .github/workflows/static.yml Signed-off-by: Kondaka * Add support for AWS security lake sink as a bucket selector mode in S3 sink Signed-off-by: Kondaka * Fixed tests Signed-off-by: Kondaka * Added javadoc for S3BucketSelector Signed-off-by: Kondaka * Added new tests for KeyGenerator Signed-off-by: Kondaka * Added new tests and fixed style errors Signed-off-by: Kondaka * Addressed review comments Signed-off-by: Kondaka * Fixed test build failure Signed-off-by: Kondaka --------- Signed-off-by: Kondaka --- data-prepper-plugins/s3-sink/build.gradle | 1 + .../plugins/sink/s3/S3SinkServiceIT.java | 6 +- .../plugins/sink/s3/KeyGenerator.java | 5 +- .../sink/s3/PredefinedObjectMetadata.java | 17 ++++ .../plugins/sink/s3/S3BucketSelector.java | 29 ++++++ .../dataprepper/plugins/sink/s3/S3Sink.java | 24 ++++- .../plugins/sink/s3/S3SinkConfig.java | 29 +++++- .../plugins/sink/s3/S3SinkService.java | 2 - .../sink/s3/SecurityLakeBucketSelector.java | 80 ++++++++++++++++ .../s3/SecurityLakeBucketSelectorConfig.java | 44 +++++++++ .../sink/s3/accumulator/BufferFactory.java | 4 +- .../sink/s3/accumulator/BufferUtilities.java | 16 ++-- .../s3/accumulator/CodecBufferFactory.java | 5 +- .../accumulator/CompressionBufferFactory.java | 5 +- .../sink/s3/accumulator/InMemoryBuffer.java | 17 +++- .../s3/accumulator/InMemoryBufferFactory.java | 5 +- .../sink/s3/accumulator/LocalFileBuffer.java | 9 +- .../accumulator/LocalFileBufferFactory.java | 3 + .../accumulator/MultipartBufferFactory.java | 4 + .../sink/s3/grouping/S3GroupIdentifier.java | 5 + .../s3/grouping/S3GroupIdentifierFactory.java | 18 +++- .../sink/s3/grouping/S3GroupManager.java | 2 +- .../plugins/sink/s3/KeyGeneratorTest.java | 66 ++++++++++++- .../s3/SecurityLakeBucketSelectorTest.java | 94 +++++++++++++++++++ .../s3/accumulator/BufferUtilitiesTest.java | 10 +- .../CompressionBufferFactoryTest.java | 16 ++-- .../InMemoryBufferFactoryTest.java | 2 +- .../s3/accumulator/InMemoryBufferTest.java | 16 ++-- .../LocalFileBufferFactoryTest.java | 2 +- .../s3/accumulator/LocalFileBufferTest.java | 2 +- .../S3GroupIdentifierFactoryTest.java | 2 +- .../s3/grouping/S3GroupIdentifierTest.java | 8 +- .../sink/s3/grouping/S3GroupManagerTest.java | 13 +-- 33 files changed, 488 insertions(+), 73 deletions(-) create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/PredefinedObjectMetadata.java create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3BucketSelector.java create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelector.java create mode 100644 data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelectorConfig.java create mode 100644 data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelectorTest.java diff --git a/data-prepper-plugins/s3-sink/build.gradle b/data-prepper-plugins/s3-sink/build.gradle index 4ea0a364fd..57198bf274 100644 --- a/data-prepper-plugins/s3-sink/build.gradle +++ b/data-prepper-plugins/s3-sink/build.gradle @@ -16,6 +16,7 @@ dependencies { implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-csv' implementation 'software.amazon.awssdk:s3' implementation 'software.amazon.awssdk:sts' + implementation 'software.amazon.awssdk:securitylake:2.26.18' implementation 'org.jetbrains.kotlin:kotlin-stdlib:1.9.22' implementation project(':data-prepper-plugins:avro-codecs') implementation libs.avro.core 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 b7bbb1b97d..68c5ffb9cd 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 @@ -199,7 +199,7 @@ void verify_flushed_object_count_into_s3_bucket() { void configureNewLineCodec() { codec = new NdjsonOutputCodec(ndjsonOutputConfig); - keyGenerator = new KeyGenerator(s3SinkConfig, StandardExtensionProvider.create(codec, CompressionOption.NONE), expressionEvaluator); + keyGenerator = new KeyGenerator(s3SinkConfig, null, StandardExtensionProvider.create(codec, CompressionOption.NONE), expressionEvaluator); } @Test @@ -272,7 +272,7 @@ void verify_flushed_records_into_s3_bucketNewLine_with_compression() throws IOEx private S3SinkService createObjectUnderTest() { OutputCodecContext codecContext = new OutputCodecContext("Tag", Collections.emptyList(), Collections.emptyList()); - final S3GroupIdentifierFactory groupIdentifierFactory = new S3GroupIdentifierFactory(keyGenerator, expressionEvaluator, s3SinkConfig); + final S3GroupIdentifierFactory groupIdentifierFactory = new S3GroupIdentifierFactory(keyGenerator, expressionEvaluator, s3SinkConfig, null); s3GroupManager = new S3GroupManager(s3SinkConfig, groupIdentifierFactory, bufferFactory, codecFactory, s3AsyncClient, bucketOwnerProvider); return new S3SinkService(s3SinkConfig, codecContext, Duration.ofSeconds(5), pluginMetrics, s3GroupManager); @@ -389,7 +389,7 @@ private void configureParquetCodec() { parquetOutputCodecConfig = new ParquetOutputCodecConfig(); parquetOutputCodecConfig.setSchema(parseSchema().toString()); codec = new ParquetOutputCodec(parquetOutputCodecConfig); - keyGenerator = new KeyGenerator(s3SinkConfig, StandardExtensionProvider.create(codec, CompressionOption.NONE), expressionEvaluator); + keyGenerator = new KeyGenerator(s3SinkConfig, null, StandardExtensionProvider.create(codec, CompressionOption.NONE), expressionEvaluator); } private Collection> getRecordList() { diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java index 7a86687684..7a742d0217 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java @@ -12,14 +12,17 @@ public class KeyGenerator { private final S3SinkConfig s3SinkConfig; + private final S3BucketSelector s3BucketSelector; private final ExtensionProvider extensionProvider; private final ExpressionEvaluator expressionEvaluator; public KeyGenerator(final S3SinkConfig s3SinkConfig, + final S3BucketSelector s3BucketSelector, final ExtensionProvider extensionProvider, final ExpressionEvaluator expressionEvaluator) { this.s3SinkConfig = s3SinkConfig; + this.s3BucketSelector = s3BucketSelector; this.extensionProvider = extensionProvider; this.expressionEvaluator = expressionEvaluator; } @@ -30,7 +33,7 @@ public KeyGenerator(final S3SinkConfig s3SinkConfig, * @return object key path. */ public String generateKeyForEvent(final Event event) { - final String pathPrefix = ObjectKey.buildingPathPrefix(s3SinkConfig, event, expressionEvaluator); + final String pathPrefix = s3BucketSelector != null ? s3BucketSelector.getPathPrefix() : ObjectKey.buildingPathPrefix(s3SinkConfig, event, expressionEvaluator); final String namePattern = ObjectKey.objectFileName(s3SinkConfig, extensionProvider.getExtension(), event, expressionEvaluator); return (!pathPrefix.isEmpty()) ? pathPrefix + namePattern : namePattern; } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/PredefinedObjectMetadata.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/PredefinedObjectMetadata.java new file mode 100644 index 0000000000..3e24c07ddd --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/PredefinedObjectMetadata.java @@ -0,0 +1,17 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import com.fasterxml.jackson.annotation.JsonProperty; +public class PredefinedObjectMetadata { + @JsonProperty("number_of_objects") + private String numberOfObjects; + + public String getNumberOfObjects() { + return numberOfObjects; + } + +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3BucketSelector.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3BucketSelector.java new file mode 100644 index 0000000000..e48314e447 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3BucketSelector.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +public interface S3BucketSelector { + /** + * initialize - initializes the selector + * @param s3SinkConfig - s3 sink configuration + */ + void initialize(S3SinkConfig s3SinkConfig); + + /** + * getBucketName - returns the name of the bucket created by the bucket selector + * + * @return - bucket name + */ + String getBucketName(); + + /** + * getPathPrefix - returns the prefix to be used for the objects created in the bucket + * + * @return path prefix + */ + String getPathPrefix(); +} + 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 4aa2898476..c4c88dc323 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 @@ -94,7 +94,17 @@ public S3Sink(final PluginSetting pluginSetting, bufferFactory = new CompressionBufferFactory(innerBufferFactory, compressionEngine, testCodec); ExtensionProvider extensionProvider = StandardExtensionProvider.create(testCodec, compressionOption); - KeyGenerator keyGenerator = new KeyGenerator(s3SinkConfig, extensionProvider, expressionEvaluator); + String bucketName; + S3BucketSelector s3BucketSelector = null; + if (s3SinkConfig.getBucketSelector() != null) { + s3BucketSelector = loadS3BucketSelector(pluginFactory); + s3BucketSelector.initialize(s3SinkConfig); + bucketName = s3BucketSelector.getBucketName(); + } else { + bucketName = s3SinkConfig.getBucketName(); + } + + KeyGenerator keyGenerator = new KeyGenerator(s3SinkConfig, s3BucketSelector, extensionProvider, expressionEvaluator); if (s3SinkConfig.getObjectKeyOptions().getPathPrefix() != null && !expressionEvaluator.isValidFormatExpression(s3SinkConfig.getObjectKeyOptions().getPathPrefix())) { @@ -106,8 +116,8 @@ public S3Sink(final PluginSetting pluginSetting, throw new InvalidPluginConfigurationException("name_pattern is not a valid format expression"); } - if (s3SinkConfig.getBucketName() != null && - !expressionEvaluator.isValidFormatExpression(s3SinkConfig.getBucketName())) { + if (bucketName != null && + !expressionEvaluator.isValidFormatExpression(bucketName)) { throw new InvalidPluginConfigurationException("bucket name is not a valid format expression"); } @@ -115,13 +125,19 @@ public S3Sink(final PluginSetting pluginSetting, testCodec.validateAgainstCodecContext(s3OutputCodecContext); - final S3GroupIdentifierFactory s3GroupIdentifierFactory = new S3GroupIdentifierFactory(keyGenerator, expressionEvaluator, s3SinkConfig); + final S3GroupIdentifierFactory s3GroupIdentifierFactory = new S3GroupIdentifierFactory(keyGenerator, expressionEvaluator, s3SinkConfig, s3BucketSelector); final S3GroupManager s3GroupManager = new S3GroupManager(s3SinkConfig, s3GroupIdentifierFactory, bufferFactory, codecFactory, s3Client, bucketOwnerProvider); s3SinkService = new S3SinkService(s3SinkConfig, s3OutputCodecContext, RETRY_FLUSH_BACKOFF, pluginMetrics, s3GroupManager); } + private S3BucketSelector loadS3BucketSelector(PluginFactory pluginFactory) { + final PluginModel modeConfiguration = s3SinkConfig.getBucketSelector(); + final PluginSetting modePluginSetting = new PluginSetting(modeConfiguration.getPluginName(), modeConfiguration.getPluginSettings()); + return pluginFactory.loadPlugin(S3BucketSelector.class, modePluginSetting); + } + @Override public boolean isReady() { return sinkInitialized; 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 eb12f0790b..71e523e5f6 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 @@ -6,8 +6,8 @@ package org.opensearch.dataprepper.plugins.sink.s3; import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.AssertTrue; import jakarta.validation.Valid; -import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.Size; import org.opensearch.dataprepper.aws.validator.AwsAccountId; @@ -36,10 +36,21 @@ public class S3SinkConfig { private AwsAuthenticationOptions awsAuthenticationOptions; @JsonProperty("bucket") - @NotEmpty @Size(min = 3, max = 500, message = "bucket length should be at least 3 characters") private String bucketName; + @JsonProperty("bucket_selector") + private PluginModel bucketSelector; + + @JsonProperty("predefined_object_metadata") + private PredefinedObjectMetadata predefinedObjectMetadata; + + @AssertTrue(message = "You may not use both bucket and bucket_selector together in one S3 sink.") + private boolean isValidBucketConfig() { + return (bucketName != null && bucketSelector == null) || + (bucketName == null && bucketSelector != null); + } + /** * The default bucket to send to if using a dynamic bucket name and failures occur * for any reason when sending to a dynamic bucket @@ -127,6 +138,18 @@ public ObjectKeyOptions getObjectKeyOptions() { return objectKeyOptions; } + public PredefinedObjectMetadata getPredefinedObjectMetadata() { + return predefinedObjectMetadata; + } + + /** + * Bucket selector configuration options. + * @return bucketSelector plugin model. + */ + public PluginModel getBucketSelector() { + return bucketSelector; + } + /** * Sink codec configuration Options. * @return codec plugin model. @@ -172,4 +195,4 @@ public Map getBucketOwners() { public String getDefaultBucketOwner() { return defaultBucketOwner; } -} \ No newline at end of file +} 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 c0b7c18db5..571a952f01 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 @@ -52,7 +52,6 @@ public class S3SinkService { private final int maxEvents; private final ByteCount maxBytes; private final Duration maxCollectionDuration; - private final String bucket; private final int maxRetries; private final Counter objectsSucceededCounter; private final Counter objectsFailedCounter; @@ -84,7 +83,6 @@ public S3SinkService(final S3SinkConfig s3SinkConfig, maxBytes = s3SinkConfig.getThresholdOptions().getMaximumSize(); maxCollectionDuration = s3SinkConfig.getThresholdOptions().getEventCollectTimeOut(); - bucket = s3SinkConfig.getBucketName(); maxRetries = s3SinkConfig.getMaxUploadRetries(); objectsSucceededCounter = pluginMetrics.counter(OBJECTS_SUCCEEDED); diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelector.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelector.java new file mode 100644 index 0000000000..3fee07fabb --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelector.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.apache.commons.lang3.RandomStringUtils; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import software.amazon.awssdk.services.securitylake.SecurityLakeClient; +import software.amazon.awssdk.services.securitylake.model.AwsIdentity; +import software.amazon.awssdk.services.securitylake.model.CreateCustomLogSourceRequest; +import software.amazon.awssdk.services.securitylake.model.CustomLogSourceProvider; +import software.amazon.awssdk.services.securitylake.model.CustomLogSourceConfiguration; +import software.amazon.awssdk.services.securitylake.model.CreateCustomLogSourceResponse; +import software.amazon.awssdk.services.securitylake.model.CustomLogSourceCrawlerConfiguration; + +import java.time.LocalDate; +import java.util.List; + +@DataPrepperPlugin(name = "aws_security_lake", pluginType = S3BucketSelector.class, pluginConfigurationType = SecurityLakeBucketSelectorConfig.class) +public class SecurityLakeBucketSelector implements S3BucketSelector { + private static final String EXT_PATH = "/ext/"; + private final SecurityLakeBucketSelectorConfig securityLakeBucketSelectorConfig; + + private S3SinkConfig s3SinkConfig; + + private String pathPrefix; + + private String sourceLocation; + + @DataPrepperPluginConstructor + public SecurityLakeBucketSelector(final SecurityLakeBucketSelectorConfig securityLakeBucketSelectorConfig) { + this.securityLakeBucketSelectorConfig = securityLakeBucketSelectorConfig; + } + + public void initialize(S3SinkConfig s3SinkConfig) { + this.s3SinkConfig = s3SinkConfig; + SecurityLakeClient securityLakeClient = SecurityLakeClient.create(); + String arn = s3SinkConfig.getAwsAuthenticationOptions().getAwsStsRoleArn(); + String principal = arn.split(":")[4]; + String sourceName = securityLakeBucketSelectorConfig.getSourceName() != null ? securityLakeBucketSelectorConfig.getSourceName() : RandomStringUtils.randomAlphabetic(7); + CreateCustomLogSourceResponse response = + securityLakeClient.createCustomLogSource( + CreateCustomLogSourceRequest.builder() + .sourceName(sourceName+RandomStringUtils.randomAlphabetic(4)) + .eventClasses(List.of(securityLakeBucketSelectorConfig.getLogClass())) + .sourceVersion(securityLakeBucketSelectorConfig.getSourceVersion()) + .configuration(CustomLogSourceConfiguration.builder() + .crawlerConfiguration(CustomLogSourceCrawlerConfiguration.builder() + .roleArn(arn) + .build()) + .providerIdentity(AwsIdentity.builder() + .externalId(securityLakeBucketSelectorConfig.getExternalId()) + .principal(principal) + .build()) + .build()) + .build()); + CustomLogSourceProvider provider = response.source().provider(); + this.sourceLocation = provider.location(); + final String region=s3SinkConfig.getAwsAuthenticationOptions().getAwsRegion().toString(); + final String accountId=arn.split(":")[4]; + + final LocalDate now = LocalDate.now(); + final String eventDay = String.format("%d%02d%02d", now.getYear(), now.getMonthValue(), now.getDayOfMonth()); + int locIndex = sourceLocation.indexOf(EXT_PATH); + pathPrefix = String.format("%sregion=%s/accountId=%s/eventDay=%s/",sourceLocation.substring(locIndex+1), region, accountId, eventDay); + } + + public String getPathPrefix() { + return pathPrefix; + } + + @Override + public String getBucketName() { + int locIndex = sourceLocation.indexOf(EXT_PATH); + return sourceLocation.substring(EXT_PATH.length(), locIndex); + } +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelectorConfig.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelectorConfig.java new file mode 100644 index 0000000000..b4ff100020 --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelectorConfig.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class SecurityLakeBucketSelectorConfig { + static final String DEFAULT_SOURCE_VERSION = "1.0"; + + static final String DEFAULT_EXTERNAL_ID = "extid"; + + @JsonProperty("source_name") + private String sourceName; + + @JsonProperty("source_version") + private String sourceVersion = DEFAULT_SOURCE_VERSION; + + @JsonProperty("external_id") + private String externalId = DEFAULT_EXTERNAL_ID; + + @JsonProperty("log_class") + private String logClass; + + + public String getSourceName() { + return sourceName; + } + + public String getSourceVersion() { + return sourceVersion; + } + + public String getExternalId() { + return externalId; + } + + public String getLogClass() { + return logClass; + } + +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferFactory.java index 84ad85fdd8..05a5a2425b 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferFactory.java @@ -8,8 +8,10 @@ import org.opensearch.dataprepper.plugins.sink.s3.ownership.BucketOwnerProvider; import software.amazon.awssdk.services.s3.S3AsyncClient; +import java.util.Map; import java.util.function.Supplier; +import java.util.function.Function; public interface BufferFactory { - Buffer getBuffer(S3AsyncClient s3Client, Supplier bucketSupplier, Supplier keySupplier, String defaultBucket, BucketOwnerProvider bucketOwnerProvider); + Buffer getBuffer(S3AsyncClient s3Client, Supplier bucketSupplier, Supplier keySupplier, String defaultBucket, Function> metadataSupplier, BucketOwnerProvider bucketOwnerProvider); } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferUtilities.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferUtilities.java index 9fd051b3d5..7af1f8e764 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferUtilities.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferUtilities.java @@ -14,6 +14,7 @@ import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.PutObjectResponse; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.function.Consumer; import java.util.function.Function; @@ -32,15 +33,18 @@ static CompletableFuture putObjectOrSendToDefaultBucket(final final String objectKey, final String targetBucket, final String defaultBucket, + final Map objectMetadata, final BucketOwnerProvider bucketOwnerProvider) { final boolean[] defaultBucketAttempted = new boolean[1]; - return s3Client.putObject( - PutObjectRequest.builder() - .bucket(targetBucket) - .key(objectKey) - .expectedBucketOwner(bucketOwnerProvider.getBucketOwner(targetBucket).orElse(null)) - .build(), requestBody) + PutObjectRequest.Builder builder = PutObjectRequest.builder() + .bucket(targetBucket) + .key(objectKey) + .expectedBucketOwner(bucketOwnerProvider.getBucketOwner(targetBucket).orElse(null)); + if (objectMetadata != null) { + builder = builder.metadata(objectMetadata); + } + return s3Client.putObject(builder.build(), requestBody) .handle((result, ex) -> { if (ex != null) { runOnFailure.accept(ex); diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CodecBufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CodecBufferFactory.java index d263926849..8ed5f32d64 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CodecBufferFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/CodecBufferFactory.java @@ -4,7 +4,9 @@ import org.opensearch.dataprepper.plugins.sink.s3.ownership.BucketOwnerProvider; import software.amazon.awssdk.services.s3.S3AsyncClient; +import java.util.Map; import java.util.function.Supplier; +import java.util.function.Function; public class CodecBufferFactory implements BufferFactory { private final BufferFactory innerBufferFactory; @@ -20,8 +22,9 @@ public Buffer getBuffer(final S3AsyncClient s3Client, final Supplier bucketSupplier, final Supplier keySupplier, final String defaultBucket, + final Function> metadataSupplier, final BucketOwnerProvider bucketOwnerProvider) { - Buffer innerBuffer = innerBufferFactory.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider); + Buffer innerBuffer = innerBufferFactory.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, metadataSupplier, bucketOwnerProvider); return new CodecBuffer(innerBuffer, bufferedCodec); } } 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 index f79cdd0779..891c1327a7 100644 --- 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 @@ -11,7 +11,9 @@ import software.amazon.awssdk.services.s3.S3AsyncClient; import java.util.Objects; +import java.util.Map; import java.util.function.Supplier; +import java.util.function.Function; public class CompressionBufferFactory implements BufferFactory { private final BufferFactory innerBufferFactory; @@ -31,8 +33,9 @@ public Buffer getBuffer(final S3AsyncClient s3Client, final Supplier bucketSupplier, final Supplier keySupplier, final String defaultBucket, + final Function> metadataSupplier, final BucketOwnerProvider bucketOwnerProvider) { - final Buffer internalBuffer = innerBufferFactory.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider); + final Buffer internalBuffer = innerBufferFactory.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, metadataSupplier, bucketOwnerProvider); if(compressionInternal) return internalBuffer; 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 5334f42313..624ade0809 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 @@ -13,11 +13,13 @@ import java.io.ByteArrayOutputStream; import java.io.OutputStream; import java.time.Duration; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.function.Supplier; +import java.util.function.Function; /** * A buffer can hold in memory data and flushing it to S3. @@ -29,6 +31,7 @@ public class InMemoryBuffer implements Buffer { private final S3AsyncClient s3Client; private final Supplier bucketSupplier; private final Supplier keySupplier; + private final Function> metadataSupplier; private final BucketOwnerProvider bucketOwnerProvider; private int eventCount; @@ -42,11 +45,13 @@ public class InMemoryBuffer implements Buffer { InMemoryBuffer(final S3AsyncClient s3Client, final Supplier bucketSupplier, final Supplier keySupplier, + final Function> metadataSupplier, final String defaultBucket, final BucketOwnerProvider bucketOwnerProvider) { this.s3Client = s3Client; this.bucketSupplier = bucketSupplier; this.keySupplier = keySupplier; + this.metadataSupplier = metadataSupplier; byteArrayOutputStream.reset(); eventCount = 0; watch = new StopWatch(); @@ -78,7 +83,7 @@ public Optional> flushToS3(final Consumer consumeO final byte[] byteArray = byteArrayOutputStream.toByteArray(); return Optional.ofNullable(BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, AsyncRequestBody.fromBytes(byteArray), consumeOnCompletion, consumeOnException, - getKey(), getBucket(), defaultBucket, bucketOwnerProvider)); + getKey(), getBucket(), defaultBucket, getMetadata(getEventCount()), bucketOwnerProvider)); } private String getBucket() { @@ -87,6 +92,14 @@ private String getBucket() { return bucket; } + private Map getMetadata(int eventCount) { + if (metadataSupplier != null) { + return metadataSupplier.apply(getEventCount()); + } else { + return null; + } + } + @Override public void setEventCount(int eventCount) { this.eventCount = eventCount; @@ -103,4 +116,4 @@ public String getKey() { public OutputStream getOutputStream() { return byteArrayPositionOutputStream; } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactory.java index 8e9cb8c7d9..2606a0a280 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactory.java @@ -8,7 +8,9 @@ import org.opensearch.dataprepper.plugins.sink.s3.ownership.BucketOwnerProvider; import software.amazon.awssdk.services.s3.S3AsyncClient; +import java.util.Map; import java.util.function.Supplier; +import java.util.function.Function; public class InMemoryBufferFactory implements BufferFactory { @Override @@ -16,7 +18,8 @@ public Buffer getBuffer(final S3AsyncClient s3Client, final Supplier bucketSupplier, final Supplier keySupplier, final String defaultBucket, + final Function> metadataSupplier, final BucketOwnerProvider bucketOwnerProvider) { - return new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider); + return new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, metadataSupplier, defaultBucket, bucketOwnerProvider); } } 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 eec6c77996..8fdee31c03 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 @@ -22,6 +22,7 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.time.Duration; +import java.util.Map; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; @@ -98,7 +99,7 @@ public Optional> flushToS3(final Consumer consumeO final CompletableFuture putObjectResponseCompletableFuture = BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, AsyncRequestBody.fromFile(localFile), consumeOnCompletion, consumeOnException, - getKey(), getBucket(), defaultBucket, bucketOwnerProvider) + getKey(), getBucket(), defaultBucket, null, bucketOwnerProvider) .whenComplete(((response, throwable) -> removeTemporaryFile())); return Optional.of(putObjectResponseCompletableFuture); } @@ -138,6 +139,10 @@ public OutputStream getOutputStream() { return outputStream; } + private Map getMetadata() { + return null; + } + private String getBucket() { if(bucket == null) @@ -151,4 +156,4 @@ public String getKey() { key = keySupplier.get(); return key; } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactory.java index b3eb3caf42..1f1176b3c2 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactory.java @@ -12,7 +12,9 @@ import java.io.File; import java.io.IOException; +import java.util.Map; import java.util.function.Supplier; +import java.util.function.Function; public class LocalFileBufferFactory implements BufferFactory { @@ -25,6 +27,7 @@ public Buffer getBuffer(final S3AsyncClient s3Client, final Supplier bucketSupplier, final Supplier keySupplier, final String defaultBucket, + final Function> metadataSupplier, final BucketOwnerProvider bucketOwnerProvider) { File tempFile = null; Buffer localfileBuffer = null; diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBufferFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBufferFactory.java index 55d8cec616..ae10c45bab 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBufferFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/MultipartBufferFactory.java @@ -9,7 +9,10 @@ import org.opensearch.dataprepper.plugins.sink.s3.ownership.BucketOwnerProvider; import software.amazon.awssdk.services.s3.S3AsyncClient; +import java.util.Map; import java.util.function.Supplier; +import java.util.function.Function; + public class MultipartBufferFactory implements BufferFactory { @Override @@ -17,6 +20,7 @@ public Buffer getBuffer(final S3AsyncClient s3Client, final Supplier bucketSupplier, final Supplier keySupplier, final String defaultBucket, + final Function> metadataSupplier, final BucketOwnerProvider bucketOwnerProvider) { return new MultipartBuffer(new S3OutputStream(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider)); } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifier.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifier.java index 170a2426dd..52fa2578fd 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifier.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifier.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.sink.s3.grouping; +import org.opensearch.dataprepper.plugins.sink.s3.PredefinedObjectMetadata; import java.util.Map; import java.util.Objects; @@ -12,13 +13,16 @@ class S3GroupIdentifier { private final Map groupIdentifierHash; private final String groupIdentifierFullObjectKey; + private final PredefinedObjectMetadata predefinedObjectMetadata; private final String fullBucketName; public S3GroupIdentifier(final Map groupIdentifierHash, final String groupIdentifierFullObjectKey, + final PredefinedObjectMetadata predefineObjectMetadata, final String fullBucketName) { this.groupIdentifierHash = groupIdentifierHash; this.groupIdentifierFullObjectKey = groupIdentifierFullObjectKey; + this.predefinedObjectMetadata = predefineObjectMetadata; this.fullBucketName = fullBucketName; } @@ -39,5 +43,6 @@ public int hashCode() { public Map getGroupIdentifierHash() { return groupIdentifierHash; } + public Map getMetadata(int eventCount) { return predefinedObjectMetadata != null ? Map.of(predefinedObjectMetadata.getNumberOfObjects(), Integer.toString(eventCount)) : null; } public String getFullBucketName() { return fullBucketName; } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierFactory.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierFactory.java index 5ec264616a..89315d95a1 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierFactory.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierFactory.java @@ -9,6 +9,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.plugins.sink.s3.KeyGenerator; import org.opensearch.dataprepper.plugins.sink.s3.S3SinkConfig; +import org.opensearch.dataprepper.plugins.sink.s3.S3BucketSelector; import java.util.HashMap; import java.util.List; @@ -25,30 +26,36 @@ public class S3GroupIdentifierFactory { private final ExpressionEvaluator expressionEvaluator; private final S3SinkConfig s3SinkConfig; + private final S3BucketSelector s3BucketSelector; private static final String BUCKET_NAME_REPLACEMENT_FOR_NON_EXISTING_KEYS = ""; public S3GroupIdentifierFactory(final KeyGenerator keyGenerator, final ExpressionEvaluator expressionEvaluator, - final S3SinkConfig s3SinkConfig) { + final S3SinkConfig s3SinkConfig, + final S3BucketSelector s3BucketSelector) { this.keyGenerator = keyGenerator; this.expressionEvaluator = expressionEvaluator; this.s3SinkConfig = s3SinkConfig; + this.s3BucketSelector = s3BucketSelector; dynamicExpressions = expressionEvaluator.extractDynamicExpressionsFromFormatExpression(s3SinkConfig.getObjectKeyOptions().getPathPrefix()); dynamicExpressions.addAll(expressionEvaluator.extractDynamicExpressionsFromFormatExpression(s3SinkConfig.getObjectKeyOptions().getNamePattern())); - dynamicExpressions.addAll(expressionEvaluator.extractDynamicExpressionsFromFormatExpression(s3SinkConfig.getBucketName())); + if (s3BucketSelector == null) + dynamicExpressions.addAll(expressionEvaluator.extractDynamicExpressionsFromFormatExpression(s3SinkConfig.getBucketName())); dynamicEventsKeys = expressionEvaluator.extractDynamicKeysFromFormatExpression(s3SinkConfig.getObjectKeyOptions().getPathPrefix()); dynamicEventsKeys.addAll(expressionEvaluator.extractDynamicKeysFromFormatExpression(s3SinkConfig.getObjectKeyOptions().getNamePattern())); - dynamicEventsKeys.addAll(expressionEvaluator.extractDynamicKeysFromFormatExpression(s3SinkConfig.getBucketName())); + if (s3BucketSelector == null) + dynamicEventsKeys.addAll(expressionEvaluator.extractDynamicKeysFromFormatExpression(s3SinkConfig.getBucketName())); } public S3GroupIdentifier getS3GroupIdentifierForEvent(final Event event) { final String fullObjectKey = keyGenerator.generateKeyForEvent(event); - final String fullBucketName = event.formatString(s3SinkConfig.getBucketName(), expressionEvaluator, BUCKET_NAME_REPLACEMENT_FOR_NON_EXISTING_KEYS); + final String fullBucketName = s3BucketSelector != null ? s3BucketSelector.getBucketName() : + event.formatString(s3SinkConfig.getBucketName(), expressionEvaluator, BUCKET_NAME_REPLACEMENT_FOR_NON_EXISTING_KEYS); final Map groupIdentificationHash = new HashMap<>(); @@ -62,6 +69,7 @@ public S3GroupIdentifier getS3GroupIdentifierForEvent(final Event event) { groupIdentificationHash.put(expression, value); } - return new S3GroupIdentifier(groupIdentificationHash, fullObjectKey, fullBucketName); + + return new S3GroupIdentifier(groupIdentificationHash, fullObjectKey, s3SinkConfig.getPredefinedObjectMetadata(), fullBucketName); } } diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupManager.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupManager.java index beae9ed157..5ac8adc325 100644 --- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupManager.java +++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupManager.java @@ -79,7 +79,7 @@ public S3Group getOrCreateGroupForEvent(final Event event) { if (allGroups.containsKey(s3GroupIdentifier)) { return allGroups.get(s3GroupIdentifier); } else { - final Buffer bufferForNewGroup = bufferFactory.getBuffer(s3Client, s3GroupIdentifier::getFullBucketName, s3GroupIdentifier::getGroupIdentifierFullObjectKey, s3SinkConfig.getDefaultBucket(), bucketOwnerProvider); + final Buffer bufferForNewGroup = bufferFactory.getBuffer(s3Client, s3GroupIdentifier::getFullBucketName, s3GroupIdentifier::getGroupIdentifierFullObjectKey, s3SinkConfig.getDefaultBucket(), s3GroupIdentifier::getMetadata, bucketOwnerProvider); final OutputCodec outputCodec = codecFactory.provideCodec(); final S3Group s3Group = new S3Group(s3GroupIdentifier, bufferForNewGroup, outputCodec); allGroups.put(s3GroupIdentifier, s3Group); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGeneratorTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGeneratorTest.java index 64189cd939..edd8b18953 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGeneratorTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGeneratorTest.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.sink.s3; +import org.apache.commons.lang3.RandomStringUtils; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -41,8 +42,8 @@ void setUp() { } - private KeyGenerator createObjectUnderTest() { - return new KeyGenerator(s3SinkConfig, extensionProvider, expressionEvaluator); + private KeyGenerator createObjectUnderTest(S3BucketSelector s3BucketSelector) { + return new KeyGenerator(s3SinkConfig, s3BucketSelector, extensionProvider, expressionEvaluator); } @Test @@ -51,7 +52,7 @@ void test_generateKey_with_date_prefix() { final String objectName = UUID.randomUUID().toString(); when(extensionProvider.getExtension()).thenReturn(null); - final KeyGenerator objectUnderTest = createObjectUnderTest(); + final KeyGenerator objectUnderTest = createObjectUnderTest(null); final Event event = mock(Event.class); @@ -78,7 +79,7 @@ void generateKey_with_extension() { String pathPrefix = "events/"; final Event event = mock(Event.class); - final KeyGenerator objectUnderTest = createObjectUnderTest(); + final KeyGenerator objectUnderTest = createObjectUnderTest(null); try (final MockedStatic objectKeyMockedStatic = mockStatic(ObjectKey.class)) { objectKeyMockedStatic.when(() -> ObjectKey.buildingPathPrefix(s3SinkConfig, event, expressionEvaluator)) @@ -93,4 +94,59 @@ void generateKey_with_extension() { } } -} \ No newline at end of file + + @Test + void test_generateKey_with_date_prefix_with_bucketSelector() { + String objectKeyPathPrefix = "logdata/"; + final String objectName = UUID.randomUUID().toString(); + when(extensionProvider.getExtension()).thenReturn(null); + + String pathPrefix = RandomStringUtils.randomAlphabetic(5); + S3BucketSelector s3BucketSelector = mock(S3BucketSelector.class); + when(s3BucketSelector.getPathPrefix()).thenReturn(pathPrefix); + final KeyGenerator objectUnderTest = createObjectUnderTest(s3BucketSelector); + + final Event event = mock(Event.class); + + try (final MockedStatic objectKeyMockedStatic = mockStatic(ObjectKey.class)) { + + objectKeyMockedStatic.when(() -> ObjectKey.buildingPathPrefix(s3SinkConfig, event, expressionEvaluator)) + .thenReturn(objectKeyPathPrefix); + objectKeyMockedStatic.when(() -> ObjectKey.objectFileName(s3SinkConfig, null, event, expressionEvaluator)) + .thenReturn(objectName); + + String key = objectUnderTest.generateKeyForEvent(event); + assertNotNull(key); + assertThat(key, true); + assertThat(key.contains(pathPrefix), equalTo(true)); + assertThat(key.contains(objectName), equalTo(true)); + } + } + + @Test + void generateKey_with_extension_with_bucketSelector() { + String extension = UUID.randomUUID().toString(); + final String objectName = UUID.randomUUID().toString(); + when(extensionProvider.getExtension()).thenReturn(extension); + String objectKeyPathPrefix = "events/"; + + final Event event = mock(Event.class); + String pathPrefix = RandomStringUtils.randomAlphabetic(5); + S3BucketSelector s3BucketSelector = mock(S3BucketSelector.class); + when(s3BucketSelector.getPathPrefix()).thenReturn(pathPrefix); + final KeyGenerator objectUnderTest = createObjectUnderTest(s3BucketSelector); + try (final MockedStatic objectKeyMockedStatic = mockStatic(ObjectKey.class)) { + + objectKeyMockedStatic.when(() -> ObjectKey.buildingPathPrefix(s3SinkConfig, event, expressionEvaluator)) + .thenReturn(objectKeyPathPrefix); + objectKeyMockedStatic.when(() -> ObjectKey.objectFileName(s3SinkConfig, extension, event, expressionEvaluator)) + .thenReturn(objectName); + + String key = objectUnderTest.generateKeyForEvent(event); + assertThat(key, notNullValue()); + assertThat(key.contains(pathPrefix), equalTo(true)); + assertThat(key.contains(objectName), equalTo(true)); + } + + } +} diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelectorTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelectorTest.java new file mode 100644 index 0000000000..d250c3481a --- /dev/null +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/SecurityLakeBucketSelectorTest.java @@ -0,0 +1,94 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.s3; + +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockedStatic; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; +import static org.mockito.ArgumentMatchers.any; + + +import software.amazon.awssdk.regions.Region; +import org.opensearch.dataprepper.plugins.sink.s3.configuration.AwsAuthenticationOptions; +import software.amazon.awssdk.services.securitylake.SecurityLakeClient; +import software.amazon.awssdk.services.securitylake.model.CreateCustomLogSourceRequest; +import software.amazon.awssdk.services.securitylake.model.CustomLogSourceProvider; +import software.amazon.awssdk.services.securitylake.model.CustomLogSourceResource; +import software.amazon.awssdk.services.securitylake.model.CreateCustomLogSourceResponse; + +import java.time.LocalDate; +import java.time.format.DateTimeFormatter; + +public class SecurityLakeBucketSelectorTest { + + @Mock + SecurityLakeBucketSelectorConfig securityLakeBucketSelectorConfig; + + @Mock + S3SinkConfig s3SinkConfig; + + @Mock + AwsAuthenticationOptions awsOptions; + + static final String accountId = "123456789123"; + + static final String regionStr = "us-west-2"; + + @BeforeEach + void setup() { + s3SinkConfig = mock(S3SinkConfig.class); + securityLakeBucketSelectorConfig = mock(SecurityLakeBucketSelectorConfig.class); + awsOptions = mock(AwsAuthenticationOptions.class); + when(awsOptions.getAwsRegion()).thenReturn(Region.of(regionStr)); + when(awsOptions.getAwsStsRoleArn()).thenReturn("arn:aws:iam::"+accountId+":role/Admin"); + when(s3SinkConfig.getAwsAuthenticationOptions()).thenReturn(awsOptions); + when(securityLakeBucketSelectorConfig.getSourceName()).thenReturn(RandomStringUtils.randomAlphabetic(5)); + when(securityLakeBucketSelectorConfig.getLogClass()).thenReturn(RandomStringUtils.randomAlphabetic(5)); + when(securityLakeBucketSelectorConfig.getSourceVersion()).thenReturn(RandomStringUtils.randomAlphabetic(5)); + when(securityLakeBucketSelectorConfig.getExternalId()).thenReturn(RandomStringUtils.randomAlphabetic(5)); + } + + private SecurityLakeBucketSelector createObjectUnderTest() { + return new SecurityLakeBucketSelector(securityLakeBucketSelectorConfig); + } + + @Test + public void test_securityLakeBucketSelector() { + final SecurityLakeClient securityLakeClient = mock(SecurityLakeClient.class); + CreateCustomLogSourceRequest createCustomLogSourceRequest = mock(CreateCustomLogSourceRequest.class); + CreateCustomLogSourceResponse createCustomLogSourceResponse = mock(CreateCustomLogSourceResponse.class); + CustomLogSourceResource customLogSourceResource = mock(CustomLogSourceResource.class); + CustomLogSourceProvider customLogSourceProvider = mock(CustomLogSourceProvider.class); + when(createCustomLogSourceResponse.source()).thenReturn(customLogSourceResource); + when(customLogSourceResource.provider()).thenReturn(customLogSourceProvider); + String testLocation = "/aws/bucket1/ext/location1/"; + when(customLogSourceProvider.location()).thenReturn(testLocation); + when(securityLakeClient.createCustomLogSource(any(CreateCustomLogSourceRequest.class))).thenReturn(createCustomLogSourceResponse); + try (final MockedStatic securityLakeClientMockedStatic = mockStatic(SecurityLakeClient.class)) { + securityLakeClientMockedStatic.when(() -> SecurityLakeClient.create()) + .thenReturn(securityLakeClient); + SecurityLakeBucketSelector securityLakeBucketSelector = createObjectUnderTest(); + securityLakeBucketSelector.initialize(s3SinkConfig); + assertThat(securityLakeBucketSelector.getBucketName(), equalTo("bucket1")); + int index = testLocation.indexOf("/ext/"); + LocalDate today = LocalDate.now(); + DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyyMMdd"); + String formattedDate = today.format(formatter); + String expectedPathPrefix = testLocation.substring(index+1)+"region="+regionStr+"/accountId="+accountId+"/eventDay="+formattedDate+"/"; + assertThat(securityLakeBucketSelector.getPathPrefix(), equalTo(expectedPathPrefix)); + } + } +} + + diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferUtilitiesTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferUtilitiesTest.java index 6107e75205..a3ce02e667 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferUtilitiesTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/BufferUtilitiesTest.java @@ -80,7 +80,7 @@ void putObjectOrSendToDefaultBucket_with_no_exception_sends_to_target_bucket() { when(s3Client.putObject(any(PutObjectRequest.class), eq(requestBody))).thenReturn(successfulFuture); - BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, requestBody, mockRunOnCompletion, mockRunOnFailure, objectKey, targetBucket, defaultBucket, bucketOwnerProvider).join(); + BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, requestBody, mockRunOnCompletion, mockRunOnFailure, objectKey, targetBucket, defaultBucket, null, bucketOwnerProvider).join(); final ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(PutObjectRequest.class); verify(s3Client, times(1)).putObject(argumentCaptor.capture(), eq(requestBody)); @@ -101,7 +101,7 @@ void putObjectOrSendToDefaultBucket_with_no_such_bucket_exception_and_null_defau final CompletableFuture failedFuture = CompletableFuture.failedFuture(NoSuchBucketException.builder().build()); when(s3Client.putObject(any(PutObjectRequest.class), eq(requestBody))).thenReturn(failedFuture); - BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, requestBody, mockRunOnCompletion, mockRunOnFailure, objectKey, targetBucket, null, bucketOwnerProvider).join(); + BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, requestBody, mockRunOnCompletion, mockRunOnFailure, objectKey, targetBucket, null, null, bucketOwnerProvider).join(); verify(s3Client, times(1)).putObject(any(PutObjectRequest.class), eq(requestBody)); verify(mockRunOnCompletion).accept(false); @@ -115,7 +115,7 @@ void putObjectOrSendToDefaultBucket_with_S3Exception_that_is_not_access_denied_o when(s3Client.putObject(any(PutObjectRequest.class), eq(requestBody))).thenReturn(failedFuture); BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, requestBody, mockRunOnCompletion, mockRunOnFailure, objectKey, targetBucket, - defaultBucketEnabled ? defaultBucket : null, bucketOwnerProvider); + defaultBucketEnabled ? defaultBucket : null, null, bucketOwnerProvider); verify(s3Client, times(1)).putObject(any(PutObjectRequest.class), eq(requestBody)); verify(mockRunOnCompletion).accept(false); @@ -130,7 +130,7 @@ void putObjectOrSendToDefaultBucket_with_NoSuchBucketException_or_access_denied_ when(s3Client.putObject(any(PutObjectRequest.class), eq(requestBody))).thenReturn(failedFuture).thenReturn(successfulFuture); when(bucketOwnerProvider.getBucketOwner(anyString())).thenReturn(Optional.empty()); - BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, requestBody, mockRunOnCompletion, mockRunOnFailure, objectKey, targetBucket, defaultBucket, bucketOwnerProvider); + BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, requestBody, mockRunOnCompletion, mockRunOnFailure, objectKey, targetBucket, defaultBucket, null, bucketOwnerProvider); final ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(PutObjectRequest.class); verify(s3Client, times(2)).putObject(argumentCaptor.capture(), eq(requestBody)); @@ -168,7 +168,7 @@ void putObject_failing_to_send_to_bucket_and_default_bucket_completes_as_expecte when(bucketOwnerProvider.getBucketOwner(targetBucket)).thenReturn(Optional.of(bucketOwner)); when(bucketOwnerProvider.getBucketOwner(defaultBucket)).thenReturn(Optional.of(defaultBucketOwner)); - BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, requestBody, mockRunOnCompletion, mockRunOnFailure, objectKey, targetBucket, defaultBucket, bucketOwnerProvider); + BufferUtilities.putObjectOrSendToDefaultBucket(s3Client, requestBody, mockRunOnCompletion, mockRunOnFailure, objectKey, targetBucket, defaultBucket, null, bucketOwnerProvider); final ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(PutObjectRequest.class); verify(s3Client, times(2)).putObject(argumentCaptor.capture(), eq(requestBody)); 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 index d9c1384ae7..d07a1e71a5 100644 --- 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 @@ -91,21 +91,21 @@ class WithBuffer { @BeforeEach void setUp() { - when(innerBufferFactory.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider)).thenReturn(innerBuffer); + when(innerBufferFactory.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, null, bucketOwnerProvider)).thenReturn(innerBuffer); } @Test void getBuffer_returns_CompressionBuffer() { - final Buffer buffer = createObjectUnderTest().getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider); + final Buffer buffer = createObjectUnderTest().getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, null, bucketOwnerProvider); assertThat(buffer, instanceOf(CompressionBuffer.class)); } @Test void getBuffer_returns_new_on_each_call() { final CompressionBufferFactory objectUnderTest = createObjectUnderTest(); - final Buffer firstBuffer = objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider); + final Buffer firstBuffer = objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, null, bucketOwnerProvider); - assertThat(objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider), not(equalTo(firstBuffer))); + assertThat(objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, null, bucketOwnerProvider), not(equalTo(firstBuffer))); } @Nested @@ -117,17 +117,17 @@ void setUp() { @Test void getBuffer_returns_innerBuffer_directly() { - final Buffer buffer = createObjectUnderTest().getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider); + final Buffer buffer = createObjectUnderTest().getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, null, bucketOwnerProvider); assertThat(buffer, sameInstance(innerBuffer)); } @Test void getBuffer_calls_on_each_call() { final CompressionBufferFactory objectUnderTest = createObjectUnderTest(); - objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider); - objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider); + objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, null, bucketOwnerProvider); + objectUnderTest.getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, null, bucketOwnerProvider); - verify(innerBufferFactory, times(2)).getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket, bucketOwnerProvider); + verify(innerBufferFactory, times(2)).getBuffer(s3Client, bucketSupplier, keySupplier, defaultBucket,null, bucketOwnerProvider); } } } diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactoryTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactoryTest.java index 33cca9dd99..945717520c 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactoryTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferFactoryTest.java @@ -22,7 +22,7 @@ void test_inMemoryBufferFactory_notNull(){ void test_buffer_notNull(){ InMemoryBufferFactory inMemoryBufferFactory = new InMemoryBufferFactory(); Assertions.assertNotNull(inMemoryBufferFactory); - Buffer buffer = inMemoryBufferFactory.getBuffer(null, null, null, null, null); + Buffer buffer = inMemoryBufferFactory.getBuffer(null, null, null, null, null, null); Assertions.assertNotNull(buffer); assertThat(buffer, instanceOf(Buffer.class)); } diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferTest.java index ea42dafd27..59677f2f59 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/InMemoryBufferTest.java @@ -65,7 +65,7 @@ class InMemoryBufferTest { @Test void test_with_write_event_into_buffer() throws IOException { - inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, bucketOwnerProvider); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, null, bucketOwnerProvider); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { OutputStream outputStream = inMemoryBuffer.getOutputStream(); @@ -93,7 +93,7 @@ void test_with_write_event_into_buffer() throws IOException { */ void getDuration_provides_duration_within_expected_range() throws IOException, InterruptedException { Instant startTime = Instant.now(); - inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, bucketOwnerProvider); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, null, bucketOwnerProvider); Instant endTime = Instant.now(); @@ -122,7 +122,7 @@ void test_flush_to_s3_success() { when(keySupplier.get()).thenReturn(key); when(bucketSupplier.get()).thenReturn(bucket); - inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, bucketOwnerProvider); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, null, bucketOwnerProvider); Assertions.assertNotNull(inMemoryBuffer); final CompletableFuture expectedFuture = mock(CompletableFuture.class); @@ -130,7 +130,7 @@ void test_flush_to_s3_success() { try (final MockedStatic bufferUtilitiesMockedStatic = mockStatic(BufferUtilities.class)) { bufferUtilitiesMockedStatic.when(() -> BufferUtilities.putObjectOrSendToDefaultBucket(eq(s3Client), any(AsyncRequestBody.class), - eq(mockRunOnCompletion), eq(mockRunOnFailure), eq(key), eq(bucket), eq(null), eq(bucketOwnerProvider))) + eq(mockRunOnCompletion), eq(mockRunOnFailure), eq(key), eq(bucket), eq(null), eq(null), eq(bucketOwnerProvider))) .thenReturn(expectedFuture); final Optional> result = inMemoryBuffer.flushToS3(mockRunOnCompletion, mockRunOnFailure); @@ -143,14 +143,14 @@ void test_flush_to_s3_success() { @Test void getOutputStream_is_PositionOutputStream() { - inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, bucketOwnerProvider); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, null, bucketOwnerProvider); assertThat(inMemoryBuffer.getOutputStream(), instanceOf(PositionOutputStream.class)); } @Test void getOutputStream_getPos_equals_written_size() throws IOException { - inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, bucketOwnerProvider); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, null, bucketOwnerProvider); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { OutputStream outputStream = inMemoryBuffer.getOutputStream(); @@ -167,7 +167,7 @@ void getOutputStream_getPos_equals_written_size() throws IOException { @Test void getSize_across_multiple_in_sequence() throws IOException { - inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, bucketOwnerProvider); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, null, bucketOwnerProvider); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { OutputStream outputStream = inMemoryBuffer.getOutputStream(); @@ -177,7 +177,7 @@ void getSize_across_multiple_in_sequence() throws IOException { } assertThat(inMemoryBuffer.getSize(), equalTo((long) MAX_EVENTS * 1000)); - inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, bucketOwnerProvider); + inMemoryBuffer = new InMemoryBuffer(s3Client, bucketSupplier, keySupplier, null, null, bucketOwnerProvider); assertThat(inMemoryBuffer.getSize(), equalTo(0L)); while (inMemoryBuffer.getEventCount() < MAX_EVENTS) { diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactoryTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactoryTest.java index fb45e781aa..f19c42a433 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactoryTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferFactoryTest.java @@ -21,7 +21,7 @@ void test_localFileBufferFactory_notNull() { void test_buffer_notNull() { LocalFileBufferFactory localFileBufferFactory = new LocalFileBufferFactory(); Assertions.assertNotNull(localFileBufferFactory); - Buffer buffer = localFileBufferFactory.getBuffer(null, null, null, null, null); + Buffer buffer = localFileBufferFactory.getBuffer(null, null, null, null, null, null); Assertions.assertNotNull(buffer); assertThat(buffer, instanceOf(LocalFileBuffer.class)); } diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferTest.java index fcca931caa..bb83988e38 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/LocalFileBufferTest.java @@ -123,7 +123,7 @@ void test_with_write_events_into_buffer_and_flush_toS3() throws IOException { try (final MockedStatic bufferUtilitiesMockedStatic = mockStatic(BufferUtilities.class)) { bufferUtilitiesMockedStatic.when(() -> BufferUtilities.putObjectOrSendToDefaultBucket(eq(s3Client), any(AsyncRequestBody.class), - eq(mockRunOnCompletion), eq(mockRunOnFailure), eq(KEY), eq(BUCKET_NAME), eq(defaultBucket), eq(bucketOwnerProvider))) + eq(mockRunOnCompletion), eq(mockRunOnFailure), eq(KEY), eq(BUCKET_NAME), eq(defaultBucket), eq(null), eq(bucketOwnerProvider))) .thenReturn(expectedFuture); final Optional> result = localFileBuffer.flushToS3(mockRunOnCompletion, mockRunOnFailure); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierFactoryTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierFactoryTest.java index e76f8ecaaf..0a8bf72645 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierFactoryTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierFactoryTest.java @@ -48,7 +48,7 @@ void setup() { } private S3GroupIdentifierFactory createObjectUnderTest() { - return new S3GroupIdentifierFactory(keyGenerator, expressionEvaluator, s3SinkConfig); + return new S3GroupIdentifierFactory(keyGenerator, expressionEvaluator, s3SinkConfig, null); } @Test diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierTest.java index b9d22b8c12..5ef5c3a294 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupIdentifierTest.java @@ -24,8 +24,8 @@ void S3GroupIdentifier_with_the_same_identificationHash_and_different_fullObject final String groupTwoFullObjectKey = UUID.randomUUID().toString(); final String fullBucketName = UUID.randomUUID().toString(); - final S3GroupIdentifier s3GroupIdentifier = new S3GroupIdentifier(identificationHash, groupOneFullObjectKey, fullBucketName); - final S3GroupIdentifier seconds3GroupIdentifier = new S3GroupIdentifier(identificationHash, groupTwoFullObjectKey, fullBucketName); + final S3GroupIdentifier s3GroupIdentifier = new S3GroupIdentifier(identificationHash, groupOneFullObjectKey, null, fullBucketName); + final S3GroupIdentifier seconds3GroupIdentifier = new S3GroupIdentifier(identificationHash, groupTwoFullObjectKey, null, fullBucketName); assertThat(s3GroupIdentifier.equals(seconds3GroupIdentifier), equalTo(true)); assertThat(s3GroupIdentifier.hashCode(), equalTo(seconds3GroupIdentifier.hashCode())); @@ -39,8 +39,8 @@ void S3GroupIdentifier_with_different_identificationHash_is_not_considered_equal final String groupTwoFullObjectKey = UUID.randomUUID().toString(); final String fullBucketName = UUID.randomUUID().toString(); - final S3GroupIdentifier s3GroupIdentifier = new S3GroupIdentifier(identificationHashOne, groupOneFullObjectKey, fullBucketName); - final S3GroupIdentifier seconds3GroupIdentifier = new S3GroupIdentifier(identificationHashTwo, groupTwoFullObjectKey, fullBucketName); + final S3GroupIdentifier s3GroupIdentifier = new S3GroupIdentifier(identificationHashOne, groupOneFullObjectKey, null, fullBucketName); + final S3GroupIdentifier seconds3GroupIdentifier = new S3GroupIdentifier(identificationHashTwo, groupTwoFullObjectKey, null, fullBucketName); assertThat(s3GroupIdentifier.equals(seconds3GroupIdentifier), equalTo(false)); assertNotEquals(s3GroupIdentifier.hashCode(), seconds3GroupIdentifier.hashCode()); diff --git a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupManagerTest.java b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupManagerTest.java index 545b6feb77..8449738594 100644 --- a/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupManagerTest.java +++ b/data-prepper-plugins/s3-sink/src/test/java/org/opensearch/dataprepper/plugins/sink/s3/grouping/S3GroupManagerTest.java @@ -21,6 +21,7 @@ import java.util.Collection; import java.util.UUID; import java.util.function.Supplier; +import java.util.function.Function; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; @@ -55,7 +56,7 @@ public class S3GroupManagerTest { private BucketOwnerProvider bucketOwnerProvider; private S3GroupManager createObjectUnderTest() { - return new S3GroupManager(s3SinkConfig, s3GroupIdentifierFactory, bufferFactory, codecFactory, s3Client, bucketOwnerProvider); + return new S3GroupManager(s3SinkConfig, s3GroupIdentifierFactory, bufferFactory, codecFactory, s3Client, bucketOwnerProvider); } @Test @@ -74,7 +75,7 @@ void getOrCreateGroupForEvent_creates_expected_group_when_it_does_not_exist() { when(s3SinkConfig.getDefaultBucket()).thenReturn(defaultBucket); final Buffer buffer = mock(Buffer.class); - when(bufferFactory.getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), eq(bucketOwnerProvider))) + when(bufferFactory.getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), any(Function.class), eq(bucketOwnerProvider))) .thenAnswer(invocation -> { Supplier bucketSupplier = invocation.getArgument(1); Supplier objectKeySupplier = invocation.getArgument(2); @@ -116,7 +117,7 @@ void getOrCreateGroupForEvent_returns_expected_group_when_it_exists() { final Buffer buffer = mock(Buffer.class); final OutputCodec outputCodec = mock(OutputCodec.class); - when(bufferFactory.getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), eq(bucketOwnerProvider))) + when(bufferFactory.getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), any(Function.class), eq(bucketOwnerProvider))) .thenReturn(buffer); when(codecFactory.provideCodec()).thenReturn(outputCodec); @@ -137,7 +138,7 @@ void getOrCreateGroupForEvent_returns_expected_group_when_it_exists() { assertThat(secondResult.getS3GroupIdentifier(), equalTo(s3GroupIdentifier)); assertThat(secondResult.getBuffer(), equalTo(buffer)); - verify(bufferFactory, times(1)).getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), eq(bucketOwnerProvider)); + verify(bufferFactory, times(1)).getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), any(Function.class), eq(bucketOwnerProvider)); final Collection groups = objectUnderTest.getS3GroupEntries(); assertThat(groups, notNullValue()); @@ -177,7 +178,7 @@ void recalculateAndGetGroupSize_returns_expected_size() { final Buffer thirdBuffer = mock(Buffer.class); when(thirdBuffer.getSize()).thenReturn(bufferSizeBase * 3); - when(bufferFactory.getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), eq(bucketOwnerProvider))) + when(bufferFactory.getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), any(Function.class), eq(bucketOwnerProvider))) .thenReturn(buffer).thenReturn(secondBuffer).thenReturn(thirdBuffer); final OutputCodec outputCodec = mock(OutputCodec.class); @@ -223,7 +224,7 @@ void getGroupsOrderedBySize_returns_groups_in_expected_order() { final Buffer thirdBuffer = mock(Buffer.class); when(thirdBuffer.getSize()).thenReturn(bufferSizeBase * 3); - when(bufferFactory.getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), eq(bucketOwnerProvider))) + when(bufferFactory.getBuffer(eq(s3Client), any(Supplier.class), any(Supplier.class), eq(defaultBucket), any(Function.class), eq(bucketOwnerProvider))) .thenReturn(buffer).thenReturn(secondBuffer).thenReturn(thirdBuffer); final OutputCodec firstOutputCodec = mock(OutputCodec.class); From 923145c9ccac5676732dd0f98f770561d953ea3e Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Tue, 20 Aug 2024 13:46:09 -0700 Subject: [PATCH 122/159] Change main branch version to 2.10-SNAPSHOT (#4851) Change main branch version to 2.10-SNAPSHOT Signed-off-by: Kondaka --------- Signed-off-by: Kondaka --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index 1bd03048da..d62bb0cc08 100644 --- a/gradle.properties +++ b/gradle.properties @@ -5,5 +5,5 @@ # ATTENTION: If you are changing the version, please change the DataPrepperVersion whenever the major or minor version changes. # See: https://github.com/opensearch-project/data-prepper/blob/main/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/DataPrepperVersion.java#L9 -version=2.9.0-SNAPSHOT +version=2.10.0-SNAPSHOT org.gradle.jvmargs=-Xmx2048M From 292a225df1739f86e4c6c24e3cb8622c6dd075ed Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Tue, 20 Aug 2024 19:43:10 -0500 Subject: [PATCH 123/159] Catch error that could cause LeaderScheduler thread to crash (#4850) Signed-off-by: Taylor Gray --- .../plugins/source/dynamodb/leader/LeaderScheduler.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/leader/LeaderScheduler.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/leader/LeaderScheduler.java index 90449ece49..005d518388 100644 --- a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/leader/LeaderScheduler.java +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/leader/LeaderScheduler.java @@ -127,7 +127,11 @@ public void run() { if(leaderPartition != null) { // Extend the timeout // will always be a leader until shutdown - coordinator.saveProgressStateForPartition(leaderPartition, Duration.ofMinutes(DEFAULT_EXTEND_LEASE_MINUTES)); + try { + coordinator.saveProgressStateForPartition(leaderPartition, Duration.ofMinutes(DEFAULT_EXTEND_LEASE_MINUTES)); + } catch (final Exception e) { + LOG.error("Failed to update ownership for leader partition. Retrying..."); + } } try { Thread.sleep(leaseInterval.toMillis()); From aa1c5c592131dc43d167111bd7a34a66eb554593 Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Wed, 21 Aug 2024 10:03:27 -0500 Subject: [PATCH 124/159] Address missing processor JsonPropertyDescriptions and validations (#4837) Signed-off-by: Taylor Gray --- .../aggregate/AggregateProcessor.java | 5 ++++ .../aggregate/AggregateProcessorConfig.java | 8 +++--- .../actions/AppendAggregateActionConfig.java | 2 +- .../PercentSamplerAggregateActionConfig.java | 3 ++- .../TailSamplerAggregateActionConfig.java | 2 +- .../aggregate/AggregateProcessorIT.java | 2 ++ .../aggregate/AggregateProcessorTest.java | 14 ++++++++++ .../processor/csv/CsvProcessorConfig.java | 2 +- .../plugins/processor/date/DateProcessor.java | 5 ++++ .../processor/date/DateProcessorConfig.java | 15 ++++++----- .../processor/date/DateProcessorTests.java | 21 +++++++++++++-- .../decompress/DecompressProcessorConfig.java | 6 +++++ .../processor/dissect/DissectProcessor.java | 6 +++++ .../dissect/DissectProcessorConfig.java | 8 +++--- .../dissect/DissectProcessorTest.java | 14 ++++++++++ .../drop/DropEventProcessorConfig.java | 4 +++ .../processor/drop/DropEventsProcessor.java | 6 +++++ .../drop/DropEventsProcessorTests.java | 19 +++++++++++++- .../source/dynamodb/DynamoDBSourceConfig.java | 4 +++ .../dynamodb/configuration/TableConfig.java | 3 +++ .../processor/flatten/FlattenProcessor.java | 6 +++++ .../flatten/FlattenProcessorConfig.java | 2 +- .../flatten/FlattenProcessorTest.java | 14 ++++++++++ .../geoip/processor/GeoIPProcessorIT.java | 2 ++ .../plugins/geoip/processor/EntryConfig.java | 6 +++++ .../geoip/processor/GeoIPProcessor.java | 7 +++++ .../geoip/processor/GeoIPProcessorConfig.java | 8 ++++++ .../geoip/processor/GeoIPProcessorTest.java | 17 +++++++++++- .../plugins/processor/grok/GrokProcessor.java | 6 +++++ .../processor/grok/GrokProcessorTests.java | 13 ++++++++++ .../keyvalue/KeyValueProcessorConfig.java | 2 +- .../mutateevent/AddEntryProcessor.java | 9 +++++++ .../ConvertEntryTypeProcessor.java | 7 +++++ .../ConvertEntryTypeProcessorConfig.java | 8 ++++++ .../mutateevent/CopyValueProcessor.java | 9 +++++++ .../mutateevent/DeleteEntryProcessor.java | 7 +++++ .../mutateevent/ListToMapProcessor.java | 8 ++++++ .../mutateevent/ListToMapProcessorConfig.java | 2 +- .../mutateevent/MapToListProcessor.java | 8 ++++++ .../mutateevent/MapToListProcessorConfig.java | 2 +- .../mutateevent/RenameKeyProcessor.java | 10 +++++++ .../mutateevent/RenameKeyProcessorConfig.java | 5 ++++ .../SelectEntriesProcessorConfig.java | 5 ++++ .../mutateevent/AddEntryProcessorTests.java | 14 ++++++++++ .../ConvertEntryTypeProcessorTests.java | 14 ++++++++++ .../mutateevent/CopyValueProcessorTests.java | 15 +++++++++++ .../DeleteEntryProcessorTests.java | 14 ++++++++++ .../mutateevent/ListToMapProcessorTest.java | 13 ++++++++++ .../mutateevent/MapToListProcessorTest.java | 13 ++++++++++ .../mutateevent/RenameKeyProcessorTests.java | 14 ++++++++++ .../SelectEntriesProcessorTests.java | 10 +++++++ .../mutatestring/SplitStringProcessor.java | 7 +++++ .../SubstituteStringProcessor.java | 7 +++++ .../SubstituteStringProcessorConfig.java | 5 +++- .../SplitStringProcessorTests.java | 13 ++++++++++ .../SubstituteStringProcessorTests.java | 12 +++++++++ .../ObfuscationProcessorConfig.java | 5 ++-- .../parse/AbstractParseProcessor.java | 8 ++++++ .../parse/ion/ParseIonProcessorConfig.java | 7 +++++ .../parse/json/ParseJsonProcessorConfig.java | 7 +++++ .../parse/xml/ParseXmlProcessorConfig.java | 7 +++++ .../parse/ion/ParseIonProcessorTest.java | 17 +++++++++++- .../parse/json/ParseJsonProcessorTest.java | 26 +++++++++++++++---- .../parse/xml/ParseXmlProcessorTest.java | 15 ++++++++++- .../processor/truncate/TruncateProcessor.java | 9 +++++++ .../truncate/TruncateProcessorConfig.java | 1 + .../truncate/TruncateProcessorTests.java | 14 ++++++++++ .../write_json/WriteJsonProcessorConfig.java | 3 +++ 68 files changed, 545 insertions(+), 37 deletions(-) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java index 68cb6f6e65..544928c242 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessor.java @@ -13,6 +13,7 @@ import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.peerforwarder.RequiresPeerForwarding; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; @@ -78,6 +79,10 @@ public AggregateProcessor(final AggregateProcessorConfig aggregateProcessorConfi this.localMode = aggregateProcessorConfig.getLocalMode(); pluginMetrics.gauge(CURRENT_AGGREGATE_GROUPS, aggregateGroupManager, AggregateGroupManager::getAllGroupsSize); + + if (aggregateProcessorConfig.getWhenCondition() != null && (!expressionEvaluator.isValidExpressionStatement(aggregateProcessorConfig.getWhenCondition()))) { + throw new InvalidPluginConfigurationException("aggregate_when {} is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax"); + } } private AggregateAction loadAggregateAction(final PluginFactory pluginFactory) { diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java index 1c2c9fa701..cfb986fe53 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java @@ -5,8 +5,8 @@ package org.opensearch.dataprepper.plugins.processor.aggregate; -import org.opensearch.dataprepper.model.configuration.PluginModel; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import org.opensearch.dataprepper.model.configuration.PluginModel; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; @@ -28,7 +28,7 @@ public class AggregateProcessorConfig { @JsonProperty("group_duration") private Duration groupDuration = Duration.ofSeconds(DEFAULT_GROUP_DURATION_SECONDS); - @JsonPropertyDescription("The action to be performed on each group. One of the available aggregate actions must be provided, or you can create custom aggregate actions. remove_duplicates and put_all are the available actions. For more information, see Creating New Aggregate Actions.") + @JsonPropertyDescription("The action to be performed on each group. One of the available aggregate actions must be provided.") @JsonProperty("action") @NotNull private PluginModel aggregateAction; @@ -46,7 +46,7 @@ public class AggregateProcessorConfig { @JsonProperty("aggregated_events_tag") private String aggregatedEventsTag; - @JsonPropertyDescription("A Data Prepper conditional expression (https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the processor will be run on the event.") + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the processor will be run on the event.") @JsonProperty("aggregate_when") private String whenCondition; @@ -74,7 +74,7 @@ public Boolean getLocalMode() { return localMode; } - @AssertTrue(message="Aggragated Events Tag must be set when output_unaggregated_events is set") + @AssertTrue(message="Aggregated Events Tag must be set when output_unaggregated_events is set") boolean isValidConfig() { return (!outputUnaggregatedEvents || (outputUnaggregatedEvents && aggregatedEventsTag != null)); } diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java index 3d4a9b4a86..529ef0bde3 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/AppendAggregateActionConfig.java @@ -12,8 +12,8 @@ public class AppendAggregateActionConfig { - @JsonPropertyDescription("List of keys to append.") @JsonProperty("keys_to_append") + @JsonPropertyDescription("A list of keys to append to for the aggregated result.") List keysToAppend; public List getKeysToAppend() { diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java index be9770400a..0a17e37c43 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/PercentSamplerAggregateActionConfig.java @@ -6,12 +6,13 @@ package org.opensearch.dataprepper.plugins.processor.aggregate.actions; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; public class PercentSamplerAggregateActionConfig { - @JsonPropertyDescription("Percent value of the sampling to be done. 0.0 < percent < 100.0") + @JsonPropertyDescription("The percentage of events to be processed during a one second interval. Must be greater than 0.0 and less than 100.0") @JsonProperty("percent") @NotNull private double percent; diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java index f86672e3b9..85ce0b1135 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/actions/TailSamplerAggregateActionConfig.java @@ -23,7 +23,7 @@ public class TailSamplerAggregateActionConfig { @NotNull private Integer percent; - @JsonPropertyDescription("A Data Prepper conditional expression (https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the event is an error event or not") + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the event is an error event or not") @JsonProperty("condition") private String condition; diff --git a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java index fc416b0e45..46ec0a996e 100644 --- a/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java +++ b/data-prepper-plugins/aggregate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorIT.java @@ -221,6 +221,7 @@ void aggregateWithPutAllActionAndCondition() throws InterruptedException { String condition = "/firstRandomNumber < 100"; when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); when(aggregateProcessorConfig.getWhenCondition()).thenReturn(condition); + when(expressionEvaluator.isValidExpressionStatement(condition)).thenReturn(true); int count = 0; for (Record record: eventBatch) { Event event = record.getData(); @@ -410,6 +411,7 @@ void aggregateWithCountAggregateActionWithCondition() throws InterruptedExceptio final String condition = "/firstRandomNumber < 100"; when(aggregateProcessorConfig.getGroupDuration()).thenReturn(Duration.ofSeconds(GROUP_DURATION_FOR_ONLY_SINGLE_CONCLUDE)); when(aggregateProcessorConfig.getWhenCondition()).thenReturn(condition); + when(expressionEvaluator.isValidExpressionStatement(condition)).thenReturn(true); int count = 0; eventBatch = getBatchOfEvents(true); for (Record record: eventBatch) { 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 6e60e7d965..a72921cf71 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 @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; import io.micrometer.core.instrument.Counter; @@ -41,6 +42,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; @@ -152,6 +154,16 @@ void setUp() { when(pluginMetrics.timer(MetricNames.TIME_ELAPSED)).thenReturn(timeElapsed); } + @Test + void invalid_aggregate_when_statement_throws_InvalidPluginConfigurationException() { + final String whenCondition = UUID.randomUUID().toString(); + when(aggregateProcessorConfig.getWhenCondition()).thenReturn(whenCondition); + + when(expressionEvaluator.isValidExpressionStatement(whenCondition)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test void getIdentificationKeys_should_return_configured_identification_keys() { final List keys = List.of("key"); @@ -218,6 +230,7 @@ void handleEvent_returning_with_condition_eliminates_one_record() { when(identificationKeysHasher.createIdentificationKeysMapFromEvent(firstEvent)) .thenReturn(identificationKeysMap); when(aggregateActionSynchronizer.handleEventForGroup(firstEvent, identificationKeysMap, aggregateGroup)).thenReturn(firstAggregateActionResponse); + when(expressionEvaluator.isValidExpressionStatement(condition)).thenReturn(true); when(expressionEvaluator.evaluateConditional(condition, event)).thenReturn(true); when(expressionEvaluator.evaluateConditional(condition, firstEvent)).thenReturn(true); when(expressionEvaluator.evaluateConditional(condition, secondEvent)).thenReturn(false); @@ -280,6 +293,7 @@ void handleEvent_returning_with_condition_eliminates_one_record_local_only() { when(identificationKeysHasher.createIdentificationKeysMapFromEvent(firstEvent)) .thenReturn(identificationKeysMap); when(aggregateActionSynchronizer.handleEventForGroup(firstEvent, identificationKeysMap, aggregateGroup)).thenReturn(firstAggregateActionResponse); + when(expressionEvaluator.isValidExpressionStatement(condition)).thenReturn(true); when(expressionEvaluator.evaluateConditional(condition, event)).thenReturn(true); when(expressionEvaluator.evaluateConditional(condition, firstEvent)).thenReturn(true); when(expressionEvaluator.evaluateConditional(condition, secondEvent)).thenReturn(false); diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java index fb803798b2..e21968ebdf 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorConfig.java @@ -57,7 +57,7 @@ public class CsvProcessorConfig { private List columnNames; @JsonProperty("csv_when") - @JsonPropertyDescription("Allows you to specify a [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + @JsonPropertyDescription("Allows you to specify a Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + "such as `/some-key == \"test\"`, that will be evaluated to determine whether " + "the processor should be applied to the event.") private String csvWhen; diff --git a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessor.java b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessor.java index a494cf5334..328f8294bc 100644 --- a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessor.java +++ b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessor.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -63,6 +64,10 @@ public DateProcessor(PluginMetrics pluginMetrics, final DateProcessorConfig date if (dateProcessorConfig.getMatch() != null) extractKeyAndFormatters(); + + if (dateProcessorConfig.getDateWhen() != null && (!expressionEvaluator.isValidExpressionStatement(dateProcessorConfig.getDateWhen()))) { + throw new InvalidPluginConfigurationException("date_when {} is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax"); + } } @Override diff --git a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java index aed3a38674..b62f1b6efd 100644 --- a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java +++ b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java @@ -31,7 +31,7 @@ public static class DateMatch { @JsonProperty("patterns") @JsonPropertyDescription("A list of possible patterns that the timestamp value of the key can have. The patterns " + "are based on a sequence of letters and symbols. The `patterns` support all the patterns listed in the " + - "Java [DatetimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) reference. " + + "Java DateTimeFormatter (https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html) reference. " + "The timestamp value also supports `epoch_second`, `epoch_milli`, and `epoch_nano` values, " + "which represent the timestamp as the number of seconds, milliseconds, and nanoseconds since the epoch. " + "Epoch values always use the UTC time zone.") @@ -54,6 +54,7 @@ public List getPatterns() { } @JsonIgnore + @AssertTrue public boolean isValidPatterns() { // For now, allow only one of the three "epoch_" pattern int count = 0; @@ -119,23 +120,23 @@ public static boolean isValidPattern(final String pattern) { @JsonProperty("source_timezone") @JsonPropertyDescription("The time zone used to parse dates, including when the zone or offset cannot be extracted " + "from the value. If the zone or offset are part of the value, then the time zone is ignored. " + - "A list of all the available time zones is contained in the **TZ database name** column of " + - "[the list of database time zones](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones#List).") + "A list of all the available time zones is contained in the TZ database name column of " + + "(https://en.wikipedia.org/wiki/List_of_tz_database_time_zones#List).") private String sourceTimezone = DEFAULT_SOURCE_TIMEZONE; @JsonProperty("destination_timezone") @JsonPropertyDescription("The time zone used for storing the timestamp in the `destination` field. " + - "A list of all the available time zones is contained in the **TZ database name** column of " + - "[the list of database time zones](https://en.wikipedia.org/wiki/List_of_tz_database_time_zones#List).") + "A list of all the available time zones is contained in the TZ database name column of " + + "(https://en.wikipedia.org/wiki/List_of_tz_database_time_zones#List).") private String destinationTimezone = DEFAULT_DESTINATION_TIMEZONE; @JsonProperty("locale") @JsonPropertyDescription("The location used for parsing dates. Commonly used for parsing month names (`MMM`). " + "The value can contain language, country, or variant fields in IETF BCP 47, such as `en-US`, " + "or a string representation of the " + - "[locale](https://docs.oracle.com/javase/8/docs/api/java/util/Locale.html) object, such as `en_US`. " + + "locale (https://docs.oracle.com/javase/8/docs/api/java/util/Locale.html) object, such as `en_US`. " + "A full list of locale fields, including language, country, and variant, can be found in " + - "[the language subtag registry](https://www.iana.org/assignments/language-subtag-registry/language-subtag-registry). " + + "(https://www.iana.org/assignments/language-subtag-registry/language-subtag-registry). " + "Default is `Locale.ROOT`.") private String locale; diff --git a/data-prepper-plugins/date-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorTests.java b/data-prepper-plugins/date-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorTests.java index c6688d08e3..91299dc4b3 100644 --- a/data-prepper-plugins/date-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorTests.java +++ b/data-prepper-plugins/date-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorTests.java @@ -22,6 +22,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import java.time.Instant; @@ -44,6 +45,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.params.provider.Arguments.arguments; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.lenient; @@ -105,6 +107,17 @@ private DateProcessor createObjectUnderTest() { return new DateProcessor(pluginMetrics, mockDateProcessorConfig, expressionEvaluator); } + @Test + void invalid_date_when_condition_throws_InvalidPluginConfigurationException() { + final String dateWhen = UUID.randomUUID().toString(); + + when(mockDateProcessorConfig.getDateWhen()).thenReturn(dateWhen); + + when(expressionEvaluator.isValidExpressionStatement(dateWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test void from_time_received_with_default_destination_test() { when(mockDateProcessorConfig.getFromTimeReceived()).thenReturn(true); @@ -130,7 +143,9 @@ void from_time_received_with_default_destination_test() { @Test void date_when_does_not_run_date_processor_for_event_with_date_when_as_false() { - when(mockDateProcessorConfig.getDateWhen()).thenReturn(UUID.randomUUID().toString()); + final String dateWhen = UUID.randomUUID().toString(); + when(mockDateProcessorConfig.getDateWhen()).thenReturn(dateWhen); + when(expressionEvaluator.isValidExpressionStatement(dateWhen)).thenReturn(true); dateProcessor = createObjectUnderTest(); Map testData = getTestData(); @@ -526,7 +541,9 @@ void match_without_year_test(String pattern) { @Test void date_processor_catches_exceptions_instead_of_throwing() { - when(mockDateProcessorConfig.getDateWhen()).thenReturn(UUID.randomUUID().toString()); + final String dateWhen = UUID.randomUUID().toString(); + when(mockDateProcessorConfig.getDateWhen()).thenReturn(dateWhen); + when(expressionEvaluator.isValidExpressionStatement(dateWhen)).thenReturn(true); when(expressionEvaluator.evaluateConditional(any(String.class), any(Event.class))) .thenThrow(RuntimeException.class); diff --git a/data-prepper-plugins/decompress-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/decompress/DecompressProcessorConfig.java b/data-prepper-plugins/decompress-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/decompress/DecompressProcessorConfig.java index ce2d985277..32248cdba5 100644 --- a/data-prepper-plugins/decompress-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/decompress/DecompressProcessorConfig.java +++ b/data-prepper-plugins/decompress-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/decompress/DecompressProcessorConfig.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.plugins.processor.decompress.encoding.EncodingType; @@ -16,17 +17,22 @@ public class DecompressProcessorConfig { + @JsonPropertyDescription("The keys in the event that will be decompressed.") @JsonProperty("keys") @NotEmpty + @NotNull private List keys; + @JsonPropertyDescription("The type of decompression to use for the keys in the event. Only gzip is supported.") @JsonProperty("type") @NotNull private DecompressionType decompressionType; + @JsonPropertyDescription("A conditional expression that determines when the decompress processor will run on certain events.") @JsonProperty("decompress_when") private String decompressWhen; + @JsonPropertyDescription("A list of strings with which to tag events when the processor fails to decompress the keys inside an event. Defaults to _decompression_failure.") @JsonProperty("tags_on_failure") private List tagsOnFailure = List.of("_decompression_failure"); diff --git a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessor.java b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessor.java index 5e5fc296bc..cc93a9a4d1 100644 --- a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessor.java +++ b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -49,6 +50,11 @@ public DissectProcessor(PluginMetrics pluginMetrics, final DissectProcessorConfi dissectorMap.put(key, dissector); } + if (dissectConfig.getDissectWhen() != null && + (!expressionEvaluator.isValidExpressionStatement(dissectConfig.getDissectWhen()))) { + throw new InvalidPluginConfigurationException("dissect_when {} is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax"); + } + } @Override diff --git a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java index bc8ef4705a..f934919e81 100644 --- a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java +++ b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java @@ -11,15 +11,15 @@ public class DissectProcessorConfig { @NotNull @JsonProperty("map") @JsonPropertyDescription("Defines the `dissect` patterns for specific keys. For details on how to define fields " + - "in the `dissect` pattern, see [Field notations](#field-notations).") + "in the `dissect` pattern, see (https://opensearch.org/docs/latest/data-prepper/pipelines/configuration/processors/dissect/#field-notations).") private Map map; @JsonProperty("target_types") @JsonPropertyDescription("Specifies the data types for extract fields. Valid options are `integer`, " + - "`double`, `string`, and `boolean`. By default, all fields are of the `string` type.") + "`double`, `string`, `long`, `big_decimal`, and `boolean`. By default, all fields are of the `string` type.") private Map targetTypes; @JsonProperty("dissect_when") - @JsonPropertyDescription("Specifies a condition for performing the `dissect` operation using a " + - "[Data Prepper expression]({{site.url}}{{site.baseurl}}/data-prepper/pipelines/expression-syntax/). " + + @JsonPropertyDescription("Specifies a condition for performing the `dissect` operation using a Data Prepper [conditional expression]" + + "(https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/). " + "If specified, the `dissect` operation will only run when the expression evaluates to true.") private String dissectWhen; diff --git a/data-prepper-plugins/dissect-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorTest.java b/data-prepper-plugins/dissect-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorTest.java index b344bda68e..9ab6dec585 100644 --- a/data-prepper-plugins/dissect-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorTest.java +++ b/data-prepper-plugins/dissect-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorTest.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.processor.dissect.Fields.AppendField; import org.opensearch.dataprepper.plugins.processor.dissect.Fields.Field; @@ -25,9 +26,11 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.UUID; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.when; @@ -49,6 +52,17 @@ void setUp() { when(dissectConfig.getMap()).thenReturn(Map.of()); } + @Test + void invalid_dissect_when_condition_throws_InvalidPluginConfigurationException() { + final String dissectWhen = UUID.randomUUID().toString(); + + when(dissectConfig.getDissectWhen()).thenReturn(dissectWhen); + + when(expressionEvaluator.isValidExpressionStatement(dissectWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test void test_normal_fields_dissect_succeeded() throws NoSuchFieldException, IllegalAccessException { diff --git a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java index ecc2d2d065..b1383fbdf0 100644 --- a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java +++ b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java @@ -6,14 +6,18 @@ package org.opensearch.dataprepper.plugins.processor.drop; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; public class DropEventProcessorConfig { + + @JsonPropertyDescription("Accepts a Data Prepper conditional expression string following the [Data Prepper Expression Syntax](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/). Configuring drop_events with drop_when: true drops all the events received.") @JsonProperty("drop_when") @NotEmpty private String dropWhen; + @JsonPropertyDescription("Specifies how exceptions are handled when an exception occurs while evaluating an event. Default value is 'drop', which drops the event so that it is not sent to OpenSearch. Available options are 'drop', 'drop_silently', 'skip', and 'skip_silently'.") @JsonProperty("handle_failed_events") private HandleFailedEventsOption handleFailedEventsOption = HandleFailedEventsOption.SKIP; diff --git a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessor.java b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessor.java index ccfec9a3ab..6196894beb 100644 --- a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessor.java +++ b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.annotations.SingleThread; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -33,6 +34,11 @@ public DropEventsProcessor( ) { super(pluginMetrics); + if (dropEventProcessorConfig.getDropWhen() != null && + (!expressionEvaluator.isValidExpressionStatement(dropEventProcessorConfig.getDropWhen()))) { + throw new InvalidPluginConfigurationException("drop_when {} is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax"); + } + whenCondition = new DropEventsWhenCondition.Builder() .withDropEventsProcessorConfig(dropEventProcessorConfig) .withExpressionEvaluator(expressionEvaluator) diff --git a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessorTests.java b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessorTests.java index 1af2146139..65cc35f80f 100644 --- a/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessorTests.java +++ b/data-prepper-plugins/drop-events-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventsProcessorTests.java @@ -9,6 +9,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -28,11 +29,15 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.lenient; 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) public class DropEventsProcessorTests { @@ -49,11 +54,23 @@ public class DropEventsProcessorTests { @BeforeEach void beforeEach() { whenSetting = UUID.randomUUID().toString(); - doReturn(HandleFailedEventsOption.SKIP) + when(expressionEvaluator.isValidExpressionStatement(anyString())).thenReturn(true); + lenient().doReturn(HandleFailedEventsOption.SKIP) .when(dropEventProcessorConfig) .getHandleFailedEventsOption(); } + @Test + void invalid_drop_when_throws_InvalidPluginConfigurationException() { + + final String dropWhen = UUID.randomUUID().toString(); + + when(dropEventProcessorConfig.getDropWhen()).thenReturn(dropWhen); + when(expressionEvaluator.isValidExpressionStatement(dropWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, () -> new DropEventsProcessor(pluginMetrics, dropEventProcessorConfig, expressionEvaluator)); + } + @Test void testSingleMessageToDropProcessor() { doReturn("true") diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfig.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfig.java index 5babe81a29..dae4931d6e 100644 --- a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfig.java +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfig.java @@ -8,6 +8,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; +import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.AwsAuthenticationConfig; import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.TableConfig; @@ -22,6 +23,9 @@ public class DynamoDBSourceConfig { @JsonProperty("tables") + @NotEmpty + @NotNull + @Valid private List tableConfigs = Collections.emptyList(); @JsonProperty("aws") diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/TableConfig.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/TableConfig.java index d7c90ea1dd..d0e2b2c25d 100644 --- a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/TableConfig.java +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/TableConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.source.dynamodb.configuration; import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -17,9 +18,11 @@ public class TableConfig { private String tableArn; @JsonProperty("export") + @Valid private ExportConfig exportConfig; @JsonProperty(value = "stream") + @Valid private StreamConfig streamConfig; diff --git a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessor.java b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessor.java index 9e3218be88..0e22a6db10 100644 --- a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessor.java +++ b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessor.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -40,6 +41,11 @@ public FlattenProcessor(final PluginMetrics pluginMetrics, final FlattenProcesso for (final String key : config.getExcludeKeys()) { excludeKeysAndJsonPointers.put(key, getJsonPointer(config.getSource(), key)); } + + if (config.getFlattenWhen() != null && + (!expressionEvaluator.isValidExpressionStatement(config.getFlattenWhen()))) { + throw new InvalidPluginConfigurationException("flatten_when {} is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax"); + } } @Override diff --git a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java index 783f73a9da..648936f2bf 100644 --- a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java +++ b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java @@ -50,7 +50,7 @@ public class FlattenProcessorConfig { private List excludeKeys = DEFAULT_EXCLUDE_KEYS; @JsonProperty("flatten_when") - @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + "such as `/some-key == \"test\"'`, that determines whether the `flatten` processor will be run on the " + "event. Default is `null`, which means that all events will be processed unless otherwise stated.") private String flattenWhen; diff --git a/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorTest.java b/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorTest.java index df693f7f6f..d5c0a4ad2a 100644 --- a/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorTest.java +++ b/data-prepper-plugins/flatten-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorTest.java @@ -17,6 +17,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import java.util.ArrayList; @@ -29,6 +30,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @@ -58,6 +60,17 @@ void setUp() { lenient().when(mockConfig.getExcludeKeys()).thenReturn(new ArrayList<>()); } + @Test + void invalid_flatten_when_expression_throws_InvalidPluginConfigurationException() { + final String flattenWhen = UUID.randomUUID().toString(); + + when(mockConfig.getFlattenWhen()).thenReturn(flattenWhen); + + when(expressionEvaluator.isValidExpressionStatement(flattenWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test void testFlattenEntireEventData() { final FlattenProcessor processor = createObjectUnderTest(); @@ -252,6 +265,7 @@ void testFlattenWithSpecificFieldsAsSourceAndTargetAndRemoveListIndicesAndRemove public void testEventNotProcessedWhenTheWhenConditionIsFalse() { final String whenCondition = UUID.randomUUID().toString(); when(mockConfig.getFlattenWhen()).thenReturn(whenCondition); + when(expressionEvaluator.isValidExpressionStatement(whenCondition)).thenReturn(true); final FlattenProcessor processor = createObjectUnderTest(); final Record testRecord = createTestRecord(createTestData()); diff --git a/data-prepper-plugins/geoip-processor/src/integrationTest/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorIT.java b/data-prepper-plugins/geoip-processor/src/integrationTest/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorIT.java index 57de2a3185..d222dd4828 100644 --- a/data-prepper-plugins/geoip-processor/src/integrationTest/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorIT.java +++ b/data-prepper-plugins/geoip-processor/src/integrationTest/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorIT.java @@ -110,6 +110,8 @@ public void setUp() { lenient().when(pluginMetrics.counter(GEO_IP_EVENTS_FAILED)).thenReturn(geoIpEventsFailed); lenient().when(pluginMetrics.counter(GEO_IP_EVENTS_FAILED_ENGINE_EXCEPTION)).thenReturn(geoIpEventsFailedEngineException); lenient().when(pluginMetrics.counter(GEO_IP_EVENTS_FAILED_IP_NOT_FOUND)).thenReturn(geoIpEventsFailedIPNotFound); + + when(expressionEvaluator.isValidExpressionStatement("/peer/status == \"success\"")).thenReturn(true); } public GeoIPProcessor createObjectUnderTest() { diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/EntryConfig.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/EntryConfig.java index d1bfe17dbd..b425ce3bbb 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/EntryConfig.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/EntryConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.geoip.processor; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; import org.opensearch.dataprepper.plugins.geoip.GeoIPField; @@ -16,16 +17,21 @@ public class EntryConfig { static final String DEFAULT_TARGET = "geo"; + + @JsonPropertyDescription("The key of the source field containing the IP address to geolocate.") @JsonProperty("source") @NotEmpty private String source; + @JsonPropertyDescription("The key of the target field in which to save the geolocation data. Default is geo.") @JsonProperty("target") private String target = DEFAULT_TARGET; + @JsonPropertyDescription("The list of geolocation fields to include in the target object. By default, this is all the fields provided by the configured databases.") @JsonProperty("include_fields") private List includeFields; + @JsonPropertyDescription("The list of geolocation fields to exclude from the target object.") @JsonProperty("exclude_fields") private List excludeFields; diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessor.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessor.java index 933cec7c89..a264d7f66a 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessor.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessor.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -73,6 +74,12 @@ public GeoIPProcessor(final PluginMetrics pluginMetrics, final GeoIpConfigSupplier geoIpConfigSupplier, final ExpressionEvaluator expressionEvaluator) { super(pluginMetrics); + + if (geoIPProcessorConfig.getWhenCondition() != null && + (!expressionEvaluator.isValidExpressionStatement(geoIPProcessorConfig.getWhenCondition()))) { + throw new InvalidPluginConfigurationException("geoip_when {} is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax"); + } + this.geoIPProcessorService = geoIpConfigSupplier.getGeoIPProcessorService().orElseThrow(() -> new IllegalStateException("geoip_service configuration is required when using geoip processor.")); this.geoIPProcessorConfig = geoIPProcessorConfig; diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorConfig.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorConfig.java index 8d9c56c899..23b351390a 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorConfig.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.geoip.processor; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.Size; @@ -21,18 +22,25 @@ public class GeoIPProcessorConfig { @NotNull @Size(min = 1) @JsonProperty("entries") + @JsonPropertyDescription("The list of entries marked for enrichment.") private List entries; @JsonProperty("tags_on_engine_failure") + @JsonPropertyDescription("The tags to add to the event metadata if the geoip processor is unable to enrich an event due to an engine failure.") private List tagsOnEngineFailure; @JsonProperty("tags_on_ip_not_found") + @JsonPropertyDescription("The tags to add to the event metadata if the geoip processor is unable to find a location for the IP address.") private List tagsOnIPNotFound; @JsonProperty("tags_on_no_valid_ip") + @JsonPropertyDescription("The tags to add to the event metadata if the source field is not a valid IP address. This includes the localhost IP address.") private List tagsOnNoValidIp; @JsonProperty("geoip_when") + @JsonPropertyDescription("Specifies a condition for including Events in the `geoip` processor using a Data Prepper [conditional expression]" + + "(https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/)." + + " If specified, the `geoip` processor will only run when the expression evaluates to true.") private String whenCondition; /** diff --git a/data-prepper-plugins/geoip-processor/src/test/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorTest.java b/data-prepper-plugins/geoip-processor/src/test/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorTest.java index 0508f05b6c..7d28c46e0d 100644 --- a/data-prepper-plugins/geoip-processor/src/test/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorTest.java +++ b/data-prepper-plugins/geoip-processor/src/test/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorTest.java @@ -20,6 +20,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.geoip.GeoIPField; import org.opensearch.dataprepper.plugins.geoip.exception.EngineFailureException; @@ -43,6 +44,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doThrow; @@ -91,7 +93,7 @@ class GeoIPProcessorTest { @BeforeEach void setUp() { - when(geoIpConfigSupplier.getGeoIPProcessorService()).thenReturn(Optional.of(geoIPProcessorService)); + lenient().when(geoIpConfigSupplier.getGeoIPProcessorService()).thenReturn(Optional.of(geoIPProcessorService)); lenient().when(geoIPProcessorService.getGeoIPDatabaseReader()).thenReturn(geoIPDatabaseReader); lenient().when(pluginMetrics.counter(GEO_IP_EVENTS_PROCESSED)).thenReturn(geoIpEventsProcessed); lenient().when(pluginMetrics.counter(GEO_IP_EVENTS_SUCCEEDED)).thenReturn(geoIpEventsSucceeded); @@ -113,12 +115,24 @@ private GeoIPProcessor createObjectUnderTest() { return new GeoIPProcessor(pluginMetrics, geoIPProcessorConfig, geoIpConfigSupplier, expressionEvaluator); } + @Test + void invalid_geoip_when_condition_throws_InvalidPluginConfigurationException() { + final String geoipWhen = UUID.randomUUID().toString(); + + when(geoIPProcessorConfig.getWhenCondition()).thenReturn(geoipWhen); + + when(expressionEvaluator.isValidExpressionStatement(geoipWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test void doExecuteTest_with_when_condition_should_enrich_events_that_match_when_condition() { final String whenCondition = "/peer/status == success"; when(geoIPProcessorConfig.getEntries()).thenReturn(List.of(entry)); when(geoIPProcessorConfig.getWhenCondition()).thenReturn(whenCondition); + when(expressionEvaluator.isValidExpressionStatement(whenCondition)).thenReturn(true); when(entry.getSource()).thenReturn("/peer/ip"); when(entry.getTarget()).thenReturn(TARGET); when(entry.getGeoIPFields()).thenReturn(setFields()); @@ -150,6 +164,7 @@ void doExecuteTest_with_when_condition_should_not_enrich_if_when_condition_is_fa when(geoIPProcessorConfig.getEntries()).thenReturn(List.of(entry)); when(geoIPProcessorConfig.getWhenCondition()).thenReturn(whenCondition); + when(expressionEvaluator.isValidExpressionStatement(whenCondition)).thenReturn(true); final GeoIPProcessor geoIPProcessor = createObjectUnderTest(); diff --git a/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessor.java b/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessor.java index 8cc9c6a716..6470830e49 100644 --- a/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessor.java +++ b/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessor.java @@ -17,6 +17,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.annotations.SingleThread; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -119,6 +120,11 @@ public GrokProcessor(final PluginMetrics pluginMetrics, registerPatterns(); compileMatchPatterns(); + + if (grokProcessorConfig.getGrokWhen() != null && + (!expressionEvaluator.isValidExpressionStatement(grokProcessorConfig.getGrokWhen()))) { + throw new InvalidPluginConfigurationException("grok_when {} is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax"); + } } /** diff --git a/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorTests.java b/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorTests.java index aedad1fe5c..26f8420c1f 100644 --- a/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorTests.java +++ b/data-prepper-plugins/grok-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorTests.java @@ -25,6 +25,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import java.util.ArrayList; @@ -46,6 +47,7 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.not; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.anyBoolean; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.any; @@ -155,6 +157,16 @@ private GrokProcessor createObjectUnderTest() { pluginMetrics, grokProcessorConfig, grokCompiler, executorService, expressionEvaluator); } + @Test + void invalid_grok_when_throws_InvalidPluginConfigurationException() { + final String grokWhen = UUID.randomUUID().toString(); + + when(grokProcessorConfig.getGrokWhen()).thenReturn(grokWhen); + when(expressionEvaluator.isValidExpressionStatement(grokWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test public void testMatchMerge() throws JsonProcessingException, ExecutionException, InterruptedException, TimeoutException { when(grokProcessorConfig.getIncludePerformanceMetadata()).thenReturn(false); @@ -798,6 +810,7 @@ private void configureDefaultGrokProcessorConfig() { public void testNoGrok_when_GrokWhen_returns_false() throws JsonProcessingException { final String grokWhen = UUID.randomUUID().toString(); when(grokProcessorConfig.getGrokWhen()).thenReturn(grokWhen); + when(expressionEvaluator.isValidExpressionStatement(grokWhen)).thenReturn(true); grokProcessor = createObjectUnderTest(); capture.put("key_capture_1", "value_capture_1"); diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java index bcc8eb0a27..b3f283136a 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java @@ -176,7 +176,7 @@ public class KeyValueProcessorConfig { private boolean dropKeysWithNoValue = false; @JsonProperty("key_value_when") - @JsonPropertyDescription("Allows you to specify a [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + @JsonPropertyDescription("Allows you to specify a Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + "such as `/some-key == \"test\"`, that will be evaluated to determine whether " + "the processor should be applied to the event.") private String keyValueWhen; diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessor.java index b501eea7e0..816e03c211 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessor.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.exceptions.EventKeyNotFoundException; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -39,6 +40,14 @@ public AddEntryProcessor(final PluginMetrics pluginMetrics, final AddEntryProces super(pluginMetrics); this.entries = config.getEntries(); this.expressionEvaluator = expressionEvaluator; + + config.getEntries().forEach(entry -> { + if (entry.getAddWhen() != null + && !expressionEvaluator.isValidExpressionStatement(entry.getAddWhen())) { + throw new InvalidPluginConfigurationException( + String.format("add_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", entry.getAddWhen())); + } + }); } @Override diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java index 24f56ef2ba..53ca543c17 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -55,6 +56,12 @@ public ConvertEntryTypeProcessor(final PluginMetrics pluginMetrics, .orElse(List.of()); this.expressionEvaluator = expressionEvaluator; this.tagsOnFailure = convertEntryTypeProcessorConfig.getTagsOnFailure(); + + if (convertWhen != null + && !expressionEvaluator.isValidExpressionStatement(convertWhen)) { + throw new InvalidPluginConfigurationException( + String.format("convert_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", convertWhen)); + } } @Override diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java index 448d9bb0a4..ce6aaed15c 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import org.opensearch.dataprepper.typeconverter.ConverterArguments; import java.util.List; @@ -13,27 +14,34 @@ public class ConvertEntryTypeProcessorConfig implements ConverterArguments { @JsonProperty("key") + @JsonPropertyDescription("Key whose value needs to be converted to a different type.") private String key; @JsonProperty("keys") + @JsonPropertyDescription("List of keys whose value needs to be converted to a different type.") private List keys; @JsonProperty("type") + @JsonPropertyDescription("Target type for the key-value pair. Possible values are integer, long, double, big_decimal, string, and boolean. Default value is integer.") private TargetType type = TargetType.INTEGER; /** * Optional scale value used only in the case of BigDecimal converter */ @JsonProperty("scale") + @JsonPropertyDescription("Modifies the scale of the big_decimal when converting to a big_decimal. The default value is 0.") private int scale = 0; @JsonProperty("convert_when") + @JsonPropertyDescription("Specifies a condition using a [Data Prepper expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/) for performing the convert_entry_type operation. If specified, the convert_entry_type operation runs only when the expression evaluates to true.") private String convertWhen; @JsonProperty("null_values") + @JsonPropertyDescription("String representation of what constitutes a null value. If the field value equals one of these strings, then the value is considered null and is converted to null.") private List nullValues; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of tags to be added to the event metadata when the event fails to convert.") private List tagsOnFailure; public String getKey() { diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessor.java index 845ae40e38..0ed56b1b8d 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -38,6 +39,14 @@ public CopyValueProcessor(final PluginMetrics pluginMetrics, final CopyValueProc this.config = config; this.entries = config.getEntries(); this.expressionEvaluator = expressionEvaluator; + + config.getEntries().forEach(entry -> { + if (entry.getCopyWhen() != null + && !expressionEvaluator.isValidExpressionStatement(entry.getCopyWhen())) { + throw new InvalidPluginConfigurationException( + String.format("copy_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", entry.getCopyWhen())); + } + }); } @Override diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessor.java index cfadf70d03..5dc0ee8b42 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessor.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -38,6 +39,12 @@ public DeleteEntryProcessor(final PluginMetrics pluginMetrics, final DeleteEntry this.entries = config.getWithKeys(); this.deleteWhen = config.getDeleteWhen(); this.expressionEvaluator = expressionEvaluator; + + if (deleteWhen != null + && !expressionEvaluator.isValidExpressionStatement(deleteWhen)) { + throw new InvalidPluginConfigurationException( + String.format("delete_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", deleteWhen)); + } } @Override diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessor.java index d042f8fa28..9c8655a4cb 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -38,6 +39,13 @@ public ListToMapProcessor(final PluginMetrics pluginMetrics, final ListToMapProc super(pluginMetrics); this.config = config; this.expressionEvaluator = expressionEvaluator; + + if (config.getListToMapWhen() != null + && !expressionEvaluator.isValidExpressionStatement(config.getListToMapWhen())) { + throw new InvalidPluginConfigurationException( + String.format("list_to_map_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", + config.getListToMapWhen())); + } } @Override diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java index b63deb727c..13b6c2f0fb 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java @@ -84,7 +84,7 @@ static FlattenedElement fromOptionValue(final String option) { private FlattenedElement flattenedElement = FlattenedElement.FIRST; @JsonProperty("list_to_map_when") - @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will be " + "run on the event. Default is `null`. All events will be processed unless otherwise stated.") private String listToMapWhen; diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessor.java index d911cd6194..38b5c3cf82 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessor.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -43,6 +44,13 @@ public MapToListProcessor(final PluginMetrics pluginMetrics, final MapToListProc this.config = config; this.expressionEvaluator = expressionEvaluator; excludeKeySet.addAll(config.getExcludeKeys()); + + if (config.getMapToListWhen() != null + && !expressionEvaluator.isValidExpressionStatement(config.getMapToListWhen())) { + throw new InvalidPluginConfigurationException( + String.format("map_to_list_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", + config.getMapToListWhen())); + } } @Override diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java index ce317eca49..a402c27cdc 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java @@ -40,7 +40,7 @@ public class MapToListProcessorConfig { private String valueName = DEFAULT_VALUE_NAME; @JsonProperty("map_to_list_when") - @JsonPropertyDescription("A [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will " + "be run on the event. Default is `null`. All events will be processed unless otherwise stated.") private String mapToListWhen; diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessor.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessor.java index 05c1ad7530..25fb27a9fe 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessor.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.record.Record; @@ -35,6 +36,15 @@ public RenameKeyProcessor(final PluginMetrics pluginMetrics, final RenameKeyProc super(pluginMetrics); this.entries = config.getEntries(); this.expressionEvaluator = expressionEvaluator; + + config.getEntries().forEach(entry -> { + if (entry.getRenameWhen() != null + && !expressionEvaluator.isValidExpressionStatement(entry.getRenameWhen())) { + throw new InvalidPluginConfigurationException( + String.format("rename_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", + entry.getRenameWhen())); + } + }); } @Override diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java index d1ee0178a6..7449cc9968 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -30,9 +31,13 @@ public static class Entry { private EventKey toKey; @JsonProperty("overwrite_if_to_key_exists") + @JsonPropertyDescription("When set to true, the existing value is overwritten if key already exists in the event. The default value is false.") private boolean overwriteIfToKeyExists = false; @JsonProperty("rename_when") + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will be " + + "run on the event. Default is `null`. All events will be processed unless otherwise stated.") private String renameWhen; public EventKey getFromKey() { diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorConfig.java index e19723f20d..a9a8ff9ce6 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorConfig.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -15,9 +16,13 @@ public class SelectEntriesProcessorConfig { @NotEmpty @NotNull @JsonProperty("include_keys") + @JsonPropertyDescription("A list of keys to be selected from an event.") private List includeKeys; @JsonProperty("select_when") + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will be " + + "run on the event. Default is `null`. All events will be processed unless otherwise stated.") private String selectWhen; public List getIncludeKeys() { diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorTests.java index 36dc7ac5d4..2a61d05241 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorTests.java @@ -13,6 +13,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import java.util.ArrayList; @@ -46,6 +47,17 @@ public class AddEntryProcessorTests { @Mock private ExpressionEvaluator expressionEvaluator; + @Test + void invalid_add_when_throws_InvalidPluginConfigurationException() { + final String addWhen = UUID.randomUUID().toString(); + + when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("newMessage", null, 3, null, null, false, false,addWhen))); + + when(expressionEvaluator.isValidExpressionStatement(addWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test public void testSingleAddProcessorTests() { when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("newMessage", null, 3, null, null, false, false,null))); @@ -417,6 +429,7 @@ public void testKeyIsNotAdded_when_addWhen_condition_is_false() { final String addWhen = UUID.randomUUID().toString(); when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("newMessage", null, 3, null, null, false, false,addWhen))); + when(expressionEvaluator.isValidExpressionStatement(addWhen)).thenReturn(true); final AddEntryProcessor processor = createObjectUnderTest(); final Record record = getEvent("thisisamessage"); @@ -434,6 +447,7 @@ public void testMetadataKeyIsNotAdded_when_addWhen_condition_is_false() { final String addWhen = UUID.randomUUID().toString(); when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry(null, "newMessage", 3, null, null, false, false,addWhen))); + when(expressionEvaluator.isValidExpressionStatement(addWhen)).thenReturn(true); final AddEntryProcessor processor = createObjectUnderTest(); final Record record = getEventWithMetadata("thisisamessage", Map.of("key", "value")); diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java index 5f8b66a6a6..4165e32934 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorTests.java @@ -17,6 +17,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import java.math.BigDecimal; @@ -89,6 +90,18 @@ private Event executeAndGetProcessedEvent(final Record record) { return event; } + @Test + void invalid_convert_when_throws_InvalidPluginConfigurationException() { + final String convertWhen = UUID.randomUUID().toString(); + + when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("integer")); + when(mockConfig.getConvertWhen()).thenReturn(convertWhen); + + when(expressionEvaluator.isValidExpressionStatement(convertWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, () -> new ConvertEntryTypeProcessor(pluginMetrics, mockConfig, expressionEvaluator)); + } + @Test void testStringToIntegerConvertEntryTypeProcessor() { Integer testValue = 123; @@ -289,6 +302,7 @@ void testNoConversionWhenConvertWhenIsFalse() { final String convertWhen = UUID.randomUUID().toString(); when(mockConfig.getType()).thenReturn(TargetType.fromOptionValue("integer")); when(mockConfig.getConvertWhen()).thenReturn(convertWhen); + when(expressionEvaluator.isValidExpressionStatement(convertWhen)).thenReturn(true); final Record record = getMessage(UUID.randomUUID().toString(), testValue); when(expressionEvaluator.evaluateConditional(convertWhen, record.getData())).thenReturn(false); diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorTests.java index 98c6ad7d2c..bc1a9f6d1a 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorTests.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import java.util.Arrays; @@ -27,6 +28,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @@ -48,6 +50,16 @@ void setUp() { lenient().when(mockConfig.getOverwriteIfToListExists()).thenReturn(false); } + @Test + void invalid_copy_when_throws_InvalidPluginConfigurationException() { + final String copyWhen = UUID.randomUUID().toString(); + + when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("message", "newMessage", false, copyWhen))); + when(expressionEvaluator.isValidExpressionStatement(copyWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test public void testSingleCopyProcessorTests() { when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("message", "newMessage", false, null))); @@ -184,6 +196,7 @@ public void testKey_is_not_copied_when_copyWhen_returns_false() { when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("message2", "newMessage", true, copyWhen))); + when(expressionEvaluator.isValidExpressionStatement(copyWhen)).thenReturn(true); final CopyValueProcessor processor = createObjectUnderTest(); final Record record = getEvent("thisisamessage"); @@ -277,6 +290,8 @@ public void testCopyEntriesFromListWithWhenConditions() { createEntry("color", "fruit_color", true, copyWhen) )); + when(expressionEvaluator.isValidExpressionStatement(copyWhen)).thenReturn(true); + final CopyValueProcessor processor = createObjectUnderTest(); final Record record = getEventWithLists(List.of( Map.of("name", "apple", "color", "red", "shape", "round"), diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorTests.java index bc0fb78870..f2453e6ac0 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorTests.java @@ -15,6 +15,7 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import java.util.Collections; @@ -25,6 +26,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -40,6 +42,17 @@ public class DeleteEntryProcessorTests { private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); + @Test + void invalid_delete_when_throws_InvalidPluginConfigurationException() { + final String deleteWhen = UUID.randomUUID().toString(); + + when(mockConfig.getDeleteWhen()).thenReturn(deleteWhen); + + when(expressionEvaluator.isValidExpressionStatement(deleteWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test public void testSingleDeleteProcessorTest() { when(mockConfig.getWithKeys()).thenReturn(List.of(eventKeyFactory.createEventKey("message", EventKeyFactory.EventAction.DELETE))); @@ -92,6 +105,7 @@ public void testKeyIsNotDeleted_when_deleteWhen_returns_false() { when(mockConfig.getWithKeys()).thenReturn(List.of(eventKeyFactory.createEventKey("message", EventKeyFactory.EventAction.DELETE))); final String deleteWhen = UUID.randomUUID().toString(); when(mockConfig.getDeleteWhen()).thenReturn(deleteWhen); + when(expressionEvaluator.isValidExpressionStatement(deleteWhen)).thenReturn(true); final DeleteEntryProcessor processor = createObjectUnderTest(); final Record record = getEvent("thisisamessage"); diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorTest.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorTest.java index a32bd8f8d5..a5321b719e 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorTest.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorTest.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import java.util.Collections; @@ -26,6 +27,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @@ -51,6 +53,16 @@ void setUp() { lenient().when(mockConfig.getExtractValue()).thenReturn(false); } + @Test + void invalid_list_to_map_when_throws_InvalidPluginConfigurationException() { + final String listToMapWhen = UUID.randomUUID().toString(); + when(mockConfig.getListToMapWhen()).thenReturn(listToMapWhen); + + when(expressionEvaluator.isValidExpressionStatement(listToMapWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test public void testValueExtractionWithFlattenAndWriteToRoot() { when(mockConfig.getValueKey()).thenReturn("value"); @@ -329,6 +341,7 @@ public void testTagsAreAddedOnFailure() { public void testNoValueExtraction_when_the_when_condition_returns_false() { final String whenCondition = UUID.randomUUID().toString(); when(mockConfig.getListToMapWhen()).thenReturn(whenCondition); + when(expressionEvaluator.isValidExpressionStatement(whenCondition)).thenReturn(true); final ListToMapProcessor processor = createObjectUnderTest(); final Record testRecord = createTestRecord(); diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorTest.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorTest.java index 83d736ba21..1b2ca68833 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorTest.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorTest.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import java.util.ArrayList; @@ -26,6 +27,7 @@ import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.when; @@ -53,6 +55,16 @@ void setUp() { lenient().when(mockConfig.getTagsOnFailure()).thenReturn(new ArrayList<>()); } + @Test + void invalid_map_to_list_when_throws_InvalidPluginConfigurationException() { + final String mapToListWhen = UUID.randomUUID().toString(); + when(mockConfig.getMapToListWhen()).thenReturn(mapToListWhen); + + when(expressionEvaluator.isValidExpressionStatement(mapToListWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test void testMapToListSuccessWithDefaultOptions() { @@ -312,6 +324,7 @@ public void testConvertFieldToListSuccessWithRootAsSource() { public void testEventNotProcessedWhenTheWhenConditionIsFalse() { final String whenCondition = UUID.randomUUID().toString(); when(mockConfig.getMapToListWhen()).thenReturn(whenCondition); + when(expressionEvaluator.isValidExpressionStatement(whenCondition)).thenReturn(true); final MapToListProcessor processor = createObjectUnderTest(); final Record testRecord = createTestRecord(); diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorTests.java index 6ae362bc46..3cdf47e344 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorTests.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -29,6 +30,7 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -44,6 +46,17 @@ public class RenameKeyProcessorTests { private final EventKeyFactory eventKeyFactory = TestEventKeyFactory.getTestEventFactory(); + @Test + void invalid_rename_when_throws_InvalidPluginConfigurationException() { + final String renameWhen = UUID.randomUUID().toString(); + when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("message", "newMessage", true, renameWhen))); + + + when(expressionEvaluator.isValidExpressionStatement(renameWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test public void testSingleOverwriteRenameProcessorTests() { when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("message", "newMessage", true, null))); @@ -123,6 +136,7 @@ public void testNoRename_when_RenameWhen_returns_false() { final String renameWhen = UUID.randomUUID().toString(); when(mockConfig.getEntries()).thenReturn(createListOfEntries(createEntry("message", "newMessage", false, renameWhen))); + when(expressionEvaluator.isValidExpressionStatement(renameWhen)).thenReturn(true); final RenameKeyProcessor processor = createObjectUnderTest(); final Record record = getEvent("thisisamessage"); diff --git a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorTests.java b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorTests.java index 048f304582..c514687606 100644 --- a/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorTests.java +++ b/data-prepper-plugins/mutate-event-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorTests.java @@ -39,6 +39,16 @@ public class SelectEntriesProcessorTests { @Mock private ExpressionEvaluator expressionEvaluator; + @Test + void invalid_select_when_throws_InvalidPluginConfigurationException() { + final String selectWhen = UUID.randomUUID().toString(); + + when(mockConfig.getSelectWhen()).thenReturn(selectWhen); + + when(expressionEvaluator.isValidExpressionStatement(selectWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } @Test public void testSelectEntriesProcessor() { when(mockConfig.getIncludeKeys()).thenReturn(List.of("key1", "key2")); diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessor.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessor.java index 6bc89178d8..eacc41b151 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessor.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessor.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.Processor; import java.util.HashMap; @@ -43,6 +44,12 @@ public SplitStringProcessor(final PluginMetrics pluginMetrics, final SplitString } else { patternMap.put(entry.getDelimiter(), Pattern.compile(Pattern.quote(entry.getDelimiter()))); } + + if (entry.getSplitWhen() != null + && !expressionEvaluator.isValidExpressionStatement(entry.getSplitWhen())) { + throw new InvalidPluginConfigurationException( + String.format("split_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", entry.getSplitWhen())); + } } } diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessor.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessor.java index e6dceb62fc..dbfc2d07c8 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessor.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessor.java @@ -11,6 +11,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventKey; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.Processor; import java.util.HashMap; @@ -35,6 +36,12 @@ public SubstituteStringProcessor(final PluginMetrics pluginMetrics, final Substi for(final SubstituteStringProcessorConfig.Entry entry : config.getEntries()) { patternMap.put(entry.getFrom(), Pattern.compile(entry.getFrom())); + + if (entry.getSubstituteWhen() != null + && !expressionEvaluator.isValidExpressionStatement(entry.getSubstituteWhen())) { + throw new InvalidPluginConfigurationException( + String.format("substitute_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", entry.getSubstituteWhen())); + } } } diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java index 4a8f53f0fe..722a42dcbe 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java @@ -24,6 +24,9 @@ public static class Entry { private String to; @JsonProperty("substitute_when") + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), " + + "such as `/some-key == \"test\"'`, that will be evaluated to determine whether the processor will be " + + "run on the event. Default is `null`. All events will be processed unless otherwise stated.") private String substituteWhen; public EventKey getSource() { @@ -50,7 +53,7 @@ public Entry(final EventKey source, final String from, final String to, final St public Entry() {} } - @JsonPropertyDescription("List of entries. Valid values are `source`, `from`, and `to`.") + @JsonPropertyDescription("List of entries. Valid values are `source`, `from`, and `to`, and `substitute_when`.") private List entries; public List getEntries() { diff --git a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorTests.java b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorTests.java index 7883dcfd05..ad24657f1a 100644 --- a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorTests.java +++ b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorTests.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -57,6 +58,17 @@ private SplitStringProcessor createObjectUnderTest() { return new SplitStringProcessor(pluginMetrics, config, expressionEvaluator); } + @Test + void invalid_split_when_throws_InvalidPluginConfigurationException() { + final String splitWhen = UUID.randomUUID().toString(); + when(config.getIterativeConfig()).thenReturn(Collections.singletonList(createEntry("message", ",", null, splitWhen))); + when(config.getEntries()).thenReturn(Collections.singletonList(createEntry("message", ",", null, splitWhen))); + + when(expressionEvaluator.isValidExpressionStatement(splitWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @ParameterizedTest @ArgumentsSource(SplitStringArgumentsProvider.class) void testSingleSplitProcessor(final String message, final List splitMessage) { @@ -111,6 +123,7 @@ void test_event_is_the_same_when_splitWhen_condition_returns_false() { when(config.getIterativeConfig()).thenReturn(Collections.singletonList(createEntry("message", ",", null, splitWhen))); when(config.getEntries()).thenReturn(Collections.singletonList(createEntry("message", ",", null, splitWhen))); + when(expressionEvaluator.isValidExpressionStatement(splitWhen)).thenReturn(true); final SplitStringProcessor splitStringProcessor = createObjectUnderTest(); final Record record = createEvent(message); diff --git a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorTests.java b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorTests.java index dd8d9b1dd8..854eb0a72c 100644 --- a/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorTests.java +++ b/data-prepper-plugins/mutate-string-processors/src/test/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorTests.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.event.EventKey; import org.opensearch.dataprepper.model.event.EventKeyFactory; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -56,6 +57,16 @@ public void setup() { lenient().when(config.getEntries()).thenReturn(Collections.singletonList(createEntry("message", "a", "b", null))); } + @Test + void invalid_substitute_when_throws_InvalidPluginConfigurationException() { + final String substituteWhen = UUID.randomUUID().toString(); + when(config.getEntries()).thenReturn(Collections.singletonList(createEntry("message", "a", "b", substituteWhen))); + + when(expressionEvaluator.isValidExpressionStatement(substituteWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test public void testHappyPathSubstituteStringProcessor() { final SubstituteStringProcessor processor = createObjectUnderTest(); @@ -151,6 +162,7 @@ public void test_events_are_identical_when_substituteWhen_condition_returns_fals when(config.getIterativeConfig()).thenReturn(Collections.singletonList(createEntry("message", "[?\\\\+]", "b", substituteWhen))); when(config.getEntries()).thenReturn(Collections.singletonList(createEntry("message", "[?\\\\+]", "b", substituteWhen))); + when(expressionEvaluator.isValidExpressionStatement(substituteWhen)).thenReturn(true); final SubstituteStringProcessor processor = createObjectUnderTest(); final Record record = getEvent("abcd"); diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java index e5893476e0..8a6f8e17e5 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java @@ -24,6 +24,7 @@ public class ObfuscationProcessorConfig { private String source; @JsonProperty("patterns") + @JsonPropertyDescription("A list of regex patterns that allow you to obfuscate specific parts of a field. Only parts that match the regex pattern will obfuscate. When not provided, the processor obfuscates the whole field.") private List patterns; @JsonProperty("target") @@ -33,7 +34,7 @@ public class ObfuscationProcessorConfig { private String target; @JsonProperty("action") - @JsonPropertyDescription("The obfuscation action. As of Data Prepper 2.3, only the `mask` action is supported.") + @JsonPropertyDescription("The obfuscation action. Available actions include 'hash' and 'mask'.") private PluginModel action; @JsonProperty("obfuscate_when") @@ -94,7 +95,7 @@ public boolean getSingleWordOnly() { void validateObfuscateWhen(final ExpressionEvaluator expressionEvaluator) { if (obfuscateWhen != null && !expressionEvaluator.isValidExpressionStatement(obfuscateWhen)) { - throw new InvalidPluginConfigurationException(String.format("obfuscate_when value %s is not a valid Data Prepper expression statement", obfuscateWhen)); + throw new InvalidPluginConfigurationException(String.format("obfuscate_when value %s is not a valid Data Prepper expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", obfuscateWhen)); } } } diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java index 18acb3dfd8..a22d635163 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java @@ -13,6 +13,7 @@ import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.record.Record; import io.micrometer.core.instrument.Counter; @@ -67,6 +68,13 @@ protected AbstractParseProcessor(final PluginMetrics pluginMetrics, processingFailuresCounter = pluginMetrics.counter(PROCESSING_FAILURES); this.expressionEvaluator = expressionEvaluator; this.eventKeyFactory = eventKeyFactory; + + if (commonParseConfig.getParseWhen() != null + && !expressionEvaluator.isValidExpressionStatement(commonParseConfig.getParseWhen())) { + throw new InvalidPluginConfigurationException( + String.format("parse_when value of %s is not a valid expression statement. " + + "See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax.", commonParseConfig.getParseWhen())); + } } /** diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java index 6fad364e17..2a8484131e 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java @@ -21,24 +21,31 @@ public class ParseIonProcessorConfig implements CommonParseConfig { @NotBlank @JsonProperty("source") + @JsonPropertyDescription("The field in the event that will be parsed. Default value is message.") private String source = DEFAULT_SOURCE; @JsonProperty("destination") + @JsonPropertyDescription("The destination field of the parsed JSON. Defaults to the root of the event. Cannot be an empty string, /, or any white-space-only string because these are not valid event fields.") private String destination; @JsonProperty("pointer") + @JsonPropertyDescription("A JSON pointer to the field to be parsed. There is no pointer by default, meaning the entire source is parsed. The pointer can access JSON array indexes as well. If the JSON pointer is invalid then the entire source data is parsed into the outgoing event. If the key that is pointed to already exists in the event and the destination is the root, then the pointer uses the entire path of the key.") private String pointer; @JsonProperty("parse_when") + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the processor will be run on the event.") private String parseWhen; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of strings specifying the tags to be set in the event that the processor fails or an unknown exception occurs during parsing.") private List tagsOnFailure; @JsonProperty("overwrite_if_destination_exists") + @JsonPropertyDescription("Overwrites the destination if set to true. Set to false to prevent changing a destination value that exists. Defaults to true.") private boolean overwriteIfDestinationExists = true; @JsonProperty + @JsonPropertyDescription("If true, the configured source field will be deleted after the JSON data is parsed into separate fields.") private boolean deleteSource = false; @JsonProperty("handle_failed_events") diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java index b6a1b14a23..13a6e2e24a 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java @@ -21,24 +21,31 @@ public class ParseJsonProcessorConfig implements CommonParseConfig { @NotBlank @JsonProperty("source") + @JsonPropertyDescription("The field in the event that will be parsed. Default value is message.") private String source = DEFAULT_SOURCE; @JsonProperty("destination") + @JsonPropertyDescription("The destination field of the parsed JSON. Defaults to the root of the event. Cannot be an empty string, /, or any white-space-only string because these are not valid event fields.") private String destination; @JsonProperty("pointer") + @JsonPropertyDescription("A JSON pointer to the field to be parsed. There is no pointer by default, meaning the entire source is parsed. The pointer can access JSON array indexes as well. If the JSON pointer is invalid then the entire source data is parsed into the outgoing event. If the key that is pointed to already exists in the event and the destination is the root, then the pointer uses the entire path of the key.") private String pointer; @JsonProperty("parse_when") + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the processor will be run on the event.") private String parseWhen; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of strings specifying the tags to be set in the event that the processor fails or an unknown exception occurs during parsing.") private List tagsOnFailure; @JsonProperty("overwrite_if_destination_exists") + @JsonPropertyDescription("Overwrites the destination if set to true. Set to false to prevent changing a destination value that exists. Defaults to true.") private boolean overwriteIfDestinationExists = true; @JsonProperty + @JsonPropertyDescription("If true, the configured source field will be deleted after the JSON data is parsed into separate fields.") private boolean deleteSource = false; @JsonProperty("handle_failed_events") diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java index f84f2de4b6..82d19a2098 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java @@ -16,24 +16,31 @@ public class ParseXmlProcessorConfig implements CommonParseConfig { @NotBlank @JsonProperty("source") + @JsonPropertyDescription("The field in the event that will be parsed. Default value is message.") private String source = DEFAULT_SOURCE; @JsonProperty("destination") + @JsonPropertyDescription("The destination field of the parsed JSON. Defaults to the root of the event. Cannot be an empty string, /, or any white-space-only string because these are not valid event fields.") private String destination; @JsonProperty("pointer") + @JsonPropertyDescription("A JSON pointer to the field to be parsed. There is no pointer by default, meaning the entire source is parsed. The pointer can access JSON array indexes as well. If the JSON pointer is invalid then the entire source data is parsed into the outgoing event. If the key that is pointed to already exists in the event and the destination is the root, then the pointer uses the entire path of the key.") private String pointer; @JsonProperty("parse_when") + @JsonPropertyDescription("A Data Prepper [conditional expression](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/), such as '/some-key == \"test\"', that will be evaluated to determine whether the processor will be run on the event.") private String parseWhen; @JsonProperty("tags_on_failure") + @JsonPropertyDescription("A list of strings specifying the tags to be set in the event that the processor fails or an unknown exception occurs during parsing.") private List tagsOnFailure; @JsonProperty("overwrite_if_destination_exists") + @JsonPropertyDescription("Overwrites the destination if set to true. Set to false to prevent changing a destination value that exists. Defaults to true.") private boolean overwriteIfDestinationExists = true; @JsonProperty + @JsonPropertyDescription("If true, the configured source field will be deleted after the JSON data is parsed into separate fields.") private boolean deleteSource = false; @JsonProperty("handle_failed_events") diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java index 8bd63c3eec..dce1c79b2e 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorTest.java @@ -12,13 +12,18 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.opensearch.dataprepper.plugins.processor.parse.json.ParseJsonProcessorTest; +import java.util.UUID; + import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verifyNoInteractions; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -40,7 +45,7 @@ public void setup() { when(pluginMetrics.counter("recordsIn")).thenReturn(mock(Counter.class)); when(pluginMetrics.counter("recordsOut")).thenReturn(mock(Counter.class)); when(pluginMetrics.counter("processingFailures")).thenReturn(this.processingFailuresCounter); - when(pluginMetrics.counter("parseErrors")).thenReturn(this.parseErrorsCounter); + lenient().when(pluginMetrics.counter("parseErrors")).thenReturn(this.parseErrorsCounter); when(processorConfig.getHandleFailedEventsOption()).thenReturn(handleFailedEventsOption); } @@ -49,6 +54,16 @@ protected AbstractParseProcessor createObjectUnderTest() { return new ParseIonProcessor(pluginMetrics, ionProcessorConfig, expressionEvaluator, testEventKeyFactory); } + @Test + void invalid_parse_when_throws_InvalidPluginConfigurationException() { + final String parseWhen = UUID.randomUUID().toString(); + + when(processorConfig.getParseWhen()).thenReturn(parseWhen); + when(expressionEvaluator.isValidExpressionStatement(parseWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test void test_when_using_ion_features_then_processorParsesCorrectly() { parseJsonProcessor = createObjectUnderTest(); diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java index 9aac54b23f..50dd55b501 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorTest.java @@ -6,6 +6,11 @@ package org.opensearch.dataprepper.plugins.processor.parse.json; import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.expression.ExpressionEvaluator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -13,14 +18,10 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.event.TestEventFactory; import org.opensearch.dataprepper.event.TestEventKeyFactory; -import org.opensearch.dataprepper.expression.ExpressionEvaluator; -import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventBuilder; import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; -import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.opensearch.dataprepper.plugins.processor.parse.CommonParseConfig; @@ -35,7 +36,9 @@ import static java.util.Map.entry; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; @@ -82,7 +85,7 @@ public void setup() { when(pluginMetrics.counter("recordsIn")).thenReturn(mock(Counter.class)); when(pluginMetrics.counter("recordsOut")).thenReturn(mock(Counter.class)); when(pluginMetrics.counter("processingFailures")).thenReturn(processingFailuresCounter); - when(pluginMetrics.counter("parseErrors")).thenReturn(parseErrorsCounter); + lenient().when(pluginMetrics.counter("parseErrors")).thenReturn(parseErrorsCounter); when(processorConfig.getHandleFailedEventsOption()).thenReturn(handleFailedEventsOption); } @@ -90,6 +93,16 @@ protected AbstractParseProcessor createObjectUnderTest() { return new ParseJsonProcessor(pluginMetrics, jsonProcessorConfig, expressionEvaluator, testEventKeyFactory); } + @Test + void invalid_parse_when_throws_InvalidPluginConfigurationException() { + final String parseWhen = UUID.randomUUID().toString(); + + when(processorConfig.getParseWhen()).thenReturn(parseWhen); + when(expressionEvaluator.isValidExpressionStatement(parseWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test void test_when_differentSourceAndDestination_then_processorParsesCorrectly() { final String source = "different_source"; @@ -413,6 +426,7 @@ void test_when_condition_skips_processing_when_evaluates_to_false() { final Map data = Collections.singletonMap("key", "value"); final String serializedMessage = convertMapToJSONString(data); final Record testEvent = createMessageEvent(serializedMessage); + when(expressionEvaluator.isValidExpressionStatement(whenCondition)).thenReturn(true); when(expressionEvaluator.evaluateConditional(whenCondition, testEvent.getData())).thenReturn(false); parseJsonProcessor = createObjectUnderTest(); // need to recreate so that new config options are used @@ -439,6 +453,7 @@ void test_tags_when_json_parse_fails() { List testTags = List.of("tag1", "tag2"); when(processorConfig.getTagsOnFailure()).thenReturn(testTags); final Record testEvent = createMessageEvent("{key:}"); + when(expressionEvaluator.isValidExpressionStatement(whenCondition)).thenReturn(true); when(expressionEvaluator.evaluateConditional(whenCondition, testEvent.getData())).thenReturn(true); parseJsonProcessor = createObjectUnderTest(); @@ -461,6 +476,7 @@ void when_evaluate_conditional_throws_RuntimeException_events_are_not_dropped() final Map data = Collections.singletonMap("key", "value"); final String serializedMessage = convertMapToJSONString(data); final Record testEvent = createMessageEvent(serializedMessage); + when(expressionEvaluator.isValidExpressionStatement(whenCondition)).thenReturn(true); when(expressionEvaluator.evaluateConditional(whenCondition, testEvent.getData())).thenThrow(RuntimeException.class); parseJsonProcessor = createObjectUnderTest(); diff --git a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java index 900a7a7bef..b1dc756c25 100644 --- a/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java +++ b/data-prepper-plugins/parse-json-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorTest.java @@ -18,6 +18,7 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.event.EventKeyFactory; import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.processor.parse.AbstractParseProcessor; import org.opensearch.dataprepper.test.helper.ReflectivelySetField; @@ -32,9 +33,11 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.Mockito.when; import static org.opensearch.dataprepper.plugins.processor.parse.xml.ParseXmlProcessorConfig.DEFAULT_SOURCE; @@ -72,7 +75,7 @@ public void setup() { when(pluginMetrics.counter("recordsIn")).thenReturn(mock(Counter.class)); when(pluginMetrics.counter("recordsOut")).thenReturn(mock(Counter.class)); when(pluginMetrics.counter("processingFailures")).thenReturn(processingFailuresCounter); - when(pluginMetrics.counter("parseErrors")).thenReturn(parseErrorsCounter); + lenient().when(pluginMetrics.counter("parseErrors")).thenReturn(parseErrorsCounter); when(processorConfig.getHandleFailedEventsOption()).thenReturn(handleFailedEventsOption); } @@ -80,6 +83,16 @@ protected AbstractParseProcessor createObjectUnderTest() { return new ParseXmlProcessor(pluginMetrics, processorConfig, expressionEvaluator, testEventKeyFactory); } + @Test + void invalid_parse_when_throws_InvalidPluginConfigurationException() { + final String parseWhen = UUID.randomUUID().toString(); + + when(processorConfig.getParseWhen()).thenReturn(parseWhen); + when(expressionEvaluator.isValidExpressionStatement(parseWhen)).thenReturn(false); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @Test void test_when_using_xml_features_then_processorParsesCorrectly() { parseXmlProcessor = createObjectUnderTest(); diff --git a/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessor.java b/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessor.java index 2063652a7c..6d29082bd6 100644 --- a/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessor.java +++ b/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessor.java @@ -10,6 +10,7 @@ import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.processor.AbstractProcessor; import org.opensearch.dataprepper.model.processor.Processor; @@ -39,6 +40,14 @@ public TruncateProcessor(final PluginMetrics pluginMetrics, final TruncateProces super(pluginMetrics); this.expressionEvaluator = expressionEvaluator; this.entries = config.getEntries(); + + config.getEntries().forEach(entry -> { + if (entry.getTruncateWhen() != null + && !expressionEvaluator.isValidExpressionStatement(entry.getTruncateWhen())) { + throw new InvalidPluginConfigurationException( + String.format("truncate_when %s is not a valid expression statement. See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax", entry.getTruncateWhen())); + } + }); } private String getTruncatedValue(final String value, final int startIndex, final Integer length) { diff --git a/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java b/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java index 02c83f5773..0d525cddd7 100644 --- a/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java +++ b/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java @@ -32,6 +32,7 @@ public static class Entry { private Integer length; @JsonProperty("recursive") + @JsonPropertyDescription("Recursively truncates the fields. If the value of a field is a map (json object), then it recursively applies truncate operation on the fields in the map.") private Boolean recurse = false; @JsonProperty("truncate_when") diff --git a/data-prepper-plugins/truncate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorTests.java b/data-prepper-plugins/truncate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorTests.java index 00af15ed63..065b87ab80 100644 --- a/data-prepper-plugins/truncate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorTests.java +++ b/data-prepper-plugins/truncate-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorTests.java @@ -9,12 +9,15 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.record.Record; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.api.extension.ExtensionContext; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; + +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.params.provider.Arguments.arguments; import org.junit.jupiter.params.provider.ArgumentsProvider; import org.junit.jupiter.params.provider.ArgumentsSource; @@ -50,6 +53,16 @@ private TruncateProcessor createObjectUnderTest() { return new TruncateProcessor(pluginMetrics, config, expressionEvaluator); } + @Test + void invalid_truncate_when_throws_InvalidPluginConfigurationException() { + final String truncateWhen = UUID.randomUUID().toString(); + when(expressionEvaluator.isValidExpressionStatement(truncateWhen)).thenReturn(false); + + when(config.getEntries()).thenReturn(Collections.singletonList(createEntry(List.of("message"), null, null, truncateWhen, false))); + + assertThrows(InvalidPluginConfigurationException.class, this::createObjectUnderTest); + } + @ParameterizedTest @ArgumentsSource(TruncateArgumentsProvider.class) void testTruncateProcessor(final Object messageValue, final Integer startAt, final Integer truncateLength, final Object truncatedMessage) { @@ -84,6 +97,7 @@ void test_event_is_the_same_when_truncateWhen_condition_returns_false() { final String message = UUID.randomUUID().toString(); when(config.getEntries()).thenReturn(Collections.singletonList(createEntry(List.of("message"), null, 5, truncateWhen, false))); + when(expressionEvaluator.isValidExpressionStatement(truncateWhen)).thenReturn(true); final TruncateProcessor truncateProcessor = createObjectUnderTest(); final Record record = createEvent("message", message); diff --git a/data-prepper-plugins/write-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/write_json/WriteJsonProcessorConfig.java b/data-prepper-plugins/write-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/write_json/WriteJsonProcessorConfig.java index 86d69dfedd..a36bc41ad5 100644 --- a/data-prepper-plugins/write-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/write_json/WriteJsonProcessorConfig.java +++ b/data-prepper-plugins/write-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/write_json/WriteJsonProcessorConfig.java @@ -6,14 +6,17 @@ package org.opensearch.dataprepper.plugins.processor.write_json; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; public class WriteJsonProcessorConfig { @JsonProperty("source") + @JsonPropertyDescription("Specifies the name of the field in the event containing the message or object to be parsed.") @NotNull private String source; @JsonProperty("target") + @JsonPropertyDescription("Specifies the name of the field in which the resulting JSON string should be stored. If target is not specified, then the source field is used.") private String target; public String getSource() { From b10d4651f0a9d34abe51a0d8693c893f08a6dd81 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Wed, 21 Aug 2024 14:32:05 -0500 Subject: [PATCH 125/159] ENH: respect json order and class description on processors (#4857) * ENH: json order and class description on processors Signed-off-by: George Chen --- .../dataprepper/model/configuration/ConditionalRoute.java | 5 +++++ .../schemas/DataPrepperPluginSchemaExecute.java | 3 ++- .../processor/aggregate/AggregateProcessorConfig.java | 8 +++++++- .../anomalydetector/AnomalyDetectorProcessorConfig.java | 5 +++++ .../dataprepper/plugins/processor/DelayProcessor.java | 5 +++++ .../dataprepper/plugins/processor/StringProcessor.java | 4 ++++ .../plugins/codec/csv/CsvOutputCodecConfig.java | 4 ++++ .../plugins/processor/date/DateProcessorConfig.java | 6 ++++++ .../processor/decompress/DecompressProcessorConfig.java | 5 +++++ .../plugins/processor/dissect/DissectProcessorConfig.java | 6 ++++++ .../plugins/processor/drop/DropEventProcessorConfig.java | 4 ++++ .../plugins/processor/flatten/FlattenProcessorConfig.java | 4 ++++ .../plugins/geoip/processor/GeoIPProcessorConfig.java | 5 +++++ .../plugins/processor/grok/GrokProcessorConfig.java | 5 +++++ .../processor/keyvalue/KeyValueProcessorConfig.java | 4 ++++ .../processor/mutateevent/AddEntryProcessorConfig.java | 4 ++++ .../mutateevent/ConvertEntryTypeProcessorConfig.java | 5 +++++ .../processor/mutateevent/CopyValueProcessorConfig.java | 5 +++++ .../processor/mutateevent/DeleteEntryProcessorConfig.java | 6 ++++++ .../processor/mutateevent/ListToMapProcessorConfig.java | 5 +++++ .../processor/mutateevent/MapToListProcessorConfig.java | 5 +++++ .../processor/mutateevent/RenameKeyProcessorConfig.java | 4 ++++ .../mutateevent/SelectEntriesProcessorConfig.java | 4 ++++ .../mutatestring/SplitStringProcessorConfig.java | 5 +++++ .../mutatestring/SubstituteStringProcessorConfig.java | 5 +++++ .../plugins/processor/mutatestring/WithKeysConfig.java | 5 +++++ .../processor/obfuscation/ObfuscationProcessorConfig.java | 5 +++++ .../otelmetrics/OtelMetricsRawProcessorConfig.java | 5 +++++ .../processor/oteltrace/OtelTraceRawProcessorConfig.java | 5 +++++ .../processor/parse/ion/ParseIonProcessorConfig.java | 4 ++++ .../processor/parse/json/ParseJsonProcessorConfig.java | 5 +++++ .../processor/parse/xml/ParseXmlProcessorConfig.java | 4 ++++ .../plugins/processor/ServiceMapProcessorConfig.java | 5 +++++ .../processor/splitevent/SplitEventProcessorConfig.java | 8 ++++++-- .../processor/translate/TranslateProcessorConfig.java | 5 ++++- .../processor/truncate/TruncateProcessorConfig.java | 5 +++++ .../processor/useragent/UserAgentProcessorConfig.java | 5 +++++ .../processor/write_json/WriteJsonProcessorConfig.java | 4 ++++ 38 files changed, 181 insertions(+), 5 deletions(-) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/ConditionalRoute.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/ConditionalRoute.java index 9dbc7bcc26..0055702169 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/ConditionalRoute.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/ConditionalRoute.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.model.configuration; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.core.JacksonException; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; @@ -27,6 +29,9 @@ * * @since 2.0 */ +@JsonPropertyOrder +@JsonClassDescription("The key-value pair defines routing condition, where the key is the name of a route and the " + + "value is a Data Prepper expression representing the routing condition.") @JsonSerialize(using = ConditionalRoute.ConditionalRouteSerializer.class) @JsonDeserialize(using = ConditionalRoute.ConditionalRouteDeserializer.class) public class ConditionalRoute { diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java index a505a013c4..75115eb7e6 100644 --- a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java +++ b/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/DataPrepperPluginSchemaExecute.java @@ -25,6 +25,7 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static com.github.victools.jsonschema.module.jackson.JacksonOption.RESPECT_JSONPROPERTY_ORDER; import static com.github.victools.jsonschema.module.jackson.JacksonOption.RESPECT_JSONPROPERTY_REQUIRED; public class DataPrepperPluginSchemaExecute implements Runnable { @@ -53,7 +54,7 @@ public static void main(String[] args) { @Override public void run() { final List modules = List.of( - new CustomJacksonModule(RESPECT_JSONPROPERTY_REQUIRED), + new CustomJacksonModule(RESPECT_JSONPROPERTY_REQUIRED, RESPECT_JSONPROPERTY_ORDER), new JakartaValidationModule(JakartaValidationOption.NOT_NULLABLE_FIELD_IS_REQUIRED, JakartaValidationOption.INCLUDE_PATTERN_EXPRESSIONS) ); diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java index cfb986fe53..40ed0f14d7 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.aggregate; -import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonClassDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import org.opensearch.dataprepper.model.configuration.PluginModel; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; @@ -15,6 +17,10 @@ import java.time.Duration; import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `aggregate` processor groups events based on the values of identification_keys. " + + "Then, the processor performs an action on each group, helping reduce unnecessary log volume and " + + "creating aggregated logs over time.") public class AggregateProcessorConfig { static int DEFAULT_GROUP_DURATION_SECONDS = 180; 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 6545bbb552..0eb59edc58 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 @@ -5,6 +5,8 @@ package org.opensearch.dataprepper.plugins.processor.anomalydetector; +import com.fasterxml.jackson.annotation.JsonClassDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import org.opensearch.dataprepper.model.configuration.PluginModel; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.fasterxml.jackson.annotation.JsonProperty; @@ -14,6 +16,9 @@ import java.util.Collections; import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The anomaly detector processor takes structured data and runs anomaly detection algorithms " + + "on fields that you can configure in that data.") public class AnomalyDetectorProcessorConfig { @JsonPropertyDescription("The ML algorithm (or model) used to detect anomalies. You must provide a mode. See random_cut_forest mode.") @JsonProperty("mode") diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/DelayProcessor.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/DelayProcessor.java index e1ce11ff92..b6681fea19 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/DelayProcessor.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/DelayProcessor.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.plugins.processor; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.processor.Processor; @@ -52,6 +54,9 @@ public void shutdown() { } + @JsonPropertyOrder + @JsonClassDescription("This processor will add a delay into the processor chain. " + + "Typically, you should use this only for testing, experimenting, and debugging.") public static class Configuration { @JsonProperty("for") private Duration delayFor = Duration.ofSeconds(1); diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java index 3cf2953e06..c3d908a8da 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/processor/StringProcessor.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.plugins.processor; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.configuration.PluginSetting; @@ -40,6 +42,8 @@ public class StringProcessor implements Processor, Record> private final boolean upperCase; + @JsonPropertyOrder + @JsonClassDescription("The `string_converter` processor converts a string to uppercase or lowercase.") public static class Configuration { @JsonPropertyDescription("Whether to convert to uppercase (`true`) or lowercase (`false`).") private boolean upperCase = true; diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodecConfig.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodecConfig.java index d6862ab2e1..133e11cad9 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodecConfig.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/codec/csv/CsvOutputCodecConfig.java @@ -4,12 +4,16 @@ */ package org.opensearch.dataprepper.plugins.codec.csv; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.Valid; import jakarta.validation.constraints.Size; import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `csv` processor parses comma-separated values (CSVs) from the event into columns.") public class CsvOutputCodecConfig { static final String DEFAULT_DELIMITER = ","; diff --git a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java index b62f1b6efd..c6a4c471b7 100644 --- a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java +++ b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java @@ -5,9 +5,11 @@ package org.opensearch.dataprepper.plugins.processor.date; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.AssertTrue; import java.time.ZoneId; @@ -15,6 +17,10 @@ import java.util.Locale; import java.time.format.DateTimeFormatter; +@JsonPropertyOrder +@JsonClassDescription("The `date` processor adds a default timestamp to an event, parses timestamp fields, " + + "and converts timestamp information to the International Organization for Standardization (ISO) 8601 format. " + + "This timestamp information can be used as an event timestamp.") public class DateProcessorConfig { static final Boolean DEFAULT_FROM_TIME_RECEIVED = false; static final Boolean DEFAULT_TO_ORIGINATION_METADATA = false; diff --git a/data-prepper-plugins/decompress-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/decompress/DecompressProcessorConfig.java b/data-prepper-plugins/decompress-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/decompress/DecompressProcessorConfig.java index 32248cdba5..be1238885c 100644 --- a/data-prepper-plugins/decompress-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/decompress/DecompressProcessorConfig.java +++ b/data-prepper-plugins/decompress-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/decompress/DecompressProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.decompress; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -15,6 +17,9 @@ import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `decompress` processor decompresses any Base64-encoded " + + "compressed fields inside of an event.") public class DecompressProcessorConfig { @JsonPropertyDescription("The keys in the event that will be decompressed.") diff --git a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java index f934919e81..f1e7ce0686 100644 --- a/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java +++ b/data-prepper-plugins/dissect-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/dissect/DissectProcessorConfig.java @@ -1,12 +1,18 @@ package org.opensearch.dataprepper.plugins.processor.dissect; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.plugins.processor.mutateevent.TargetType; import java.util.Map; +@JsonPropertyOrder +@JsonClassDescription("The `dissect` processor extracts values from an event and maps them to individual fields " + + "based on user-defined `dissect` patterns. The processor is well suited for field extraction from log " + + "messages with a known structure.") public class DissectProcessorConfig { @NotNull @JsonProperty("map") diff --git a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java index b1383fbdf0..4f79c6575f 100644 --- a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java +++ b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java @@ -5,11 +5,15 @@ package org.opensearch.dataprepper.plugins.processor.drop; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import org.opensearch.dataprepper.model.event.HandleFailedEventsOption; +@JsonPropertyOrder +@JsonClassDescription("The `drop_events` processor drops all the events that are passed into it.") public class DropEventProcessorConfig { @JsonPropertyDescription("Accepts a Data Prepper conditional expression string following the [Data Prepper Expression Syntax](https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/). Configuring drop_events with drop_when: true drops all the events received.") diff --git a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java index 648936f2bf..f3fc6326ee 100644 --- a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java +++ b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java @@ -5,15 +5,19 @@ package org.opensearch.dataprepper.plugins.processor.flatten; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotNull; import java.util.ArrayList; import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `flatten` processor transforms nested objects inside of events into flattened structures.") public class FlattenProcessorConfig { private static final List DEFAULT_EXCLUDE_KEYS = new ArrayList<>(); diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorConfig.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorConfig.java index 23b351390a..2d48fd8fda 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorConfig.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessorConfig.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.plugins.geoip.processor; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.NotNull; @@ -16,6 +18,9 @@ /** * An implementation class of GeoIP Processor configuration */ +@JsonPropertyOrder +@JsonClassDescription("The `geoip` processor enriches events with geographic information extracted from IP addresses " + + "contained in the events.") public class GeoIPProcessorConfig { @Valid diff --git a/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfig.java b/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfig.java index 2d2ae1ef41..7fe746ad8e 100644 --- a/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfig.java +++ b/data-prepper-plugins/grok-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/grok/GrokProcessorConfig.java @@ -5,13 +5,18 @@ package org.opensearch.dataprepper.plugins.processor.grok; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import java.util.Collections; import java.util.List; import java.util.Map; +@JsonPropertyOrder +@JsonClassDescription("The `grok` processor uses pattern matching to structure and extract important keys from " + + "unstructured data.") public class GrokProcessorConfig { static final String TOTAL_PATTERNS_ATTEMPTED_METADATA_KEY = "_total_grok_patterns_attempted"; diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java index b3f283136a..34fd2c805b 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.keyvalue; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.AssertTrue; @@ -16,6 +18,8 @@ import java.util.List; import java.util.Map; +@JsonPropertyOrder +@JsonClassDescription("You can use the `key_value` processor to parse the specified field into key-value pairs.") public class KeyValueProcessorConfig { static final String DEFAULT_SOURCE = "message"; static final String DEFAULT_DESTINATION = "parsed_message"; diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java index df23740344..0c0f80be47 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; @@ -15,6 +17,8 @@ import java.util.List; import java.util.stream.Stream; +@JsonPropertyOrder +@JsonClassDescription("The `add_entries` processor adds entries to an event.") public class AddEntryProcessorConfig { public static class Entry { diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java index ce6aaed15c..b2b09ccfca 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java @@ -5,13 +5,18 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import org.opensearch.dataprepper.typeconverter.ConverterArguments; import java.util.List; import java.util.Optional; +@JsonPropertyOrder +@JsonClassDescription("The `convert_entry_type` processor converts a value type associated with the specified key in " + + "a event to the specified type. It is a casting processor that changes the types of some fields in events.") public class ConvertEntryTypeProcessorConfig implements ConverterArguments { @JsonProperty("key") @JsonPropertyDescription("Key whose value needs to be converted to a different type.") diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java index 1d24198d84..1b3afeed0c 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotEmpty; @@ -14,6 +16,9 @@ import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `copy_values` processor copies values within an event and is a [mutate event]" + + "(https://opensearch.org/docs/latest/data-prepper/pipelines/configuration/processors/mutate-event/) processor.") public class CopyValueProcessorConfig { public static class Entry { @NotEmpty diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java index b1df976770..2cfc3c5aba 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/DeleteEntryProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.model.event.EventKey; @@ -15,6 +17,10 @@ import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `delete_entries` processor deletes entries, such as key-value pairs, from an event. " + + "You can define the keys you want to delete in the `with-keys` field following `delete_entries` in the YAML " + + "configuration file. Those keys and their values are deleted.") public class DeleteEntryProcessorConfig { @NotEmpty @NotNull diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java index 13b6c2f0fb..7185576084 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ListToMapProcessorConfig.java @@ -5,9 +5,11 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -16,6 +18,9 @@ import java.util.Map; import java.util.stream.Collectors; +@JsonPropertyOrder +@JsonClassDescription("The `list_to_map` processor converts a list of objects from an event, " + + "where each object contains a `key` field, into a map of target keys.") public class ListToMapProcessorConfig { enum FlattenedElement { FIRST("first"), diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java index a402c27cdc..d914d987fc 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/MapToListProcessorConfig.java @@ -5,14 +5,19 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import java.util.ArrayList; import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `map_to_list` processor converts a map of key-value pairs to a list of objects. " + + "Each object contains the key and value in separate fields.") public class MapToListProcessorConfig { private static final String DEFAULT_KEY_NAME = "key"; private static final String DEFAULT_VALUE_NAME = "value"; diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java index 7449cc9968..d0067a330e 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/RenameKeyProcessorConfig.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.Valid; import jakarta.validation.constraints.NotEmpty; @@ -16,6 +18,8 @@ import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `rename_keys` processor renames keys in an event.") public class RenameKeyProcessorConfig { public static class Entry { @NotEmpty diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorConfig.java index a9a8ff9ce6..b0e6a98f26 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/SelectEntriesProcessorConfig.java @@ -5,13 +5,17 @@ package org.opensearch.dataprepper.plugins.processor.mutateevent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `select_entries` processor selects entries from a Data Prepper event.") public class SelectEntriesProcessorConfig { @NotEmpty @NotNull diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java index cb8edabfb6..c0a9a4ba70 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SplitStringProcessorConfig.java @@ -5,9 +5,11 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.Valid; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; @@ -16,6 +18,9 @@ import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `split_string` processor splits a field into an array using a delimiting character and is a " + + "[mutate string](https://github.com/opensearch-project/data-prepper/tree/main/data-prepper-plugins/mutate-string-processors#mutate-string-processors) processor.") public class SplitStringProcessorConfig implements StringProcessorConfig { public static class Entry { diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java index 722a42dcbe..f7b008f015 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/SubstituteStringProcessorConfig.java @@ -5,12 +5,17 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import org.opensearch.dataprepper.model.event.EventKey; import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `substitute_string` processor matches a key’s value against a regular expression and " + + "replaces all matches with a replacement string.") public class SubstituteStringProcessorConfig implements StringProcessorConfig { public static class Entry { @JsonPropertyDescription("The key to modify.") diff --git a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java index 3660b5d73d..a8087954d1 100644 --- a/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java +++ b/data-prepper-plugins/mutate-string-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutatestring/WithKeysConfig.java @@ -5,14 +5,19 @@ package org.opensearch.dataprepper.plugins.processor.mutatestring; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.model.event.EventKey; import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("This processor is a [mutate string]" + + "(https://github.com/opensearch-project/data-prepper/tree/main/data-prepper-plugins/mutate-string-processors#mutate-string-processors) processor.") public class WithKeysConfig implements StringProcessorConfig { @NotNull diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java index 8a6f8e17e5..fa4df27ce1 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.obfuscation; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.expression.ExpressionEvaluator; @@ -15,6 +17,9 @@ import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `obfuscate` process enables obfuscation of fields inside your documents in order to " + + "protect sensitive data.") public class ObfuscationProcessorConfig { @JsonProperty("source") diff --git a/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OtelMetricsRawProcessorConfig.java b/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OtelMetricsRawProcessorConfig.java index b71a0d1800..496df187e6 100644 --- a/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OtelMetricsRawProcessorConfig.java +++ b/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OtelMetricsRawProcessorConfig.java @@ -7,9 +7,14 @@ import static org.opensearch.dataprepper.plugins.otel.codec.OTelProtoCodec.DEFAULT_EXPONENTIAL_HISTOGRAM_MAX_ALLOWED_SCALE; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; +@JsonPropertyOrder +@JsonClassDescription("The `otel_metrics` processor serializes a collection of `ExportMetricsServiceRequest` records " + + "sent from the [OTel metrics source](https://opensearch.org/docs/latest/data-prepper/pipelines/configuration/sources/otel-metrics-source/) into a collection of string records.") public class OtelMetricsRawProcessorConfig { @JsonProperty("flatten_attributes") diff --git a/data-prepper-plugins/otel-trace-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OtelTraceRawProcessorConfig.java b/data-prepper-plugins/otel-trace-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OtelTraceRawProcessorConfig.java index 6b850f7354..16b3017a81 100644 --- a/data-prepper-plugins/otel-trace-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OtelTraceRawProcessorConfig.java +++ b/data-prepper-plugins/otel-trace-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OtelTraceRawProcessorConfig.java @@ -5,11 +5,16 @@ package org.opensearch.dataprepper.plugins.processor.oteltrace; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import java.time.Duration; +@JsonPropertyOrder +@JsonClassDescription("The `otel_trace` processor completes trace-group-related fields in all incoming Data Prepper " + + "span records by state caching the root span information for each `traceId`.") public class OtelTraceRawProcessorConfig { static final long DEFAULT_TG_FLUSH_INTERVAL_SEC = 180L; static final Duration DEFAULT_TRACE_ID_TTL = Duration.ofSeconds(15L); diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java index 2a8484131e..5e2468e7b9 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/ion/ParseIonProcessorConfig.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.plugins.processor.parse.ion; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotBlank; @@ -16,6 +18,8 @@ import java.util.List; import java.util.Objects; +@JsonPropertyOrder +@JsonClassDescription("The `parse_ion` processor parses [Amazon Ion](https://amazon-ion.github.io/ion-docs/) data.") public class ParseIonProcessorConfig implements CommonParseConfig { static final String DEFAULT_SOURCE = "message"; diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java index 13a6e2e24a..add5ba259a 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/json/ParseJsonProcessorConfig.java @@ -5,7 +5,9 @@ package org.opensearch.dataprepper.plugins.processor.parse.json; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotBlank; @@ -16,6 +18,9 @@ import java.util.Objects; import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `parse_json` processor parses JSON data for an event, including any nested fields. " + + "The processor extracts the JSON pointer data and adds the input event to the extracted fields.") public class ParseJsonProcessorConfig implements CommonParseConfig { static final String DEFAULT_SOURCE = "message"; diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java index 82d19a2098..92cd9f553a 100644 --- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java +++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/xml/ParseXmlProcessorConfig.java @@ -1,6 +1,8 @@ package org.opensearch.dataprepper.plugins.processor.parse.xml; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotBlank; @@ -11,6 +13,8 @@ import java.util.List; import java.util.Objects; +@JsonPropertyOrder +@JsonClassDescription("The `parse_xml` processor parses XML data for an event.") public class ParseXmlProcessorConfig implements CommonParseConfig { static final String DEFAULT_SOURCE = "message"; diff --git a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfig.java b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfig.java index 7f72fb5286..faf98b2133 100644 --- a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfig.java +++ b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapProcessorConfig.java @@ -5,9 +5,14 @@ package org.opensearch.dataprepper.plugins.processor; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; +@JsonPropertyOrder +@JsonClassDescription("The `service_map` processor uses OpenTelemetry data to create a distributed service map for " + + "visualization in OpenSearch Dashboards.") public class ServiceMapProcessorConfig { private static final String WINDOW_DURATION = "window_duration"; static final int DEFAULT_WINDOW_DURATION = 180; diff --git a/data-prepper-plugins/split-event-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorConfig.java b/data-prepper-plugins/split-event-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorConfig.java index db70e3c6db..140e280710 100644 --- a/data-prepper-plugins/split-event-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorConfig.java +++ b/data-prepper-plugins/split-event-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/splitevent/SplitEventProcessorConfig.java @@ -10,13 +10,17 @@ package org.opensearch.dataprepper.plugins.processor.splitevent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; -import jakarta.validation.constraints.Size; - +import jakarta.validation.constraints.Size; +@JsonPropertyOrder +@JsonClassDescription("The `split_event` processor is used to split events based on a delimiter and " + + "generates multiple events from a user-specified field.") public class SplitEventProcessorConfig { @NotEmpty @NotNull diff --git a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java index de0949a31c..b4a9df2f85 100644 --- a/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java +++ b/data-prepper-plugins/translate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/translate/TranslateProcessorConfig.java @@ -6,9 +6,11 @@ package org.opensearch.dataprepper.plugins.processor.translate; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.Valid; import jakarta.validation.constraints.AssertTrue; @@ -16,7 +18,8 @@ import java.util.List; import java.util.Objects; - +@JsonPropertyOrder +@JsonClassDescription("The `translate` processor transforms values in events into preconfigured values.") public class TranslateProcessorConfig { @JsonProperty("file") diff --git a/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java b/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java index 0d525cddd7..ce713d061e 100644 --- a/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java +++ b/data-prepper-plugins/truncate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/truncate/TruncateProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.truncate; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import jakarta.validation.constraints.AssertTrue; @@ -14,6 +16,9 @@ import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `truncate` processor truncates a key’s value at the beginning, the end, " + + "or on both sides of the value string, based on the processor’s configuration.") public class TruncateProcessorConfig { public static class Entry { @JsonProperty("source_keys") diff --git a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java index b1660b37d0..df343d9b0a 100644 --- a/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java +++ b/data-prepper-plugins/user-agent-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/useragent/UserAgentProcessorConfig.java @@ -5,8 +5,10 @@ package org.opensearch.dataprepper.plugins.processor.useragent; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import jakarta.validation.constraints.NotEmpty; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.model.event.EventKey; @@ -15,6 +17,9 @@ import java.util.List; +@JsonPropertyOrder +@JsonClassDescription("The `user_agent` processor parses any user agent (UA) string in an event and then adds the " + + "parsing results to the event’s write data.") public class UserAgentProcessorConfig { private static final int DEFAULT_CACHE_SIZE = 1000; diff --git a/data-prepper-plugins/write-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/write_json/WriteJsonProcessorConfig.java b/data-prepper-plugins/write-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/write_json/WriteJsonProcessorConfig.java index a36bc41ad5..f93e53bc24 100644 --- a/data-prepper-plugins/write-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/write_json/WriteJsonProcessorConfig.java +++ b/data-prepper-plugins/write-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/write_json/WriteJsonProcessorConfig.java @@ -5,10 +5,14 @@ package org.opensearch.dataprepper.plugins.processor.write_json; +import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.NotNull; +@JsonPropertyOrder +@JsonClassDescription("The `write_json` processor converts an object in an event into a JSON string.") public class WriteJsonProcessorConfig { @JsonProperty("source") @JsonPropertyDescription("Specifies the name of the field in the event containing the message or object to be parsed.") From 0387808822f9a7da7e3ac04fc503b495e9d2689c Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Wed, 21 Aug 2024 17:41:09 -0500 Subject: [PATCH 126/159] ENH: collect plugin config and loading errors during data prepper launch (#4816) * ENH: collect plugin errors within core application context Signed-off-by: George Chen --- .../configuration/PipelinesDataFlowModel.java | 3 +- .../plugin/DefaultPluginFactoryIT.java | 2 + .../dataprepper/plugins/ExtensionsIT.java | 4 + .../parser/PipelineTransformer.java | 106 ++++++++++++++---- .../config/PipelineParserConfiguration.java | 9 +- .../dataprepper/TestDataProvider.java | 3 +- .../parser/PipelineTransformerTests.java | 31 ++++- .../PipelineParserConfigurationTest.java | 7 +- ...hild_pipeline_due_to_invalid_processor.yml | 26 +++++ ...ct_child_pipeline_due_to_invalid_sink.yml} | 0 .../dataprepper/validation/PluginError.java | 47 ++++++++ .../validation/PluginErrorCollector.java | 31 +++++ .../validation/DataPrepperValidateTest.java | 3 + .../validation/PluginErrorCollectorTest.java | 30 +++++ .../validation/PluginErrorTest.java | 58 ++++++++++ .../dataprepper/plugin/ExtensionLoader.java | 37 +++++- .../plugin/ExtensionLoaderTest.java | 63 ++++++++++- 17 files changed, 420 insertions(+), 40 deletions(-) create mode 100644 data-prepper-core/src/test/resources/connected_pipeline_incorrect_child_pipeline_due_to_invalid_processor.yml rename data-prepper-core/src/test/resources/{connected_pipeline_incorrect_child_pipeline.yml => connected_pipeline_incorrect_child_pipeline_due_to_invalid_sink.yml} (100%) create mode 100644 data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginError.java create mode 100644 data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorCollector.java create mode 100644 data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/DataPrepperValidateTest.java create mode 100644 data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorCollectorTest.java create mode 100644 data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorTest.java diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelinesDataFlowModel.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelinesDataFlowModel.java index 12b092ac36..7eea063bd0 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelinesDataFlowModel.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/PipelinesDataFlowModel.java @@ -27,12 +27,13 @@ * @since 1.2 */ public class PipelinesDataFlowModel { + public static final String EXTENSION_PLUGIN_TYPE = "extension"; @JsonInclude(JsonInclude.Include.NON_NULL) private DataPrepperVersion version; @JsonAlias("pipeline_configurations") - @JsonProperty("extension") + @JsonProperty(EXTENSION_PLUGIN_TYPE) @JsonInclude(JsonInclude.Include.NON_NULL) @JsonSetter(nulls = Nulls.SKIP) private PipelineExtensions pipelineExtensions; diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java index ea0654563a..89ab07d11d 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java @@ -17,6 +17,7 @@ import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.plugins.TestObjectPlugin; import org.opensearch.dataprepper.plugins.test.TestPlugin; +import org.opensearch.dataprepper.validation.PluginErrorCollector; import org.springframework.context.annotation.AnnotationConfigApplicationContext; import java.util.HashMap; @@ -61,6 +62,7 @@ private DefaultPluginFactory createObjectUnderTest() { coreContext.scan(DefaultAcknowledgementSetManager.class.getPackage().getName()); coreContext.scan(DefaultPluginFactory.class.getPackage().getName()); coreContext.register(PluginBeanFactoryProvider.class); + coreContext.registerBean(PluginErrorCollector.class, PluginErrorCollector::new); coreContext.registerBean(ExtensionsConfiguration.class, () -> extensionsConfiguration); coreContext.registerBean(PipelinesDataFlowModel.class, () -> pipelinesDataFlowModel); coreContext.refresh(); diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/ExtensionsIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/ExtensionsIT.java index 176e11c4b2..1c4dd2f967 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/ExtensionsIT.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/ExtensionsIT.java @@ -31,6 +31,7 @@ import org.opensearch.dataprepper.plugin.TestPluggableInterface; import org.opensearch.dataprepper.plugins.test.TestExtension; import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; +import org.opensearch.dataprepper.validation.PluginErrorCollector; import org.springframework.context.annotation.AnnotationConfigApplicationContext; import java.util.ArrayList; @@ -68,6 +69,7 @@ public class ExtensionsIT { private AnnotationConfigApplicationContext publicContext; private AnnotationConfigApplicationContext coreContext; private PluginFactory pluginFactory; + private PluginErrorCollector pluginErrorCollector; private String pluginName; private String pipelineName; @@ -75,6 +77,7 @@ public class ExtensionsIT { void setUp() { pluginName = "test_plugin_using_extension"; pipelineName = UUID.randomUUID().toString(); + pluginErrorCollector = new PluginErrorCollector(); publicContext = new AnnotationConfigApplicationContext(); publicContext.refresh(); @@ -104,6 +107,7 @@ void setUp() { coreContext.registerBean(ObjectMapperConfiguration.class, ObjectMapperConfiguration::new); coreContext.registerBean(ObjectMapper.class, () -> new ObjectMapper(new YAMLFactory())); coreContext.register(PipelineParserConfiguration.class); + coreContext.registerBean(PluginErrorCollector.class, () -> pluginErrorCollector); coreContext.refresh(); pluginFactory = coreContext.getBean(DefaultPluginFactory.class); diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/PipelineTransformer.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/PipelineTransformer.java index 0f96717c6c..a01103c03a 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/PipelineTransformer.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/PipelineTransformer.java @@ -8,9 +8,11 @@ import org.opensearch.dataprepper.breaker.CircuitBreakerManager; import org.opensearch.dataprepper.model.annotations.SingleThread; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.configuration.PipelineModel; import org.opensearch.dataprepper.model.configuration.PipelinesDataFlowModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.peerforwarder.RequiresPeerForwarding; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.processor.Processor; import org.opensearch.dataprepper.model.sink.Sink; @@ -30,10 +32,13 @@ import org.opensearch.dataprepper.pipeline.router.Router; import org.opensearch.dataprepper.pipeline.router.RouterFactory; import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; +import org.opensearch.dataprepper.validation.PluginError; +import org.opensearch.dataprepper.validation.PluginErrorCollector; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.time.Duration; +import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -59,6 +64,7 @@ public class PipelineTransformer { private final EventFactory eventFactory; private final AcknowledgementSetManager acknowledgementSetManager; private final SourceCoordinatorFactory sourceCoordinatorFactory; + private final PluginErrorCollector pluginErrorCollector; public PipelineTransformer(final PipelinesDataFlowModel pipelinesDataFlowModel, final PluginFactory pluginFactory, @@ -68,7 +74,8 @@ public PipelineTransformer(final PipelinesDataFlowModel pipelinesDataFlowModel, final CircuitBreakerManager circuitBreakerManager, final EventFactory eventFactory, final AcknowledgementSetManager acknowledgementSetManager, - final SourceCoordinatorFactory sourceCoordinatorFactory) { + final SourceCoordinatorFactory sourceCoordinatorFactory, + final PluginErrorCollector pluginErrorCollector) { this.pipelinesDataFlowModel = pipelinesDataFlowModel; this.pluginFactory = Objects.requireNonNull(pluginFactory); this.peerForwarderProvider = Objects.requireNonNull(peerForwarderProvider); @@ -78,6 +85,7 @@ public PipelineTransformer(final PipelinesDataFlowModel pipelinesDataFlowModel, this.eventFactory = eventFactory; this.acknowledgementSetManager = acknowledgementSetManager; this.sourceCoordinatorFactory = sourceCoordinatorFactory; + this.pluginErrorCollector = pluginErrorCollector; } public Map transformConfiguration() { @@ -112,11 +120,34 @@ private void buildPipelineFromConfiguration( final PluginSetting sourceSetting = pipelineConfiguration.getSourcePluginSetting(); final Optional pipelineSource = getSourceIfPipelineType(pipelineName, sourceSetting, pipelineMap, pipelineConfigurationMap); - final Source source = pipelineSource.orElseGet(() -> - pluginFactory.loadPlugin(Source.class, sourceSetting)); + final Source source = pipelineSource.orElseGet(() -> { + try { + return pluginFactory.loadPlugin(Source.class, sourceSetting); + } catch (Exception e) { + final PluginError pluginError = PluginError.builder() + .componentType(PipelineModel.SOURCE_PLUGIN_TYPE) + .pipelineName(pipelineName) + .pluginName(sourceSetting.getName()) + .exception(e) + .build(); + pluginErrorCollector.collectPluginError(pluginError); + return null; + } + }); LOG.info("Building buffer for the pipeline [{}]", pipelineName); - final Buffer pipelineDefinedBuffer = pluginFactory.loadPlugin(Buffer.class, pipelineConfiguration.getBufferPluginSetting(), source.getDecoder()); + Buffer pipelineDefinedBuffer = null; + final PluginSetting bufferPluginSetting = pipelineConfiguration.getBufferPluginSetting(); + try { + pipelineDefinedBuffer = pluginFactory.loadPlugin(Buffer.class, bufferPluginSetting, source.getDecoder()); + } catch (Exception e) { + final PluginError pluginError = PluginError.builder() + .componentType(PipelineModel.BUFFER_PLUGIN_TYPE) + .pipelineName(pipelineName) + .pluginName(bufferPluginSetting.getName()) + .build(); + pluginErrorCollector.collectPluginError(pluginError); + } LOG.info("Building processors for the pipeline [{}]", pipelineName); final int processorThreads = pipelineConfiguration.getWorkers(); @@ -125,6 +156,20 @@ private void buildPipelineFromConfiguration( .map(this::newProcessor) .collect(Collectors.toList()); + LOG.info("Building sinks for the pipeline [{}]", pipelineName); + final List> sinks = pipelineConfiguration.getSinkPluginSettings().stream() + .map(this::buildRoutedSinkOrConnector) + .collect(Collectors.toList()); + + final List subPipelinePluginErrors = pluginErrorCollector.getPluginErrors() + .stream().filter(pluginError -> pipelineName.equals(pluginError.getPipelineName())) + .collect(Collectors.toList()); + if (!subPipelinePluginErrors.isEmpty()) { + throw new InvalidPluginConfigurationException( + String.format("One or more plugins are not configured correctly in the pipeline: %s.\n", + pipelineName) + pluginErrorCollector.getConsolidatedErrorMessage()); + } + final List> decoratedProcessorSets = processorSets.stream() .map(processorComponentList -> { final List processors = processorComponentList.stream().map(IdentifiedComponent::getComponent).collect(Collectors.toList()); @@ -138,11 +183,6 @@ private void buildPipelineFromConfiguration( final int readBatchDelay = pipelineConfiguration.getReadBatchDelay(); - LOG.info("Building sinks for the pipeline [{}]", pipelineName); - final List> sinks = pipelineConfiguration.getSinkPluginSettings().stream() - .map(this::buildRoutedSinkOrConnector) - .collect(Collectors.toList()); - final List secondaryBuffers = getSecondaryBuffers(); LOG.info("Constructing MultiBufferDecorator with [{}] secondary buffers for pipeline [{}]", secondaryBuffers.size(), pipelineName); final MultiBufferDecorator multiBufferDecorator = new MultiBufferDecorator(pipelineDefinedBuffer, secondaryBuffers); @@ -167,16 +207,27 @@ private void buildPipelineFromConfiguration( } private List> newProcessor(final PluginSetting pluginSetting) { - final List processors = pluginFactory.loadPlugins( - Processor.class, - pluginSetting, - actualClass -> actualClass.isAnnotationPresent(SingleThread.class) ? - pluginSetting.getNumberOfProcessWorkers() : - 1); - - return processors.stream() - .map(processor -> new IdentifiedComponent<>(processor, pluginSetting.getName())) - .collect(Collectors.toList()); + try { + final List processors = pluginFactory.loadPlugins( + Processor.class, + pluginSetting, + actualClass -> actualClass.isAnnotationPresent(SingleThread.class) ? + pluginSetting.getNumberOfProcessWorkers() : + 1); + + return processors.stream() + .map(processor -> new IdentifiedComponent<>(processor, pluginSetting.getName())) + .collect(Collectors.toList()); + } catch (Exception e) { + final PluginError pluginError = PluginError.builder() + .componentType(PipelineModel.PROCESSOR_PLUGIN_TYPE) + .pipelineName(pluginSetting.getPipelineName()) + .pluginName(pluginSetting.getName()) + .exception(e) + .build(); + pluginErrorCollector.collectPluginError(pluginError); + return Collections.emptyList(); + } } private Optional getSourceIfPipelineType( @@ -213,9 +264,20 @@ private Optional getSourceIfPipelineType( } private DataFlowComponent buildRoutedSinkOrConnector(final SinkContextPluginSetting pluginSetting) { - final Sink sink = buildSinkOrConnector(pluginSetting, pluginSetting.getSinkContext()); - - return new DataFlowComponent<>(sink, pluginSetting.getSinkContext().getRoutes()); + try { + final Sink sink = buildSinkOrConnector(pluginSetting, pluginSetting.getSinkContext()); + + return new DataFlowComponent<>(sink, pluginSetting.getSinkContext().getRoutes()); + } catch (Exception e) { + final PluginError pluginError = PluginError.builder() + .componentType(PipelineModel.SINK_PLUGIN_TYPE) + .pipelineName(pluginSetting.getPipelineName()) + .pluginName(pluginSetting.getName()) + .exception(e) + .build(); + pluginErrorCollector.collectPluginError(pluginError); + return null; + } } private Sink buildSinkOrConnector(final PluginSetting pluginSetting, final SinkContext sinkContext) { diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/config/PipelineParserConfiguration.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/config/PipelineParserConfiguration.java index f4f586c5b6..3fbeedf5af 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/config/PipelineParserConfiguration.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/config/PipelineParserConfiguration.java @@ -21,6 +21,7 @@ import org.opensearch.dataprepper.pipeline.parser.transformer.PipelineConfigurationTransformer; import org.opensearch.dataprepper.pipeline.router.RouterFactory; import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; +import org.opensearch.dataprepper.validation.PluginErrorCollector; import org.springframework.beans.factory.annotation.Qualifier; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.ComponentScan; @@ -44,8 +45,9 @@ public PipelineTransformer pipelineParser( final CircuitBreakerManager circuitBreakerManager, final EventFactory eventFactory, final AcknowledgementSetManager acknowledgementSetManager, - final SourceCoordinatorFactory sourceCoordinatorFactory - ) { + final SourceCoordinatorFactory sourceCoordinatorFactory, + final PluginErrorCollector pluginErrorCollector + ) { return new PipelineTransformer(pipelinesDataFlowModel, pluginFactory, peerForwarderProvider, @@ -54,7 +56,8 @@ public PipelineTransformer pipelineParser( circuitBreakerManager, eventFactory, acknowledgementSetManager, - sourceCoordinatorFactory); + sourceCoordinatorFactory, + pluginErrorCollector); } @Bean diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/TestDataProvider.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/TestDataProvider.java index b415b7cf6b..aa72bbf207 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/TestDataProvider.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/TestDataProvider.java @@ -16,7 +16,8 @@ public class TestDataProvider { public static final String VALID_OFF_HEAP_FILE_WITH_ACKS = "src/test/resources/multiple_pipeline_valid_off_heap_buffer_with_acks.yml"; public static final String DISCONNECTED_VALID_OFF_HEAP_FILE_WITH_ACKS = "src/test/resources/multiple_disconnected_pipeline_valid_off_heap_buffer_with_acks.yml"; public static final String CONNECTED_PIPELINE_ROOT_SOURCE_INCORRECT = "src/test/resources/connected_pipeline_incorrect_root_source.yml"; - public static final String CONNECTED_PIPELINE_CHILD_PIPELINE_INCORRECT = "src/test/resources/connected_pipeline_incorrect_child_pipeline.yml"; + public static final String CONNECTED_PIPELINE_CHILD_PIPELINE_INCORRECT_DUE_TO_SINK = "src/test/resources/connected_pipeline_incorrect_child_pipeline_due_to_invalid_sink.yml"; + public static final String CONNECTED_PIPELINE_CHILD_PIPELINE_INCORRECT_DUE_TO_PROCESSOR = "src/test/resources/connected_pipeline_incorrect_child_pipeline_due_to_invalid_processor.yml"; public static final String CYCLE_MULTIPLE_PIPELINE_CONFIG_FILE = "src/test/resources/cyclic_multiple_pipeline_configuration.yml"; public static final String INCORRECT_SOURCE_MULTIPLE_PIPELINE_CONFIG_FILE = "src/test/resources/incorrect_source_multiple_pipeline_configuration.yml"; public static final String MISSING_NAME_MULTIPLE_PIPELINE_CONFIG_FILE = "src/test/resources/missing_name_multiple_pipeline_configuration.yml"; diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/PipelineTransformerTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/PipelineTransformerTests.java index 07817f8ee0..112bb8e93f 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/PipelineTransformerTests.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/PipelineTransformerTests.java @@ -13,6 +13,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.TestDataProvider; @@ -36,6 +37,8 @@ import org.opensearch.dataprepper.pipeline.router.RouterFactory; import org.opensearch.dataprepper.plugin.DefaultPluginFactory; import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; +import org.opensearch.dataprepper.validation.PluginError; +import org.opensearch.dataprepper.validation.PluginErrorCollector; import org.springframework.context.annotation.AnnotationConfigApplicationContext; import java.time.Duration; @@ -82,6 +85,8 @@ class PipelineTransformerTests { @Mock private CircuitBreakerManager circuitBreakerManager; + private PluginErrorCollector pluginErrorCollector; + private EventFactory eventFactory; private DefaultAcknowledgementSetManager acknowledgementSetManager; @@ -93,6 +98,7 @@ void setUp() { peerForwarderProvider = mock(PeerForwarderProvider.class); eventFactory = mock(EventFactory.class); acknowledgementSetManager = mock(DefaultAcknowledgementSetManager.class); + pluginErrorCollector = new PluginErrorCollector(); final AnnotationConfigApplicationContext publicContext = new AnnotationConfigApplicationContext(); publicContext.refresh(); @@ -104,6 +110,7 @@ void setUp() { coreContext.scan(DefaultAcknowledgementSetManager.class.getPackage().getName()); coreContext.scan(DefaultPluginFactory.class.getPackage().getName()); + coreContext.registerBean(PluginErrorCollector.class, () -> pluginErrorCollector); coreContext.registerBean(DataPrepperConfiguration.class, () -> dataPrepperConfiguration); coreContext.registerBean(PipelinesDataFlowModel.class, () -> pipelinesDataFlowModel); coreContext.refresh(); @@ -121,7 +128,7 @@ private PipelineTransformer createObjectUnderTest(final String pipelineConfigura new PipelineConfigurationFileReader(pipelineConfigurationFileLocation)).parseConfiguration(); return new PipelineTransformer(pipelinesDataFlowModel, pluginFactory, peerForwarderProvider, routerFactory, dataPrepperConfiguration, circuitBreakerManager, eventFactory, - acknowledgementSetManager, sourceCoordinatorFactory); + acknowledgementSetManager, sourceCoordinatorFactory, pluginErrorCollector); } @Test @@ -193,17 +200,31 @@ void parseConfiguration_with_invalid_root_pipeline_creates_empty_pipelinesMap() final Map connectedPipelines = pipelineTransformer.transformConfiguration(); assertThat(connectedPipelines.size(), equalTo(0)); verify(dataPrepperConfiguration).getPipelineExtensions(); + assertThat(pluginErrorCollector.getPluginErrors().size(), equalTo(1)); + final PluginError pluginError = pluginErrorCollector.getPluginErrors().get(0); + assertThat(pluginError.getPipelineName(), equalTo("test-pipeline-1")); + assertThat(pluginError.getPluginName(), equalTo("file")); + assertThat(pluginError.getException(), notNullValue()); } - @Test - void parseConfiguration_with_incorrect_child_pipeline_returns_empty_pipelinesMap() { + @ParameterizedTest + @ValueSource(strings = { + TestDataProvider.CONNECTED_PIPELINE_CHILD_PIPELINE_INCORRECT_DUE_TO_SINK, + TestDataProvider.CONNECTED_PIPELINE_CHILD_PIPELINE_INCORRECT_DUE_TO_PROCESSOR + }) + void parseConfiguration_with_incorrect_child_pipeline_returns_empty_pipelinesMap( + final String pipelineConfigurationFileLocation) { mockDataPrepperConfigurationAccesses(); - final PipelineTransformer pipelineTransformer = - createObjectUnderTest(TestDataProvider.CONNECTED_PIPELINE_CHILD_PIPELINE_INCORRECT); + final PipelineTransformer pipelineTransformer = createObjectUnderTest(pipelineConfigurationFileLocation); final Map connectedPipelines = pipelineTransformer.transformConfiguration(); assertThat(connectedPipelines.size(), equalTo(0)); verifyDataPrepperConfigurationAccesses(); verify(dataPrepperConfiguration).getPipelineExtensions(); + assertThat(pluginErrorCollector.getPluginErrors().size(), equalTo(1)); + final PluginError pluginError = pluginErrorCollector.getPluginErrors().get(0); + assertThat(pluginError.getPipelineName(), equalTo("test-pipeline-2")); + assertThat(pluginError.getPluginName(), notNullValue()); + assertThat(pluginError.getException(), notNullValue()); } @Test diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/config/PipelineParserConfigurationTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/config/PipelineParserConfigurationTest.java index 3037b6a68d..9b027c6c75 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/config/PipelineParserConfigurationTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/config/PipelineParserConfigurationTest.java @@ -19,6 +19,7 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; +import org.opensearch.dataprepper.validation.PluginErrorCollector; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; @@ -55,11 +56,15 @@ class PipelineParserConfigurationTest { @Mock private AcknowledgementSetManager acknowledgementSetManager; + @Mock + private PluginErrorCollector pluginErrorCollector; + @Test void pipelineParser() { final PipelineTransformer pipelineTransformer = pipelineParserConfiguration.pipelineParser( pipelinesDataFlowModel, pluginFactory, peerForwarderProvider, routerFactory, - dataPrepperConfiguration, circuitBreakerManager, eventFactory, acknowledgementSetManager, sourceCoordinatorFactory); + dataPrepperConfiguration, circuitBreakerManager, eventFactory, acknowledgementSetManager, + sourceCoordinatorFactory, pluginErrorCollector); assertThat(pipelineTransformer, is(notNullValue())); } diff --git a/data-prepper-core/src/test/resources/connected_pipeline_incorrect_child_pipeline_due_to_invalid_processor.yml b/data-prepper-core/src/test/resources/connected_pipeline_incorrect_child_pipeline_due_to_invalid_processor.yml new file mode 100644 index 0000000000..95a821042f --- /dev/null +++ b/data-prepper-core/src/test/resources/connected_pipeline_incorrect_child_pipeline_due_to_invalid_processor.yml @@ -0,0 +1,26 @@ +# this configuration file is solely for testing formatting +test-pipeline-1: + source: + file: + path: "/tmp/file-source.tmp" + buffer: + bounded_blocking: + sink: + - pipeline: + name: "test-pipeline-2" +test-pipeline-2: + source: + pipeline: + name: "test-pipeline-1" + processor: + - invalid_processor: # this will fail plugin creation + sink: + - pipeline: + name: "test-pipeline-3" +test-pipeline-3: + source: + pipeline: + name: "test-pipeline-2" + sink: + - file: + path: "/tmp/todelete.txt" \ No newline at end of file diff --git a/data-prepper-core/src/test/resources/connected_pipeline_incorrect_child_pipeline.yml b/data-prepper-core/src/test/resources/connected_pipeline_incorrect_child_pipeline_due_to_invalid_sink.yml similarity index 100% rename from data-prepper-core/src/test/resources/connected_pipeline_incorrect_child_pipeline.yml rename to data-prepper-core/src/test/resources/connected_pipeline_incorrect_child_pipeline_due_to_invalid_sink.yml diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginError.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginError.java new file mode 100644 index 0000000000..50a130c4f4 --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginError.java @@ -0,0 +1,47 @@ +package org.opensearch.dataprepper.validation; + +import lombok.Builder; +import lombok.Getter; +import lombok.NonNull; + +@Getter +@Builder +public class PluginError { + static final String PIPELINE_DELIMITER = ":"; + static final String CAUSED_BY_DELIMITER = " caused by: "; + private final String pipelineName; + private final String componentType; + @NonNull + private final String pluginName; + @NonNull + private final Exception exception; + + public String getErrorMessage() { + final StringBuilder message = new StringBuilder(); + if (pipelineName != null) { + message.append(pipelineName); + message.append(PIPELINE_DELIMITER); + } + if (componentType != null) { + message.append(componentType); + message.append(PIPELINE_DELIMITER); + } + message.append(pluginName); + message.append(PIPELINE_DELIMITER); + message.append(getFlattenedExceptionMessage(CAUSED_BY_DELIMITER)); + return message.toString(); + } + + private String getFlattenedExceptionMessage(final String delimiter) { + final StringBuilder message = new StringBuilder(); + Throwable throwable = exception; + + while (throwable != null) { + message.append(delimiter); + message.append(throwable.getMessage()); + throwable = throwable.getCause(); + } + + return message.toString(); + } +} diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorCollector.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorCollector.java new file mode 100644 index 0000000000..0be5539019 --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorCollector.java @@ -0,0 +1,31 @@ +package org.opensearch.dataprepper.validation; + +import lombok.Getter; + +import javax.inject.Named; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +@Named +@Getter +public class PluginErrorCollector { + private final List pluginErrors = new ArrayList<>(); + + public void collectPluginError(final PluginError pluginError) { + pluginErrors.add(pluginError); + } + + public List getAllErrorMessages() { + return pluginErrors.stream().map(PluginError::getErrorMessage).collect(Collectors.toList()); + } + + public String getConsolidatedErrorMessage() { + final List allErrorMessages = getAllErrorMessages(); + + return IntStream.range(0, allErrorMessages.size()) + .mapToObj(i -> (i + 1) + ". " + allErrorMessages.get(i)) + .collect(Collectors.joining("\n")); + } +} diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/DataPrepperValidateTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/DataPrepperValidateTest.java new file mode 100644 index 0000000000..447fb21838 --- /dev/null +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/DataPrepperValidateTest.java @@ -0,0 +1,3 @@ +package org.opensearch.dataprepper.validation; +class DataPrepperValidateTest { +} \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorCollectorTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorCollectorTest.java new file mode 100644 index 0000000000..324a202fb8 --- /dev/null +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorCollectorTest.java @@ -0,0 +1,30 @@ +package org.opensearch.dataprepper.validation; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class PluginErrorCollectorTest { + + @Test + void testWithPluginErrors() { + final PluginErrorCollector objectUnderTest = new PluginErrorCollector(); + final String testErrorMessage1 = "test error message 1"; + final String testErrorMessage2 = "test error message 2"; + final PluginError pluginError1 = mock(PluginError.class); + when(pluginError1.getErrorMessage()).thenReturn(testErrorMessage1); + final PluginError pluginError2 = mock(PluginError.class); + when(pluginError2.getErrorMessage()).thenReturn(testErrorMessage2); + objectUnderTest.collectPluginError(pluginError1); + objectUnderTest.collectPluginError(pluginError2); + assertThat(objectUnderTest.getPluginErrors().size(), equalTo(2)); + assertThat(objectUnderTest.getAllErrorMessages().size(), equalTo(2)); + assertThat(objectUnderTest.getAllErrorMessages(), contains(testErrorMessage1, testErrorMessage2)); + assertThat(objectUnderTest.getConsolidatedErrorMessage(), equalTo( + String.format("1. %s\n2. %s", testErrorMessage1, testErrorMessage2))); + } +} \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorTest.java new file mode 100644 index 0000000000..c02dde376c --- /dev/null +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorTest.java @@ -0,0 +1,58 @@ +package org.opensearch.dataprepper.validation; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +class PluginErrorTest { + private static final String TEST_PIPELINE_NAME = "test-pipeline"; + private static final String TEST_COMPONENT_TYPE = "test-plugin-type"; + private static final String TEST_PLUGIN_NAME = "test-plugin"; + private static final String TEST_ERROR_MESSAGE = "test error message"; + private static final String TEST_CAUSE_ERROR_MESSAGE = "test cause error message"; + + @Test + void testGetErrorMessageWithPipelineName() { + final Exception exception = mock(Exception.class); + when(exception.getMessage()).thenReturn(TEST_ERROR_MESSAGE); + final PluginError pluginError = PluginError.builder() + .pipelineName(TEST_PIPELINE_NAME) + .componentType(TEST_COMPONENT_TYPE) + .pluginName(TEST_PLUGIN_NAME) + .exception(exception) + .build(); + assertThat(pluginError.getErrorMessage(), equalTo( + "test-pipeline:test-plugin-type:test-plugin: caused by: test error message")); + } + + @Test + void testGetErrorMessageWithoutPipelineName() { + final Exception exception = mock(Exception.class); + when(exception.getMessage()).thenReturn(TEST_ERROR_MESSAGE); + final PluginError pluginError = PluginError.builder() + .pluginName(TEST_PLUGIN_NAME) + .componentType(TEST_COMPONENT_TYPE) + .exception(exception) + .build(); + assertThat(pluginError.getErrorMessage(), equalTo( + "test-plugin-type:test-plugin: caused by: test error message")); + } + + @Test + void testGetErrorMessageWithCause() { + final Exception exception = mock(Exception.class); + final Exception cause = mock(Exception.class); + when(exception.getMessage()).thenReturn(TEST_ERROR_MESSAGE); + when(cause.getMessage()).thenReturn(TEST_CAUSE_ERROR_MESSAGE); + when(exception.getCause()).thenReturn(cause); + final PluginError pluginError = PluginError.builder() + .pluginName(TEST_PLUGIN_NAME) + .exception(exception) + .build(); + assertThat(pluginError.getErrorMessage(), equalTo( + "test-plugin: caused by: test error message caused by: test cause error message")); + } +} \ No newline at end of file diff --git a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionLoader.java b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionLoader.java index 16706448e9..f527d835a4 100644 --- a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionLoader.java +++ b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionLoader.java @@ -6,8 +6,12 @@ package org.opensearch.dataprepper.plugin; import org.opensearch.dataprepper.model.annotations.DataPrepperExtensionPlugin; +import org.opensearch.dataprepper.model.configuration.PipelinesDataFlowModel; import org.opensearch.dataprepper.model.plugin.ExtensionPlugin; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.plugin.InvalidPluginDefinitionException; +import org.opensearch.dataprepper.validation.PluginError; +import org.opensearch.dataprepper.validation.PluginErrorCollector; import javax.inject.Inject; import javax.inject.Named; @@ -21,26 +25,47 @@ public class ExtensionLoader { private final ExtensionPluginConfigurationConverter extensionPluginConfigurationConverter; private final ExtensionClassProvider extensionClassProvider; private final PluginCreator extensionPluginCreator; + private final PluginErrorCollector pluginErrorCollector; @Inject ExtensionLoader( final ExtensionPluginConfigurationConverter extensionPluginConfigurationConverter, final ExtensionClassProvider extensionClassProvider, - @Named("extensionPluginCreator") final PluginCreator extensionPluginCreator) { + @Named("extensionPluginCreator") final PluginCreator extensionPluginCreator, + final PluginErrorCollector pluginErrorCollector) { this.extensionPluginConfigurationConverter = extensionPluginConfigurationConverter; this.extensionClassProvider = extensionClassProvider; this.extensionPluginCreator = extensionPluginCreator; + this.pluginErrorCollector = pluginErrorCollector; } - List loadExtensions() { - return extensionClassProvider.loadExtensionPluginClasses() + public List loadExtensions() { + final List result = extensionClassProvider.loadExtensionPluginClasses() .stream() .map(extensionClass -> { - final PluginArgumentsContext pluginArgumentsContext = getConstructionContext(extensionClass); - return extensionPluginCreator.newPluginInstance( - extensionClass, pluginArgumentsContext, convertClassToName(extensionClass)); + final String pluginName = convertClassToName(extensionClass); + try { + final PluginArgumentsContext pluginArgumentsContext = getConstructionContext(extensionClass); + return extensionPluginCreator.newPluginInstance( + extensionClass, pluginArgumentsContext, pluginName); + } catch (Exception e) { + final PluginError pluginError = PluginError.builder() + .componentType(PipelinesDataFlowModel.EXTENSION_PLUGIN_TYPE) + .pluginName(pluginName) + .exception(e) + .build(); + pluginErrorCollector.collectPluginError(pluginError); + return null; + } }) .collect(Collectors.toList()); + if (!pluginErrorCollector.getPluginErrors().isEmpty()) { + throw new InvalidPluginConfigurationException( + "One or more extension plugins are not configured correctly.\n" + + pluginErrorCollector.getConsolidatedErrorMessage()); + } else { + return result; + } } private PluginArgumentsContext getConstructionContext(final Class extensionPluginClass) { diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionLoaderTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionLoaderTest.java index 176f3e0702..dc47a4f698 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionLoaderTest.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionLoaderTest.java @@ -6,6 +6,8 @@ package org.opensearch.dataprepper.plugin; import org.apache.commons.lang3.stream.Streams; +import org.hamcrest.CoreMatchers; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; @@ -17,9 +19,12 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.model.plugin.ExtensionPlugin; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.plugin.InvalidPluginDefinitionException; import org.opensearch.dataprepper.plugins.test.TestExtensionConfig; import org.opensearch.dataprepper.plugins.test.TestExtensionWithConfig; +import org.opensearch.dataprepper.validation.PluginError; +import org.opensearch.dataprepper.validation.PluginErrorCollector; import java.util.ArrayList; import java.util.Collection; @@ -31,7 +36,9 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; @@ -52,9 +59,16 @@ class ExtensionLoaderTest { private PluginCreator extensionPluginCreator; @Captor private ArgumentCaptor pluginArgumentsContextArgumentCaptor; + private PluginErrorCollector pluginErrorCollector; private ExtensionLoader createObjectUnderTest() { - return new ExtensionLoader(extensionPluginConfigurationConverter, extensionClassProvider, extensionPluginCreator); + return new ExtensionLoader(extensionPluginConfigurationConverter, extensionClassProvider, + extensionPluginCreator, pluginErrorCollector); + } + + @BeforeEach + void setUp() { + pluginErrorCollector = new PluginErrorCollector(); } @Test @@ -65,6 +79,7 @@ void loadExtensions_returns_empty_list_when_no_plugin_classes() { assertThat(extensionPlugins, notNullValue()); assertThat(extensionPlugins.size(), equalTo(0)); + assertThat(pluginErrorCollector.getPluginErrors().isEmpty(), is(true)); } @Test @@ -85,6 +100,27 @@ void loadExtensions_returns_single_extension_for_single_plugin_class() { assertThat(extensionPlugins, notNullValue()); assertThat(extensionPlugins.size(), equalTo(1)); assertThat(extensionPlugins.get(0), equalTo(expectedPlugin)); + assertThat(pluginErrorCollector.getPluginErrors().isEmpty(), is(true)); + } + + @Test + void loadExtensions_throws_InvalidPluginConfigurationException_when_invoking_newPluginInstance_throws_exception() { + final Class pluginClass = (Class) mock(ExtensionPlugin.class).getClass(); + + when(extensionClassProvider.loadExtensionPluginClasses()).thenReturn(Collections.singleton(pluginClass)); + + when(extensionPluginCreator.newPluginInstance( + eq(pluginClass), + any(PluginArgumentsContext.class), + startsWith("extension_plugin"))) + .thenThrow(TestException.class); + + assertThrows(InvalidPluginConfigurationException.class, () -> createObjectUnderTest().loadExtensions()); + assertThat(pluginErrorCollector.getPluginErrors().size(), equalTo(1)); + final PluginError pluginError = pluginErrorCollector.getPluginErrors().get(0); + assertThat(pluginError.getPipelineName(), nullValue()); + assertThat(pluginError.getPluginName(), CoreMatchers.startsWith("extension_plugin")); + assertThat(pluginError.getException(), instanceOf(TestException.class)); } @ParameterizedTest @@ -113,6 +149,24 @@ void loadExtensions_returns_single_extension_with_config_for_single_plugin_class assertThat(extensionPlugins, notNullValue()); assertThat(extensionPlugins.size(), equalTo(1)); assertThat(extensionPlugins.get(0), equalTo(expectedPlugin)); + assertThat(pluginErrorCollector.getPluginErrors().isEmpty(), is(true)); + } + + @Test + void loadExtensions_throws_InvalidPluginConfigurationException_when_extensionPluginConfigurationConverter_throws_exception() { + when(extensionClassProvider.loadExtensionPluginClasses()) + .thenReturn(Collections.singleton(TestExtensionWithConfig.class)); + when(extensionClassProvider.loadExtensionPluginClasses()).thenReturn( + Collections.singleton(TestExtensionWithConfig.class)); + when(extensionPluginConfigurationConverter.convert(eq(true), eq(TestExtensionConfig.class), + eq("/test_extension"))).thenThrow(TestException.class); + + assertThrows(InvalidPluginConfigurationException.class, () -> createObjectUnderTest().loadExtensions()); + assertThat(pluginErrorCollector.getPluginErrors().size(), equalTo(1)); + final PluginError pluginError = pluginErrorCollector.getPluginErrors().get(0); + assertThat(pluginError.getPipelineName(), nullValue()); + assertThat(pluginError.getPluginName(), CoreMatchers.startsWith("test_extension_with_config")); + assertThat(pluginError.getException(), instanceOf(TestException.class)); } @Test @@ -148,6 +202,7 @@ void loadExtensions_returns_multiple_extensions_for_multiple_plugin_classes() { for (ExtensionPlugin expectedPlugin : actualPlugins) { assertThat(actualPlugins, hasItem(expectedPlugin)); } + assertThat(pluginErrorCollector.getPluginErrors().isEmpty(), is(true)); } @Test @@ -175,6 +230,7 @@ void loadExtensions_invokes_newPluginInstance_with_PluginArgumentsContext_not_su final Class[] inputClasses = {String.class}; assertThrows(InvalidPluginDefinitionException.class, () -> actualPluginArgumentsContext.createArguments(inputClasses)); + assertThat(pluginErrorCollector.getPluginErrors().isEmpty(), is(true)); } @Test @@ -203,6 +259,7 @@ void loadExtensions_invokes_newPluginInstance_with_PluginArgumentsContext_which_ final Object[] arguments = actualPluginArgumentsContext.createArguments(new Class[]{}); assertThat(arguments, notNullValue()); assertThat(arguments.length, equalTo(0)); + assertThat(pluginErrorCollector.getPluginErrors().isEmpty(), is(true)); } @ParameterizedTest @@ -230,4 +287,8 @@ private interface TestExtension2 extends ExtensionPlugin { } private interface TestExtension3 extends ExtensionPlugin { } + + private static class TestException extends RuntimeException { + + } } From f3fe95c7258583b38a801cc0f8c6929e5a301408 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Thu, 22 Aug 2024 12:58:22 -0700 Subject: [PATCH 127/159] Changelog for release 2.9 (#4855) Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../data-prepper.change-log-2.9.0.md | 1813 +++++++++++++++++ 1 file changed, 1813 insertions(+) create mode 100644 release/release-notes/data-prepper.change-log-2.9.0.md diff --git a/release/release-notes/data-prepper.change-log-2.9.0.md b/release/release-notes/data-prepper.change-log-2.9.0.md new file mode 100644 index 0000000000..b72356316b --- /dev/null +++ b/release/release-notes/data-prepper.change-log-2.9.0.md @@ -0,0 +1,1813 @@ + +* __Change main branch version to 2.10-SNAPSHOT (#4851)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Tue, 20 Aug 2024 13:46:09 -0700 + + EAD -> refs/heads/main, refs/remotes/origin/main, refs/remotes/origin/HEAD + Change main branch version to 2.10-SNAPSHOT + Signed-off-by: Kondaka <krishkdk@amazon.com> + + --------- + Signed-off-by: Kondaka <krishkdk@amazon.com> + +* __Add support for AWS security lake sink as a bucket selector mode in S3 sink (#4846)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Tue, 20 Aug 2024 13:19:42 -0700 + + + * dplive1.yaml + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Delete .github/workflows/static.yml + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Add support for AWS security lake sink as a bucket selector mode in S3 sink + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Fixed tests + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Added javadoc for S3BucketSelector + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Added new tests for KeyGenerator + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Added new tests and fixed style errors + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Addressed review comments + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Fixed test build failure + Signed-off-by: Kondaka <krishkdk@amazon.com> + + --------- + Signed-off-by: Kondaka <krishkdk@amazon.com> + +* __Add handle failed events option to parse json processors (#4844)__ + + [Taylor Gray](mailto:tylgry@amazon.com) - Mon, 19 Aug 2024 17:12:29 -0500 + + + Signed-off-by: Taylor Gray <tylgry@amazon.com> + +* __Fix bug where race condition on ack callback could cause S3 folder partition to not be given up (#4835)__ + + [Taylor Gray](mailto:tylgry@amazon.com) - Mon, 19 Aug 2024 16:15:05 -0500 + + + Signed-off-by: Taylor Gray <tylgry@amazon.com> + +* __Update the parse JSON/XML/ION processors to use EventKey. (#4842)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 19 Aug 2024 11:12:27 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Add support for dynamic rule detection for pipeline config transformation (#4601)__ + + [Srikanth Govindarajan](mailto:srigovs@amazon.com) - Fri, 16 Aug 2024 18:10:52 -0700 + + + * Add support for dynamic rule detection for pipeline config transformation + Signed-off-by: Srikanth Govindarajan <srikanthjg123@gmail.com> + + * Address comments + Signed-off-by: Srikanth Govindarajan <srigovs@amazon.com> + + * Move rules and templates to plugin level + Signed-off-by: Srikanth Govindarajan <srigovs@amazon.com> + + * Add dummy plugin for testing dynamic rule detection + Signed-off-by: Srikanth Govindarajan <srigovs@amazon.com> + + * Address comments + Signed-off-by: Srikanth Govindarajan <srigovs@amazon.com> + + --------- + Signed-off-by: Srikanth Govindarajan <srikanthjg123@gmail.com> + Signed-off-by: + Srikanth Govindarajan <srigovs@amazon.com> + +* __Add startsWith expression function (#4840)__ + + [Taylor Gray](mailto:tylgry@amazon.com) - Fri, 16 Aug 2024 16:37:44 -0500 + + + Signed-off-by: Taylor Gray <tylgry@amazon.com> + +* __Logging improvements when failing to parse JSON/XML/ION. Do not include the stack trace since it doesn't provide any value with these exceptions which are expected when the JSON is invalid. Log the input string rather than the Event which was not readable. (#4839)__ + + [David Venable](mailto:dlv@amazon.com) - Fri, 16 Aug 2024 05:52:46 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Translate proc optimizations (#4824)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Thu, 15 Aug 2024 10:25:43 -0700 + + + * dplive1.yaml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Delete .github/workflows/static.yml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Optimize translateSource in translate processor + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Data Prepper expressions - Set operator fix (#4818)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Wed, 14 Aug 2024 14:05:26 -0700 + + + * dplive1.yaml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Delete .github/workflows/static.yml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Fixed check style errors + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Http chunking fixes (#4823)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Wed, 14 Aug 2024 11:49:12 -0700 + + + * dplive1.yaml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Delete .github/workflows/static.yml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Fix http message chunking bug + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Modified tests to test for chunks correctly + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Added comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed offline review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Added tests + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Added tests + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __FIX: include schema cli into release (#4833)__ + + [Qi Chen](mailto:qchea@amazon.com) - Wed, 14 Aug 2024 09:38:44 -0700 + + + MAINT: include schema cli into release + Signed-off-by: George Chen <qchea@amazon.com> + +* __PersonalizeSink: add client and configuration classes (#4803)__ + + [Ivan Tse](mailto:115105835+ivan-tse@users.noreply.github.com) - Wed, 14 Aug 2024 09:36:12 -0700 + + + PersonalizeSink: add client and configuration classes + Signed-off-by: Ivan Tse <tseiva@amazon.com> + +* __Config description changes for aggregate and anomaly detector processors. (#4829)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Tue, 13 Aug 2024 13:41:22 -0700 + + + * dplive1.yaml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Delete .github/workflows/static.yml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Add json property description for aggregate processor and anomaly detector + processors + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Fixed build failure + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Add delete_source parameter to the csv processor (#4828)__ + + [Taylor Gray](mailto:tylgry@amazon.com) - Mon, 12 Aug 2024 16:21:05 -0500 + + + Signed-off-by: Taylor Gray <tylgry@amazon.com> + +* __FIX: build service map relationship even when trace group is missing (#4822)__ + + [Qi Chen](mailto:qchea@amazon.com) - Mon, 12 Aug 2024 13:24:50 -0500 + + + Signed-off-by: George Chen <qchea@amazon.com> + +* __ENH: add folder path as output for schema generation (#4820)__ + + [Qi Chen](mailto:qchea@amazon.com) - Mon, 12 Aug 2024 11:51:51 -0500 + + + * ENH: add folder path as output + Signed-off-by: George Chen <qchea@amazon.com> + + +* __Release notes for Data Prepper 2.8.1 (#4807)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 12 Aug 2024 09:25:22 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Create docker-compose-dataprepper.yaml (#4756)__ + + [Jayesh Parmar](mailto:89792517+jayeshjeh@users.noreply.github.com) - Mon, 12 Aug 2024 08:56:55 -0700 + + + * Create docker-compose-dataprepper.yaml + Signed-off-by: Jayesh Parmar <89792517+jayeshjeh@users.noreply.github.com> + + Signed-off-by: jayeshjeh <jay.parmar.11169@gmail.com> + + * Necessary chnages made + Signed-off-by: jayeshjeh <jay.parmar.11169@gmail.com> + + --------- + Signed-off-by: Jayesh Parmar <89792517+jayeshjeh@users.noreply.github.com> + + Signed-off-by: jayeshjeh <jay.parmar.11169@gmail.com> + +* __Fixes a regex expression bug. When the left-hand side of the operation is null, always return false rather than throwing an exception. Resolves #4763. (#4798)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 12 Aug 2024 08:20:48 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Fix null document in DLQ object (#4814)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Sat, 10 Aug 2024 12:04:18 -0700 + + + * dplive1.yaml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Delete .github/workflows/static.yml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Fix null document in DLQ object + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Several improvements to RDS source (#4810)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Thu, 8 Aug 2024 16:51:12 -0500 + + + * Add schema manager to query database + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Get real primary keys for export + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Get binlog start position for stream + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Refactor SnapshotStrategy to RdsApiStrategy + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Update unit tests + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * address comments + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Add retry to database queries + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Handle describe exceptions + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Address more comments + Signed-off-by: Hai Yan <oeyh@amazon.com> + + --------- + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __ADD: data prepper plugin schema generation (#4777)__ + + [Qi Chen](mailto:qchea@amazon.com) - Fri, 2 Aug 2024 14:44:32 -0500 + + + * ADD: data-prepper-plugin-schema + Signed-off-by: George Chen <qchea@amazon.com> + +* __Using Awaitility and mocks in the LogGeneratorSourceTest to attempt to improve reliability. (#4746)__ + + [David Venable](mailto:dlv@amazon.com) - Fri, 2 Aug 2024 09:28:55 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Removes @asifsmohammed from the CODEOWNERS to allow the release to proceed. (#4800)__ + + [David Venable](mailto:dlv@amazon.com) - Thu, 1 Aug 2024 14:11:53 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Increase timeout in Acknowledgement IT tests (#4774)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Wed, 31 Jul 2024 18:45:39 -0700 + + + Increase timeout for acknowledgement IT tests + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Signed-off-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + Co-authored-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Cleanup resources properly when Opensearch sink fails to initialize (#4758)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Tue, 30 Jul 2024 13:33:51 -0700 + + + * dplive1.yaml + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * rebased to latest + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * removed unnecessary file + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Add rds source metrics (#4769)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Tue, 30 Jul 2024 12:09:20 -0500 + + + * Add rds source metrics + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Remove unused imports + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Add exportS3ObjectsErrors metric + Signed-off-by: Hai Yan <oeyh@amazon.com> + + --------- + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Reapply "Run tests on the current JVM for Java 17 & 21 / Gradle 8.8 (#4730)" (#4762) (#4771)__ + + [David Venable](mailto:dlv@amazon.com) - Tue, 30 Jul 2024 09:35:12 -0700 + + + This reverts commit 5c7d58c03059c7a753d882f5b74fa6ed32f45641. + Signed-off-by: David Venable <dlv@amazon.com> + +* __4602 one way hash (#4750)__ + + [mishavay-aws](mailto:140549901+mishavay-aws@users.noreply.github.com) - Mon, 29 Jul 2024 18:41:20 -0700 + + + added capabilities for working with OneWay Hash + Signed-off-by: mishavay-aws <140549901+mishavay-aws@users.noreply.github.com> + +* __Corrects the TRIAGING.md with a video meeting since we currently use Chime. (#4743)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 29 Jul 2024 16:34:34 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Add json property description for list-to-map, map-to-list and user-agent processor (#4759)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Mon, 29 Jul 2024 11:32:31 -0500 + + + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Add json property descriptions to dissect, flatten, copy_value and translate processor (#4760)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Mon, 29 Jul 2024 11:32:25 -0500 + + + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Lambda sink refactor (#4766)__ + + [Srikanth Govindarajan](mailto:srigovs@amazon.com) - Fri, 26 Jul 2024 22:56:36 -0500 + + + * Lambda sink refactor + Signed-off-by: Srikanth Govindarajan <srigovs@amazon.com> + + * Address comments + Signed-off-by: Srikanth Govindarajan <srigovs@amazon.com> + + --------- + Signed-off-by: Srikanth Govindarajan <srigovs@amazon.com> + +* __A few improvements to rds source (#4765)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Fri, 26 Jul 2024 13:31:54 -0500 + + + * Add error logging to event handlers + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Add tls config and enable tls by default + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Add original event name to metadata + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Update metadata for export and stream events + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Add some fixes + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Remove config alias ssl + Signed-off-by: Hai Yan <oeyh@amazon.com> + + --------- + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Fixes a bug with HistogramAggregateAction where the startTime may be incorrect. This was discovered by a flaky test. (#4749)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 24 Jul 2024 23:01:21 -0700 + + + The HistogramAggregateAction was incorrectly using the current time as the + start time for the aggregation when creating the group. The very first event's + time was overridden by the current system time. If this event had the earliest + time, then the overall histogram would never get the correct start time. This + is fixed by removing an errant line. I also added a unit test to directly test + this failure scenario. + Signed-off-by: David Venable <dlv@amazon.com> + +* __Add support for taking snapshots on RDS/Aurora Clusters (#4761)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Wed, 24 Jul 2024 22:52:38 -0500 + + + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Revert "Run tests on the current JVM for Java 17 & 21 / Gradle 8.8 (#4730)" (#4762)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Tue, 23 Jul 2024 17:03:57 -0500 + + + This reverts commit 67f3595805f07442d8f05823c9959b50358aa4d9. + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Add stream processing for rds source (#4757)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Tue, 23 Jul 2024 16:30:35 -0500 + + + * Add stream processing + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Address review comments + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Change s3 partition count default value + Signed-off-by: Hai Yan <oeyh@amazon.com> + + --------- + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Add json description to AddEntry processor (#4752)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Tue, 23 Jul 2024 14:34:05 -0500 + + + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __doc strings for enhanced UI view auto-generation (#4755)__ + + [Santhosh Gandhe](mailto:1909520+san81@users.noreply.github.com) - Mon, 22 Jul 2024 14:40:18 -0700 + + + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + +* __MAINT: Add json property descriptions for csv processor (#4751)__ + + [Katherine Shen](mailto:40495707+shenkw1@users.noreply.github.com) - Mon, 22 Jul 2024 15:06:13 -0500 + + + * add json property descriptions + Signed-off-by: Katherine Shen + <katshen@amazon.com> + +* __Improve the SQS shutdown process such that it does not prevent the pipeline from shutting down and no longer results in failures. Resolves #4575 (#4748)__ + + [David Venable](mailto:dlv@amazon.com) - Fri, 19 Jul 2024 11:40:57 -0700 + + + The previous approach to shutting down the SQS thread closed the SqsClient. + However, with acknowledgments enabled, asynchronous callbacks would result in + further attempts to either ChangeVisibilityTimeout or DeleteMessages. These + were failing because the client was closed. Also, the threads would remain and + prevent Data Prepper from correctly shutting down. With this change, we + correctly stop each processing thread. Then we close the client. Additionally, + the SqsWorker now checks that it is not stopped before attempting to change the + message visibility or delete messages. + Additionally, I found some missing test cases. Also, modifying this code and + especially unit testing it is becoming more difficult, so I performed some + refactoring to move message parsing out of the SqsWorker. + Signed-off-by: David Venable <dlv@amazon.com> + +* __MAINT: add json property descriptions for kv configs (#4747)__ + + [Katherine Shen](mailto:40495707+shenkw1@users.noreply.github.com) - Fri, 19 Jul 2024 10:04:50 -0700 + + + add json property descriptions for kv configs + Signed-off-by: Katherine Shen <katshen@amazon.com> + +* __Updates Jackson to 2.17.2. Related to #4729. (#4744)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 17 Jul 2024 15:49:55 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Updating the Python dependencies to resolve CVEs. Resolves #4715, #4713, 4714. (#4733)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 15 Jul 2024 11:55:11 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __REF: service-map processor with the latest config model (#4734)__ + + [Qi Chen](mailto:qchea@amazon.com) - Mon, 15 Jul 2024 11:02:40 -0500 + + + * REF: service-map processor with the latest config model + Signed-off-by: George Chen <qchea@amazon.com> + +* __MAINT: add documentation in json property description for date processor (#4719)__ + + [Qi Chen](mailto:qchea@amazon.com) - Mon, 15 Jul 2024 10:30:14 -0500 + + + * MAINT: add documentation in json property description for date processor + Signed-off-by: George Chen <qchea@amazon.com> + +* __REF: grok processor with the latest config model (#4731)__ + + [Qi Chen](mailto:qchea@amazon.com) - Mon, 15 Jul 2024 10:05:10 -0500 + + + * REF: grok processor with the latest config model + Signed-off-by: George Chen <qchea@amazon.com> + +* __Load exported S3 files in RDS source (#4718)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Mon, 15 Jul 2024 09:57:24 -0500 + + + * Add s3 file loader + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Make checkExportStatus a callable + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Fix unit tests + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Add load status and record converter + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Update unit tests + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Restore changes for test + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Address review comments + Signed-off-by: Hai Yan <oeyh@amazon.com> + + --------- + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __MAINT: backfill doc into json property for trim_string (#4728)__ + + [Qi Chen](mailto:qchea@amazon.com) - Sat, 13 Jul 2024 00:31:45 -0500 + + + * MAINT: backfill doc into json property for trim_string + Signed-off-by: George Chen <qchea@amazon.com> + +* __MAINT: backfill documentation in JsonPropertyDescription for split_string (#4720)__ + + [Qi Chen](mailto:qchea@amazon.com) - Sat, 13 Jul 2024 00:31:23 -0500 + + + * MAINT: add documentation in JsonPropertyDescription for split_string + processor + Signed-off-by: George Chen <qchea@amazon.com> + +* __MAINT: backfill doc in json property description for otel_metrics (#4722)__ + + [Qi Chen](mailto:qchea@amazon.com) - Sat, 13 Jul 2024 00:30:56 -0500 + + + * MAINT: backfill doc in json property description for otel_metrics + Signed-off-by: George Chen <qchea@amazon.com> + +* __MAINT: add json property description into obfuscate processor (#4706)__ + + [Qi Chen](mailto:qchea@amazon.com) - Fri, 12 Jul 2024 22:28:10 -0500 + + + * MAINT: add json property description + Signed-off-by: George Chen <qchea@amazon.com> + +* __Run tests on the current JVM for Java 17 & 21 / Gradle 8.8 (#4730)__ + + [David Venable](mailto:dlv@amazon.com) - Fri, 12 Jul 2024 15:35:02 -0700 + + + Run tests on the current JVM rather than always using Java 11 for the tests. + This fixes a problem with our current GitHub tests where we are running against + only Java 11 even though we want to run against different Java versions (11, + 17, 21). Updates the Gradle version to 8.8. + Fix Java 21 support in the AbstractSink by removing usage of Thread::stop + which now always throws an UnsupportedOperationException. + Use only microsecond precision time when comparing the times in the event_json + codec. These tests are failing now on Java 17 and 21 with precision errors. + Fixed a randomly failing test in BlockingBufferTests where a value 0 caused an + IllegalArgumentException. + Logging changes to avoid noise in the Gradle builds in GitHub. + Signed-off-by: David Venable <dlv@amazon.com> + +* __MAINT: backfill documentation into json description for truncate processor (#4726)__ + + [Qi Chen](mailto:qchea@amazon.com) - Fri, 12 Jul 2024 13:30:05 -0500 + + + * MAINT: backfill documentation into json description for truncate processor + Signed-off-by: George Chen <qchea@amazon.com> + +* __MAINT: backfill documentation into json property for substitute_string (#4727)__ + + [Qi Chen](mailto:qchea@amazon.com) - Fri, 12 Jul 2024 13:29:41 -0500 + + + * MAINT: backfill documentation into json property for substitute_string + Signed-off-by: George Chen <qchea@amazon.com> + +* __MAINT: backfill documentation into json description for delete_entries (#4721)__ + + [Qi Chen](mailto:qchea@amazon.com) - Fri, 12 Jul 2024 13:29:16 -0500 + + + * MAINT: backfill documentation into json description for delete_entries + Signed-off-by: George Chen <qchea@amazon.com> + +* __MAINT: backfill documentation in json description for otel_traces (#4724)__ + + [Qi Chen](mailto:qchea@amazon.com) - Fri, 12 Jul 2024 12:40:52 -0500 + + + * MAINT: backfill documentation in json property description for otel_traces + Signed-off-by: George Chen <qchea@amazon.com> + +* __MAINT: backfill documentation into json description for string_converter (#4725)__ + + [Qi Chen](mailto:qchea@amazon.com) - Fri, 12 Jul 2024 12:40:24 -0500 + + + * MAINT: backfill documentation into json description for string_converter + Signed-off-by: George Chen <qchea@amazon.com> + +* __Mockito 5 (#4712)__ + + [David Venable](mailto:dlv@amazon.com) - Thu, 11 Jul 2024 11:17:14 -0700 + + + Mockito 5 + + * Synchronize the MetricsTestUtil methods to avoid test failures. + * Create a copy of the collections to remove in MetricsTestUtil. + * Updated two tests to JUnit 5 and to use mocks instead of actual metrics. + Updates to MetricsTestUtil to provide clarity on NPEs. + Signed-off-by: David Venable <dlv@amazon.com> + +* __Updates to the AWS Lambda Sink tests to fix a flaky test. Also adds SLF4J logging for these tests. (#4723)__ + + [David Venable](mailto:dlv@amazon.com) - Thu, 11 Jul 2024 08:07:32 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Update the rename_keys and delete_entries processors to use the EventKey. (#4636)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 10 Jul 2024 08:50:39 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Update the mutate string processors to use the EventKey. #4646 (#4649)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 10 Jul 2024 08:50:21 -0700 + + + Change the source and keys properties for mutate string processors to use + EventKey such that they are parsed by Data Prepper core. Also, use the + TestEventFactory in the tests to avoid use of JacksonEvent directly. Removes an + unused class. + Signed-off-by: David Venable <dlv@amazon.com> + +* __Removes Zookeeper from Data Prepper. This was a transitive dependency from Hadoop. (#4707)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 10 Jul 2024 08:49:20 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Updates the user_agent processor to use the EventKey. (#4628)__ + + [David Venable](mailto:dlv@amazon.com) - Tue, 9 Jul 2024 13:57:19 -0700 + + + Updates the user_agent processor to use the EventKey. + Signed-off-by: David Venable <dlv@amazon.com> + Co-authored-by: Karsten + Schnitter <k.schnitter@sap.com> + +* __Introducing delete input configuration option for some parsers (#4702)__ + + [Santhosh Gandhe](mailto:1909520+san81@users.noreply.github.com) - Thu, 4 Jul 2024 11:11:27 -0700 + + + * Introduced delete_source configuration option to give flexibility for the + user to drop the raw source record if they don't want to propagate it + downstream + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * addressing review comments + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * added delete_source option to other similar parser classes + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + --------- + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + +* __Bump net.bytebuddy:byte-buddy in /data-prepper-plugins/opensearch (#4593)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Wed, 3 Jul 2024 13:49:02 -0700 + + + Bumps [net.bytebuddy:byte-buddy](https://github.com/raphw/byte-buddy) from + 1.14.12 to 1.14.17. + - [Release notes](https://github.com/raphw/byte-buddy/releases) + - [Changelog](https://github.com/raphw/byte-buddy/blob/master/release-notes.md) + + - + [Commits](https://github.com/raphw/byte-buddy/compare/byte-buddy-1.14.12...byte-buddy-1.14.17) + + + --- + updated-dependencies: + - dependency-name: net.bytebuddy:byte-buddy + dependency-type: direct:production + update-type: version-update:semver-patch + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Bump net.bytebuddy:byte-buddy-agent in /data-prepper-plugins/opensearch (#4592)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Wed, 3 Jul 2024 12:56:23 -0700 + + + Bumps [net.bytebuddy:byte-buddy-agent](https://github.com/raphw/byte-buddy) + from 1.14.12 to 1.14.17. + - [Release notes](https://github.com/raphw/byte-buddy/releases) + - [Changelog](https://github.com/raphw/byte-buddy/blob/master/release-notes.md) + + - + [Commits](https://github.com/raphw/byte-buddy/compare/byte-buddy-1.14.12...byte-buddy-1.14.17) + + + --- + updated-dependencies: + - dependency-name: net.bytebuddy:byte-buddy-agent + dependency-type: direct:production + update-type: version-update:semver-patch + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Key Value processor value grouping optimization (#4704)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Wed, 3 Jul 2024 12:43:46 -0700 + + + * dplive1.yaml + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Optimize findInStartGroup in KV processor + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Rebased to latest + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Cleanup + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Minor improvements to code + Signed-off-by: Kondaka <krishkdk@amazon.com> + + * Addressed review comments + Signed-off-by: Kondaka <krishkdk@amazon.com> + + --------- + Signed-off-by: Kondaka <krishkdk@amazon.com> + +* __Updates to the CODE_OF_CONDUCT.md from the opensearch-project's official CODE_OF_CONDUCT.md. (#4665)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 3 Jul 2024 10:51:37 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Updates Parquet to 1.14.1 and Hadoop to 3.4.0. Make use of Gradle's version catalogue for Parquet. (#4705)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 3 Jul 2024 09:41:10 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Export to S3 in RDS source (#4664)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Wed, 3 Jul 2024 09:51:34 -0500 + + + * Trigger RDS export to S3 + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Add unit tests + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Remove unused imports + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Address review comments + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Address further comments + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Make checkSnapshotStatus a runnable + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Revert "Make checkSnapshotStatus a runnable" + This reverts commit 5caed6ffb218d64180b10285c5c9115f21d6f3a2. + Signed-off-by: Hai Yan <oeyh@amazon.com> + + --------- + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Parquet codec tests fix (#4698)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Tue, 2 Jul 2024 09:26:16 -0700 + + + Parquet codec tests fix + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Bump org.skyscreamer:jsonassert from 1.5.1 to 1.5.3 in /data-prepper-api (#4678)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Tue, 2 Jul 2024 07:18:48 -0700 + + + Bumps [org.skyscreamer:jsonassert](https://github.com/skyscreamer/JSONassert) + from 1.5.1 to 1.5.3. + - [Release notes](https://github.com/skyscreamer/JSONassert/releases) + - + [Changelog](https://github.com/skyscreamer/JSONassert/blob/master/CHANGELOG.md) + + - + [Commits](https://github.com/skyscreamer/JSONassert/compare/jsonassert-1.5.1...jsonassert-1.5.3) + + + --- + updated-dependencies: + - dependency-name: org.skyscreamer:jsonassert + dependency-type: direct:production + update-type: version-update:semver-patch + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Bump org.apache.maven:maven-artifact in /data-prepper-plugins/opensearch (#4692)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Mon, 1 Jul 2024 14:46:28 -0700 + + + Bumps [org.apache.maven:maven-artifact](https://github.com/apache/maven) from + 3.9.6 to 3.9.8. + - [Release notes](https://github.com/apache/maven/releases) + - [Commits](https://github.com/apache/maven/compare/maven-3.9.6...maven-3.9.8) + + --- + updated-dependencies: + - dependency-name: org.apache.maven:maven-artifact + dependency-type: direct:production + update-type: version-update:semver-patch + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Bump software.amazon.awssdk:auth in /performance-test (#4685)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Mon, 1 Jul 2024 14:45:24 -0700 + + + Bumps software.amazon.awssdk:auth from 2.25.21 to 2.26.12. + + --- + updated-dependencies: + - dependency-name: software.amazon.awssdk:auth + dependency-type: direct:production + update-type: version-update:semver-minor + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Bump braces from 3.0.2 to 3.0.3 in /testing/aws-testing-cdk (#4638)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Mon, 1 Jul 2024 14:44:18 -0700 + + + Bumps [braces](https://github.com/micromatch/braces) from 3.0.2 to 3.0.3. + - [Changelog](https://github.com/micromatch/braces/blob/master/CHANGELOG.md) + - [Commits](https://github.com/micromatch/braces/compare/3.0.2...3.0.3) + + --- + updated-dependencies: + - dependency-name: braces + dependency-type: indirect + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Bump ws from 7.5.9 to 7.5.10 in /release/staging-resources-cdk (#4639)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Mon, 1 Jul 2024 14:43:47 -0700 + + + Bumps [ws](https://github.com/websockets/ws) from 7.5.9 to 7.5.10. + - [Release notes](https://github.com/websockets/ws/releases) + - [Commits](https://github.com/websockets/ws/compare/7.5.9...7.5.10) + + --- + updated-dependencies: + - dependency-name: ws + dependency-type: indirect + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Bump org.wiremock:wiremock in /data-prepper-plugins/s3-source (#4683)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Mon, 1 Jul 2024 14:43:01 -0700 + + + Bumps [org.wiremock:wiremock](https://github.com/wiremock/wiremock) from 3.4.2 + to 3.8.0. + - [Release notes](https://github.com/wiremock/wiremock/releases) + - [Commits](https://github.com/wiremock/wiremock/compare/3.4.2...3.8.0) + + --- + updated-dependencies: + - dependency-name: org.wiremock:wiremock + dependency-type: direct:production + update-type: version-update:semver-minor + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Bump com.apptasticsoftware:rssreader in /data-prepper-plugins/rss-source (#4672)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Mon, 1 Jul 2024 14:42:31 -0700 + + + Bumps [com.apptasticsoftware:rssreader](https://github.com/w3stling/rssreader) + from 3.6.0 to 3.7.0. + - [Release notes](https://github.com/w3stling/rssreader/releases) + - [Commits](https://github.com/w3stling/rssreader/compare/v3.6.0...v3.7.0) + + --- + updated-dependencies: + - dependency-name: com.apptasticsoftware:rssreader + dependency-type: direct:production + update-type: version-update:semver-minor + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Updates our usage of the Apache Parquet project to use their new interfaces over the old Hadoop ones. This allows use to be ready to extract Hadoop as other changes are made to the Parquet project. Remove some Hadoop transitive dependencies and make Hadoop runtime only where possible. Added a test for INT96, clean up some test files. Contributes toward #4612. (#4623)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 1 Jul 2024 13:54:53 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Adds the TRIAGING.md file to outline our triaging process (#4630)__ + + [David Venable](mailto:dlv@amazon.com) - Thu, 27 Jun 2024 17:47:18 -0700 + + + Adds the TRIAGING.md file, which outlines for the community the Data Prepper + triaging process. + Signed-off-by: David Venable <dlv@amazon.com> + +* __Enhanced Kafka source logging through the use of MDC and better thread names for Kafka source threads. Resolves #4126. (#4663)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 26 Jun 2024 12:25:57 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Support default route option for Events that match no other route (#4662)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Wed, 26 Jun 2024 07:15:04 -0700 + + + Support default route option for Events that match no other route + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Updates the chunking algorithm for http source's JsonCodec to account for actual byte size. Test using Unicode characters to prove this was incorrectly chunking and verify against future changes. (#4656)__ + + [David Venable](mailto:dlv@amazon.com) - Tue, 25 Jun 2024 15:09:27 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Refactor lambda plugin (#4643)__ + + [Srikanth Govindarajan](mailto:srigovs@amazon.com) - Tue, 25 Jun 2024 09:51:28 -0700 + + + * Refactor lambda plugin + Signed-off-by: Srikanth Govindarajan <srikanthjg123@gmail.com> + + * Address comments + Signed-off-by: Srikanth Govindarajan <srigovs@amazon.com> + + * Address comments 2 + Signed-off-by: Srikanth Govindarajan <srigovs@amazon.com> + + --------- + Signed-off-by: Srikanth Govindarajan <srikanthjg123@gmail.com> + Signed-off-by: + Srikanth Govindarajan <srigovs@amazon.com> + +* __Fixes the loading of peer-forwarders when using multiple workers. This fixes a bug where the service_map processor would not load in a pipeline with multiple workers. Resolves #4660. (#4661)__ + + [David Venable](mailto:dlv@amazon.com) - Tue, 25 Jun 2024 08:07:40 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __FIX: remove logging that includes credential info on kafka (#4659)__ + + [Qi Chen](mailto:qchea@amazon.com) - Tue, 25 Jun 2024 09:11:40 -0500 + + + * FIX: use sensitive marker + Signed-off-by: George Chen <qchea@amazon.com> + +* __Add an option to count unique values of specified key(s) to CountAggregateAction (#4652)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Mon, 24 Jun 2024 15:46:42 -0700 + + + Add an option to count unique values of specified key(s) to + CountAggregateAction + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __MAINT: change log level for consumer properties in kafka source (#4658)__ + + [Qi Chen](mailto:qchea@amazon.com) - Mon, 24 Jun 2024 16:56:23 -0500 + + + Signed-off-by: George Chen <qchea@amazon.com> + +* __Fixes performance regression with JacksonEvent put/delete operations. (#4650)__ + + [David Venable](mailto:dlv@amazon.com) - Fri, 21 Jun 2024 08:37:03 -0700 + + + With the addition of the EventKey, JacksonEvent always creates a + JacksonEventKey in order to use the same code for all paths. However, when + put/delete calls are made with a String key, JacksonEvent does not need the + JSON Pointer. But, it is created anyway. This adds more work to the put/delete + calls that have not yet migrated to the String version. This fixes regression + by adding a lazy initialization option when used in JacksonEvent. We should not + be lazy when used with the EventKeyFactory since we may lose some up-front + validations. + Signed-off-by: David Venable <dlv@amazon.com> + +* __add remove_brackets option to flatten-processor (#4616) (#4653)__ + + [timo-mue](mailto:timo.mueller@tower.telekom-cloudcenter.de) - Fri, 21 Jun 2024 10:04:04 -0500 + + + Signed-off-by: Timo Mueller <timo.mueller@tower.telekom-cloudcenter.de> + +* __Add support to configure metric name for count and histogram actions (#4642)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Thu, 20 Jun 2024 11:05:40 -0700 + + + * rebased to latest + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * renamed name to metric_name + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Support plugins defining the EventKey in the plugin configuration classes. Data Prepper will deserialize the EventKey from the pipeline configuration and validate @NotEmpty validations. Builds on the #1916. (#4635)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 19 Jun 2024 15:08:12 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __DocumentDB Source improvements (#4645)__ + + [Dinu John](mailto:86094133+dinujoh@users.noreply.github.com) - Wed, 19 Jun 2024 16:09:54 -0500 + + + * Extend the export partition ownership during query partition creation + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + + * Add support to shutdown task refresher that starts export and stream + scheduler/worker on data prepper shutdown + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + + * Add AcknowledgmentStatus enum and code refactor to fail negative ack right + away + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + + --------- + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + +* __Track last scan time from before scan starts instead of based on last Modified of objects (#4493)__ + + [Taylor Gray](mailto:tylgry@amazon.com) - Wed, 19 Jun 2024 14:55:10 -0500 + + + Signed-off-by: Taylor Gray <tylgry@amazon.com> + +* __Bump urllib3 in /release/smoke-tests/otel-span-exporter (#4640)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Wed, 19 Jun 2024 07:23:44 -0700 + + + Bumps [urllib3](https://github.com/urllib3/urllib3) from 1.26.18 to 1.26.19. + - [Release notes](https://github.com/urllib3/urllib3/releases) + - [Changelog](https://github.com/urllib3/urllib3/blob/1.26.19/CHANGES.rst) + - [Commits](https://github.com/urllib3/urllib3/compare/1.26.18...1.26.19) + + --- + updated-dependencies: + - dependency-name: urllib3 + dependency-type: direct:production + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Bump urllib3 in /examples/trace-analytics-sample-app/sample-app (#4631)__ + + [dependabot[bot]](mailto:49699333+dependabot[bot]@users.noreply.github.com) - Tue, 18 Jun 2024 16:50:42 -0700 + + + Bumps [urllib3](https://github.com/urllib3/urllib3) from 2.0.7 to 2.2.2. + - [Release notes](https://github.com/urllib3/urllib3/releases) + - [Changelog](https://github.com/urllib3/urllib3/blob/main/CHANGES.rst) + - [Commits](https://github.com/urllib3/urllib3/compare/2.0.7...2.2.2) + + --- + updated-dependencies: + - dependency-name: urllib3 + dependency-type: direct:production + ... + Signed-off-by: dependabot[bot] <support@github.com> + Co-authored-by: + dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> + +* __Implements equals, hashCode, and toString for JacksonEventKey. (#4633)__ + + [David Venable](mailto:dlv@amazon.com) - Tue, 18 Jun 2024 16:49:34 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Deprecates PluginSetting which should not be used for plugins anymore. (#4624)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 17 Jun 2024 12:55:38 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Add Aggregate event handle (#4625)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Mon, 17 Jun 2024 12:34:33 -0700 + + + Aggregate event handle + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Adds the EventKey and EventKeyFactory. (#4627)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 17 Jun 2024 08:40:28 -0700 + + + Adds the EventKey and EventKeyFactory. Resolves #1916. + Signed-off-by: David Venable <dlv@amazon.com> + +* __Updates to Armeria 1.29.0 which fixes a bug that may help with #4080. (#4629)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 17 Jun 2024 07:26:05 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __MAINT: bump io.confluent:* packages to match org.apache.kafka.* (#4626)__ + + [Qi Chen](mailto:qchea@amazon.com) - Fri, 14 Jun 2024 11:55:28 -0500 + + + Signed-off-by: George Chen <qchea@amazon.com> + +* __Support multiple aggregate processors in local mode (#4574)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Fri, 14 Jun 2024 09:27:17 -0700 + + + * Rebased to latest + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Fixed tests + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Tools to generate User Agent strings in the performance-test project (#4620)__ + + [David Venable](mailto:dlv@amazon.com) - Fri, 14 Jun 2024 09:00:52 -0700 + + + Changes to the performance-test project to generate User Agent strings. Used to + help reproduce and test #4618. + Signed-off-by: David Venable <dlv@amazon.com> + +* __Rebased to latest (#4614)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Fri, 14 Jun 2024 08:37:55 -0700 + + + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __OpenSearch Sink add support for sending pipeline parameter in BulkRequest (#4609)__ + + [Souvik Bose](mailto:souvik04in@gmail.com) - Thu, 13 Jun 2024 23:36:00 -0700 + + + * Add support in OpenSearch sink to pass pipeline parameter in the index and + create action operation for bulk api requests + Signed-off-by: Souvik Bose <souvik04in@gmail.com> + + * Add more unit tests to increase code coverage + Signed-off-by: Souvik Bose <souvik04in@gmail.com> + + * Update README + Signed-off-by: Souvik Bose <souvik04in@gmail.com> + + * Fix the OpenSearch Integration tests + Signed-off-by: Souvik Bose <souvik04in@gmail.com> + + --------- + Signed-off-by: Souvik Bose <souvik04in@gmail.com> + +* __Caffeine-based caching parser for the user_agent processor (#4619)__ + + [David Venable](mailto:dlv@amazon.com) - Thu, 13 Jun 2024 08:22:15 -0700 + + + Adds and uses a Caffeine-based caching parser for the user_agent processor. + Resolves #4618 + Signed-off-by: David Venable <dlv@amazon.com> + +* __FIX: decouple msk auth from glue auth in KafkaSource (#4613)__ + + [Qi Chen](mailto:qchea@amazon.com) - Wed, 12 Jun 2024 14:46:08 -0500 + + + * FIX: decouple msk from aws block + Signed-off-by: George Chen <qchea@amazon.com> + +* __Fix missing closing parenthesis in CLOUDFRONT_ACCESS_LOG pattern (#4607)__ + + [Joël Marty](mailto:134835+joelmarty@users.noreply.github.com) - Wed, 12 Jun 2024 10:45:37 -0700 + + + Signed-off-by: Joël Marty <134835+joelmarty@users.noreply.github.com> + +* __Aggrerate processor : add option to allow raw events (#4598)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Mon, 10 Jun 2024 14:55:09 -0700 + + + * Aggregate Processor: Add support to allow raw events + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Modify test to check for aggregated tag + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Updates Python requests to 2.32.3 in the smoke tests project to address CVE-2024-35195. (#4610)__ + + [David Venable](mailto:dlv@amazon.com) - Fri, 7 Jun 2024 14:07:03 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Add support for lambda sink (#4292)__ + + [Srikanth Govindarajan](mailto:srikanthjg123@gmail.com) - Thu, 6 Jun 2024 20:17:27 +0000 + + + * Add support for lambda sink + Signed-off-by: srigovs <srigovs@amazon.com> + + * Address event handle comment + Signed-off-by: Srikanth Govindarajan <srikanthjg123@gmail.com> + + --------- + Signed-off-by: srigovs <srigovs@amazon.com> + Signed-off-by: Srikanth + Govindarajan <srikanthjg123@gmail.com> + +* __Fix KeyValue Processor value grouping bug (#4606)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Thu, 6 Jun 2024 12:25:36 -0700 + + + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Refactor http source functionality for supporting a new OpenSearch API source in DataPrepper (#4570)__ + + [Souvik Bose](mailto:souvik04in@gmail.com) - Tue, 4 Jun 2024 16:49:07 -0700 + + + Refactor http source configuration to a separate http source common package. + Signed-off-by: Souvik Bose <souvik04in@gmail.com> + +* __Modify Key Value processor to support string literal grouping (#4599)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Tue, 4 Jun 2024 14:19:48 -0700 + + + * Key Value Processor fixes + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * New options to KV processor + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Add string literal support + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Remove unnecessary changes + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Remove unnecessary changes + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Fixed tests + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Introduced BigDecimalConverter (#4557)__ + + [Santhosh Gandhe](mailto:1909520+san81@users.noreply.github.com) - Tue, 4 Jun 2024 13:40:46 -0700 + + + * Introduced BigDecimalConverter that users can use as part of + convert_entry_type processor that currently exists. Optionally, users can also + specify required scaling needed on the converted + Signed-off-by: Santhosh Gandhe <gandheaz@amazon.com> + + * Added Test case for the newly introduced class. Removed * imports as per the + review comment + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * Avoiding using a deprecated method. Added additional test cases + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * Additional tests to increase the coverage + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * removed "scale" being the state of BigDecimal converter. We are now passing + the scale while converting the instance only when the instance is + BigDecimalConverter + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * test case fix to be inline with the previous commit + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * test case fix to be inline with the previous commit + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * addressing review comments + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * renaming bigdecimal to big_decimal + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * Introduced ConverterArguments as a way to pass additional arguments to the + converter and avoided conditional statement for calling converter methods + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * Added additional override convert method to reduce the changes across the + code + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * additional Test cases to increase the coverage + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + * added additional tests for converter cases + Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> + + --------- + Signed-off-by: Santhosh Gandhe <gandheaz@amazon.com> + Signed-off-by: Santhosh + Gandhe <1909520+san81@users.noreply.github.com> + +* __Add Rds source config (#4573)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Tue, 4 Jun 2024 15:19:35 -0500 + + + * Add rds source config and some skeleton code + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Add unit tests + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Address comments + Signed-off-by: Hai Yan <oeyh@amazon.com> + + * Add cluster and aurora options + Signed-off-by: Hai Yan <oeyh@amazon.com> + + --------- + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Track the source of request for Kafka server (#4572)__ + + [Dinu John](mailto:86094133+dinujoh@users.noreply.github.com) - Fri, 31 May 2024 09:46:28 -0500 + + + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + +* __Add support for Kafka headers and timestamp in the Kafka Source (#4566)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Thu, 30 May 2024 09:31:47 -0700 + + + * Add support for Kafka headers and timestamp in the Kafka Source + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Fix the typo + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * fixed checkstyle error + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Add default role and region configuration to the data-prepper-config.yaml via extensions (#4559)__ + + [Taylor Gray](mailto:tylgry@amazon.com) - Wed, 22 May 2024 12:50:07 -0500 + + + Signed-off-by: Taylor Gray <tylgry@amazon.com> + +* __Corrected the release date for 2.8.0. (#4555)__ + + [David Venable](mailto:dlv@amazon.com) - Mon, 20 May 2024 13:30:25 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Release Notes for version 2.8 (#4538)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Mon, 20 May 2024 12:46:53 -0700 + + + * Release Notes for version 2.8 + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Addressed review comments + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Addressed review comments (#4552)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Mon, 20 May 2024 11:48:31 -0700 + + + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Fix DocDB export and stream processing self recovery with invalid database or collection name (#4553)__ + + [Dinu John](mailto:86094133+dinujoh@users.noreply.github.com) - Mon, 20 May 2024 13:15:55 -0500 + + + * Fix DocDB export and stream processing self recovery with invalid database or + collection name + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + + * Fix unit test + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + + --------- + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + +* __Change log for version 2.8 (#4539)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Thu, 16 May 2024 10:03:13 -0700 + + + * Change log for version 2.8 + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Change log - updated to latest + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + * Change log - updated to latest in 2.8 branch + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + + --------- + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Adding 'single_word_only' option to obfuscate processor (#4476)__ + + [Utkarsh Agarwal](mailto:126544832+Utkarsh-Aga@users.noreply.github.com) - Wed, 15 May 2024 15:38:36 -0700 + + + Adding 'single_word_only' option to obfuscate processor + Signed-off-by: Utkarsh Agarwal <utkarsh07379@gmail.com> + +* __Updates werkzeug to 3.0.3 in examples to fix CVE-2024-34069. Resolves #4515 (#4546)__ + + [David Venable](mailto:dlv@amazon.com) - Wed, 15 May 2024 15:04:53 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Logging update and config validation (#4541)__ + + [Dinu John](mailto:86094133+dinujoh@users.noreply.github.com) - Tue, 14 May 2024 17:57:24 -0500 + + + * Logging improvements for export and stream processing for DocumentDB source + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + + * Add validation for DocumentDB Collection Config + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + + --------- + Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> + +* __Add aggregate metrics (#4531)__ + + [Hai Yan](mailto:8153134+oeyh@users.noreply.github.com) - Tue, 14 May 2024 16:30:13 -0500 + + + Signed-off-by: Hai Yan <oeyh@amazon.com> + +* __Updates the next Data Prepper version to 2.9 (#4532)__ + + [Krishna Kondaka](mailto:41027584+kkondaka@users.noreply.github.com) - Tue, 14 May 2024 14:19:56 -0700 + + + Signed-off-by: Krishna Kondaka + <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + Co-authored-by: + Krishna Kondaka <krishkdk@dev-dsk-krishkdk-2c-bd29c437.us-west-2.amazon.com> + +* __Adds an ndjson input codec. This reads JSON objects for ND-JSON and more lenient formats that do not have the newline. (#4533)__ + + [David Venable](mailto:dlv@amazon.com) - Tue, 14 May 2024 12:10:01 -0700 + + + Signed-off-by: David Venable <dlv@amazon.com> + +* __Address route and subpipeline for pipeline tranformation (#4528)__ + + [Srikanth Govindarajan](mailto:srigovs@amazon.com) - Mon, 13 May 2024 15:58:11 -0700 + + + Address route and subpipeline for pipeline tranformation + Signed-off-by: srigovs <srigovs@amazon.com> + + From 0e32ec60272f2f239904ffa4b69249c41b6df4e3 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Thu, 22 Aug 2024 17:12:30 -0500 Subject: [PATCH 128/159] Mask s3 object key in logs (#4861) Signed-off-by: Hai Yan --- .../plugins/source/rds/export/DataFileLoader.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java index 16bd559848..c6815e37b7 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -23,6 +23,8 @@ import java.util.List; import java.util.concurrent.atomic.AtomicLong; +import static org.opensearch.dataprepper.logging.DataPrepperMarkers.SENSITIVE; + public class DataFileLoader implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(DataFileLoader.class); @@ -79,7 +81,7 @@ public static DataFileLoader create(final DataFilePartition dataFilePartition, @Override public void run() { - LOG.info("Start loading s3://{}/{}", bucket, objectKey); + LOG.info(SENSITIVE, "Start loading s3://{}/{}", bucket, objectKey); AtomicLong eventCount = new AtomicLong(); try (InputStream inputStream = objectReader.readFile(bucket, objectKey)) { @@ -110,14 +112,14 @@ public void run() { eventCount.getAndIncrement(); bytesProcessedSummary.record(bytes); } catch (Exception e) { - LOG.error("Failed to process record from object s3://{}/{}", bucket, objectKey, e); + LOG.error(SENSITIVE, "Failed to process record from object s3://{}/{}", bucket, objectKey, e); throw new RuntimeException(e); } }); - LOG.info("Completed loading object s3://{}/{} to buffer", bucket, objectKey); + LOG.info(SENSITIVE, "Completed loading object s3://{}/{} to buffer", bucket, objectKey); } catch (Exception e) { - LOG.error("Failed to load object s3://{}/{} to buffer", bucket, objectKey, e); + LOG.error(SENSITIVE, "Failed to load object s3://{}/{} to buffer", bucket, objectKey, e); throw new RuntimeException(e); } From 179e3dfd949031ec8faeb289c69d754beb5ee455 Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 23 Aug 2024 09:58:01 -0500 Subject: [PATCH 129/159] Adds integration tests for pipeline connectors. (#4834) This commit adds integration testing for the pipeline connector sink/source which connects two pipelines. There are two tests. The first tests against a single connection with a single sink. The second test also includes a second sink to verify that pipeline connections work with additional sinks. This commit also includes fixes for CoreHttpServerIT. When running the new pipeline connector tests, the CoreHttpServerIT tests started failing. I found some places where shutdowns were not occurring and fixed those. And I added some additional logging to help debug. The root problem turned out to be that the ExecutorService used in the DataPrepperServer was a static field. The CoreHttpServerIT was working because it was the first test that JUnit chose. With the new tests, it is being chosen later and by that point, the static ExecutorService was shutdown. The fix is simply to avoid using a static ExecutorService. Signed-off-by: David Venable --- .../Connected_SingleExtraSinkIT.java | 115 +++++++++++++++ .../integration/Connected_SingleIT.java | 133 ++++++++++++++++++ .../integration/CoreHttpServerIT.java | 19 ++- .../dataprepper/plugins/InMemorySource.java | 6 +- .../test/framework/DataPrepperTestRunner.java | 1 + .../single-connection-extra-sink.yaml | 19 +++ .../pipeline/connected/single-connection.yaml | 17 +++ .../dataprepper/pipeline/Pipeline.java | 6 +- .../pipeline/server/DataPrepperServer.java | 7 +- .../pipeline/server/ListPipelinesHandler.java | 1 + .../server/DataPrepperServerTest.java | 13 +- 11 files changed, 319 insertions(+), 18 deletions(-) create mode 100644 data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Connected_SingleExtraSinkIT.java create mode 100644 data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Connected_SingleIT.java create mode 100644 data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/connected/single-connection-extra-sink.yaml create mode 100644 data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/connected/single-connection.yaml diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Connected_SingleExtraSinkIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Connected_SingleExtraSinkIT.java new file mode 100644 index 0000000000..718f0cf6ea --- /dev/null +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Connected_SingleExtraSinkIT.java @@ -0,0 +1,115 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.integration; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.InMemorySinkAccessor; +import org.opensearch.dataprepper.plugins.InMemorySourceAccessor; +import org.opensearch.dataprepper.test.framework.DataPrepperTestRunner; + +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.awaitility.Awaitility.await; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; + +public class Connected_SingleExtraSinkIT { + private static final String IN_MEMORY_IDENTIFIER = "Connected_SingleExtraSinkIT"; + private static final String IN_MEMORY_IDENTIFIER_ENTRY_SINK = IN_MEMORY_IDENTIFIER + "_Entry"; + private static final String IN_MEMORY_IDENTIFIER_EXIT_SINK = IN_MEMORY_IDENTIFIER + "_Exit"; + private static final String PIPELINE_CONFIGURATION_UNDER_TEST = "connected/single-connection-extra-sink.yaml"; + private DataPrepperTestRunner dataPrepperTestRunner; + private InMemorySourceAccessor inMemorySourceAccessor; + private InMemorySinkAccessor inMemorySinkAccessor; + + @BeforeEach + void setUp() { + dataPrepperTestRunner = DataPrepperTestRunner.builder() + .withPipelinesDirectoryOrFile(PIPELINE_CONFIGURATION_UNDER_TEST) + .build(); + + dataPrepperTestRunner.start(); + inMemorySourceAccessor = dataPrepperTestRunner.getInMemorySourceAccessor(); + inMemorySinkAccessor = dataPrepperTestRunner.getInMemorySinkAccessor(); + } + + @AfterEach + void tearDown() { + dataPrepperTestRunner.stop(); + } + + @Test + void pipeline_with_single_batch_of_records() { + final int recordsToCreate = 200; + final List> inputRecords = IntStream.range(0, recordsToCreate) + .mapToObj(i -> UUID.randomUUID().toString()) + .map(JacksonEvent::fromMessage) + .map(Record::new) + .collect(Collectors.toList()); + + inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, inputRecords); + + await().atMost(800, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER_ENTRY_SINK), not(empty())); + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER_EXIT_SINK), not(empty())); + }); + + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER_ENTRY_SINK).size(), equalTo(recordsToCreate)); + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER_EXIT_SINK).size(), equalTo(recordsToCreate)); + } + + @Test + void pipeline_with_multiple_batches_of_records() { + final int recordsToCreateBatch1 = 200; + final List> inputRecordsBatch1 = IntStream.range(0, recordsToCreateBatch1) + .mapToObj(i -> UUID.randomUUID().toString()) + .map(JacksonEvent::fromMessage) + .map(Record::new) + .collect(Collectors.toList()); + + inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, inputRecordsBatch1); + + await().atMost(800, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER_ENTRY_SINK), not(empty())); + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER_EXIT_SINK), not(empty())); + }); + + assertThat(inMemorySinkAccessor.getAndClear(IN_MEMORY_IDENTIFIER_ENTRY_SINK).size(), equalTo(recordsToCreateBatch1)); + assertThat(inMemorySinkAccessor.getAndClear(IN_MEMORY_IDENTIFIER_EXIT_SINK).size(), equalTo(recordsToCreateBatch1)); + + final int recordsToCreateBatch2 = 300; + final List> inputRecordsBatch2 = IntStream.range(0, recordsToCreateBatch2) + .mapToObj(i -> UUID.randomUUID().toString()) + .map(JacksonEvent::fromMessage) + .map(Record::new) + .collect(Collectors.toList()); + + inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, inputRecordsBatch2); + + await().atMost(400, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER_ENTRY_SINK), not(empty())); + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER_EXIT_SINK), not(empty())); + }); + + assertThat(inMemorySinkAccessor.getAndClear(IN_MEMORY_IDENTIFIER_ENTRY_SINK).size(), equalTo(recordsToCreateBatch2)); + assertThat(inMemorySinkAccessor.getAndClear(IN_MEMORY_IDENTIFIER_EXIT_SINK).size(), equalTo(recordsToCreateBatch2)); + } + +} diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Connected_SingleIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Connected_SingleIT.java new file mode 100644 index 0000000000..15588165f6 --- /dev/null +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/Connected_SingleIT.java @@ -0,0 +1,133 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.integration; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.InMemorySinkAccessor; +import org.opensearch.dataprepper.plugins.InMemorySourceAccessor; +import org.opensearch.dataprepper.test.framework.DataPrepperTestRunner; + +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.awaitility.Awaitility.await; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.empty; + +public class Connected_SingleIT { + private static final String IN_MEMORY_IDENTIFIER = "Connected_SingleIT"; + private static final String PIPELINE_CONFIGURATION_UNDER_TEST = "connected/single-connection.yaml"; + private DataPrepperTestRunner dataPrepperTestRunner; + private InMemorySourceAccessor inMemorySourceAccessor; + private InMemorySinkAccessor inMemorySinkAccessor; + + @BeforeEach + void setUp() { + dataPrepperTestRunner = DataPrepperTestRunner.builder() + .withPipelinesDirectoryOrFile(PIPELINE_CONFIGURATION_UNDER_TEST) + .build(); + + dataPrepperTestRunner.start(); + inMemorySourceAccessor = dataPrepperTestRunner.getInMemorySourceAccessor(); + inMemorySinkAccessor = dataPrepperTestRunner.getInMemorySinkAccessor(); + } + + @AfterEach + void tearDown() { + dataPrepperTestRunner.stop(); + } + + @Test + void pipeline_with_no_data() throws InterruptedException { + final List> preRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); + assertThat(preRecords, is(empty())); + + Thread.sleep(1400); + + final List> postRecords = inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER); + assertThat(postRecords, is(empty())); + } + + @Test + void pipeline_with_single_record() { + final Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + final Record eventRecord = new Record<>(event); + + inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, Collections.singletonList(eventRecord)); + + await().atMost(800, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER), not(empty())); + }); + + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER).size(), equalTo(1)); + } + + @Test + void pipeline_with_single_batch_of_records() { + final int recordsToCreate = 200; + final List> inputRecords = IntStream.range(0, recordsToCreate) + .mapToObj(i -> UUID.randomUUID().toString()) + .map(JacksonEvent::fromMessage) + .map(Record::new) + .collect(Collectors.toList()); + + inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, inputRecords); + + await().atMost(800, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER).size(), equalTo(recordsToCreate)); + }); + } + + @Test + void pipeline_with_multiple_batches_of_records() { + final int recordsToCreateBatch1 = 200; + final List> inputRecordsBatch1 = IntStream.range(0, recordsToCreateBatch1) + .mapToObj(i -> UUID.randomUUID().toString()) + .map(JacksonEvent::fromMessage) + .map(Record::new) + .collect(Collectors.toList()); + + inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, inputRecordsBatch1); + + await().atMost(800, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER).size(), equalTo(recordsToCreateBatch1)); + }); + + assertThat(inMemorySinkAccessor.getAndClear(IN_MEMORY_IDENTIFIER).size(), equalTo(recordsToCreateBatch1)); + + final int recordsToCreateBatch2 = 300; + final List> inputRecordsBatch2 = IntStream.range(0, recordsToCreateBatch2) + .mapToObj(i -> UUID.randomUUID().toString()) + .map(JacksonEvent::fromMessage) + .map(Record::new) + .collect(Collectors.toList()); + + inMemorySourceAccessor.submit(IN_MEMORY_IDENTIFIER, inputRecordsBatch2); + + await().atMost(400, TimeUnit.MILLISECONDS) + .untilAsserted(() -> { + assertThat(inMemorySinkAccessor.get(IN_MEMORY_IDENTIFIER).size(), equalTo(recordsToCreateBatch2)); + }); + + assertThat(inMemorySinkAccessor.getAndClear(IN_MEMORY_IDENTIFIER).size(), equalTo(recordsToCreateBatch2)); + } + +} diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/CoreHttpServerIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/CoreHttpServerIT.java index 1132c01ac2..66ea915e71 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/CoreHttpServerIT.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/integration/CoreHttpServerIT.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.integration; import com.linecorp.armeria.client.WebClient; +import com.linecorp.armeria.common.AggregatedHttpResponse; import com.linecorp.armeria.common.HttpMethod; import com.linecorp.armeria.common.HttpStatus; import com.linecorp.armeria.common.RequestHeaders; @@ -13,18 +14,17 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.opensearch.dataprepper.plugins.InMemorySinkAccessor; -import org.opensearch.dataprepper.plugins.InMemorySourceAccessor; import org.opensearch.dataprepper.test.framework.DataPrepperTestRunner; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; class CoreHttpServerIT { + private static final Logger log = LoggerFactory.getLogger(CoreHttpServerIT.class); private static final String PIPELINE_CONFIGURATION_UNDER_TEST = "minimal-pipeline.yaml"; private DataPrepperTestRunner dataPrepperTestRunner; - private InMemorySourceAccessor inMemorySourceAccessor; - private InMemorySinkAccessor inMemorySinkAccessor; @BeforeEach void setUp() { @@ -33,8 +33,6 @@ void setUp() { .build(); dataPrepperTestRunner.start(); - inMemorySourceAccessor = dataPrepperTestRunner.getInMemorySourceAccessor(); - inMemorySinkAccessor = dataPrepperTestRunner.getInMemorySinkAccessor(); } @AfterEach @@ -44,17 +42,16 @@ void tearDown() { @Test void verify_list_api_is_running() { - WebClient.of().execute(RequestHeaders.builder() + log.info("Making API request for test."); + final AggregatedHttpResponse response = WebClient.of().execute(RequestHeaders.builder() .scheme(SessionProtocol.HTTP) .authority("127.0.0.1:4900") .method(HttpMethod.GET) .path("/list") .build()) .aggregate() - .whenComplete((response, ex) -> { - assertThat("Http Status", response.status(), equalTo(HttpStatus.OK)); - }) .join(); - } + assertThat(response.status(), equalTo(HttpStatus.OK)); + } } diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySource.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySource.java index 3afd17554c..f95e51e7f3 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySource.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/InMemorySource.java @@ -68,7 +68,11 @@ public boolean areAcknowledgementsEnabled() { @Override public void stop() { isStopped = true; - runningThread.interrupt(); + try { + runningThread.join(1000); + } catch (final InterruptedException e) { + runningThread.interrupt(); + } } private class SourceRunner implements Runnable { diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/test/framework/DataPrepperTestRunner.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/test/framework/DataPrepperTestRunner.java index abf04cfe24..976b0dce66 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/test/framework/DataPrepperTestRunner.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/test/framework/DataPrepperTestRunner.java @@ -66,6 +66,7 @@ public void start() { public void stop() { final DataPrepper dataPrepper = contextManager.getDataPrepperBean(); dataPrepper.shutdown(); + contextManager.shutdown(); } /** diff --git a/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/connected/single-connection-extra-sink.yaml b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/connected/single-connection-extra-sink.yaml new file mode 100644 index 0000000000..c08afe4168 --- /dev/null +++ b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/connected/single-connection-extra-sink.yaml @@ -0,0 +1,19 @@ +entry-pipeline: + delay: 5 + source: + in_memory: + testing_key: Connected_SingleExtraSinkIT + sink: + - pipeline: + name: exit-pipeline + - in_memory: + testing_key: Connected_SingleExtraSinkIT_Entry + +exit-pipeline: + delay: 5 + source: + pipeline: + name: entry-pipeline + sink: + - in_memory: + testing_key: Connected_SingleExtraSinkIT_Exit diff --git a/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/connected/single-connection.yaml b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/connected/single-connection.yaml new file mode 100644 index 0000000000..45093f1fdf --- /dev/null +++ b/data-prepper-core/src/integrationTest/resources/org/opensearch/dataprepper/pipeline/connected/single-connection.yaml @@ -0,0 +1,17 @@ +entry-pipeline: + delay: 5 + source: + in_memory: + testing_key: Connected_SingleIT + sink: + - pipeline: + name: exit-pipeline + +exit-pipeline: + delay: 5 + source: + pipeline: + name: entry-pipeline + sink: + - in_memory: + testing_key: Connected_SingleIT 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 c7e4e75891..29bb69db46 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 @@ -54,6 +54,7 @@ @SuppressWarnings({"rawtypes", "unchecked"}) public class Pipeline { private static final Logger LOG = LoggerFactory.getLogger(Pipeline.class); + private static final int SINK_LOGGING_FREQUENCY = (int) Duration.ofSeconds(60).toMillis(); private volatile AtomicBoolean stopRequested; private final String name; @@ -249,12 +250,13 @@ public void execute() { sinkExecutorService.submit(() -> { long retryCount = 0; + final long sleepIfNotReadyTime = 200; while (!isReady() && !isStopRequested()) { - if (retryCount++ % 60 == 0) { + if (retryCount++ % (SINK_LOGGING_FREQUENCY / sleepIfNotReadyTime) == 0) { LOG.info("Pipeline [{}] Waiting for Sink to be ready", name); } try { - Thread.sleep(1000); + Thread.sleep(sleepIfNotReadyTime); } catch (Exception e){} } startSourceAndProcessors(); diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/server/DataPrepperServer.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/server/DataPrepperServer.java index 8c2a0f1367..3158cde82c 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/server/DataPrepperServer.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/server/DataPrepperServer.java @@ -33,8 +33,8 @@ public class DataPrepperServer { private final ShutdownHandler shutdownHandler; private final PrometheusMeterRegistry prometheusMeterRegistry; private final Authenticator authenticator; + private final ExecutorService executorService; private HttpServer server; - static final ExecutorService EXECUTOR_SERVICE = Executors.newFixedThreadPool(3); @Inject public DataPrepperServer( @@ -49,6 +49,7 @@ public DataPrepperServer( this.shutdownHandler = shutdownHandler; this.prometheusMeterRegistry = prometheusMeterRegistry; this.authenticator = authenticator; + executorService = Executors.newFixedThreadPool(3); } /** @@ -56,7 +57,7 @@ public DataPrepperServer( */ public void start() { server = createServer(); - server.setExecutor(EXECUTOR_SERVICE); + server.setExecutor(executorService); server.start(); LOG.info("Data Prepper server running at :{}", server.getAddress().getPort()); } @@ -95,7 +96,7 @@ private void createContext( */ public void stop() { server.stop(0); - EXECUTOR_SERVICE.shutdownNow(); + executorService.shutdownNow(); LOG.info("Data Prepper server stopped"); } } diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/server/ListPipelinesHandler.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/server/ListPipelinesHandler.java index 72c5ffab7e..ab933b6c69 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/server/ListPipelinesHandler.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/pipeline/server/ListPipelinesHandler.java @@ -55,6 +55,7 @@ public void handle(final HttpExchange exchange) throws IOException { .stream() .map(PipelineListing::new) .collect(Collectors.toList()); + LOG.debug("List pipelines request responding with {} pipelines.", pipelines.size()); final byte[] response = OBJECT_MAPPER.writeValueAsString(Collections.singletonMap("pipelines", pipelines)).getBytes(); exchange.getResponseHeaders().add("Content-Type", "text/plain; charset=UTF-8"); exchange.sendResponseHeaders(HttpURLConnection.HTTP_OK, response.length); diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/DataPrepperServerTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/DataPrepperServerTest.java index 4b60282f37..f82ffc2660 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/DataPrepperServerTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/pipeline/server/DataPrepperServerTest.java @@ -13,12 +13,17 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import java.net.InetSocketAddress; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ThreadPoolExecutor; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; import static org.mockito.ArgumentMatchers.any; @@ -140,7 +145,13 @@ private void verifyServerStart() { verify(httpServerProvider).get(); verify(server).createContext("/list", listPipelinesHandler); verify(server).createContext(eq("/shutdown"), eq(shutdownHandler)); - verify(server).setExecutor(DataPrepperServer.EXECUTOR_SERVICE); + final ArgumentCaptor executorServiceArgumentCaptor = ArgumentCaptor.forClass(ExecutorService.class); + verify(server).setExecutor(executorServiceArgumentCaptor.capture()); + final ExecutorService actualExecutorService = executorServiceArgumentCaptor.getValue(); + assertThat(actualExecutorService, instanceOf(ThreadPoolExecutor.class)); + final ThreadPoolExecutor threadPoolExecutor = (ThreadPoolExecutor) actualExecutorService; + assertThat(threadPoolExecutor.getMaximumPoolSize(), greaterThanOrEqualTo(3)); + verify(server).start(); verify(server).getAddress(); verify(socketAddress).getPort(); From 6839492987994f7cf78d75928fb5e298f0c79eb1 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka <41027584+kkondaka@users.noreply.github.com> Date: Fri, 23 Aug 2024 11:35:50 -0700 Subject: [PATCH 130/159] Change version in DataPrepper Version class to 2.10 (#4852) * dplive1.yaml Signed-off-by: Kondaka * Delete .github/workflows/static.yml Signed-off-by: Kondaka * Change Data Prepper Version to 2.10 Signed-off-by: Kondaka --------- Signed-off-by: Kondaka --- .../dataprepper/model/configuration/DataPrepperVersion.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/DataPrepperVersion.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/DataPrepperVersion.java index 738255b650..f8e996f8d1 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/DataPrepperVersion.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/configuration/DataPrepperVersion.java @@ -6,7 +6,7 @@ import java.util.regex.Pattern; public class DataPrepperVersion { - private static final String CURRENT_VERSION = "2.9"; + private static final String CURRENT_VERSION = "2.10"; private static final String FULL_FORMAT = "%d.%d"; private static final String SHORTHAND_FORMAT = "%d"; From 910533abb35d4fd7e8ad7a64e358fa1140f24e6f Mon Sep 17 00:00:00 2001 From: Daniel Li <68623003+danhli@users.noreply.github.com> Date: Fri, 23 Aug 2024 11:59:12 -0700 Subject: [PATCH 131/159] Fix visibility timeout errors (#4812) (#4831) Fix visibility timeout errors (#4812) Signed-off-by: Daniel Li --- .../plugins/source/s3/SqsWorker.java | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java index 3c5fba0701..2861ffa6d7 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java @@ -192,6 +192,8 @@ private ReceiveMessageRequest createReceiveMessageRequest() { private List processS3EventNotificationRecords(final Collection s3EventNotificationRecords) { final List deleteMessageBatchRequestEntryCollection = new ArrayList<>(); final List parsedMessagesToRead = new ArrayList<>(); + final Map messageAcknowledgementSetMap = new HashMap<>(); + final Map> messageWaitingForAcknowledgementsMap = new HashMap<>(); for (ParsedMessage parsedMessage : s3EventNotificationRecords) { if (parsedMessage.isFailedParsing()) { @@ -224,7 +226,7 @@ && isEventBridgeEventTypeCreated(parsedMessage)) { } LOG.info("Received {} messages from SQS. Processing {} messages.", s3EventNotificationRecords.size(), parsedMessagesToRead.size()); - + for (ParsedMessage parsedMessage : parsedMessagesToRead) { List waitingForAcknowledgements = new ArrayList<>(); AcknowledgementSet acknowledgementSet = null; @@ -262,7 +264,19 @@ && isEventBridgeEventTypeCreated(parsedMessage)) { }, Duration.ofSeconds(progressCheckInterval)); } + messageAcknowledgementSetMap.put(parsedMessage, acknowledgementSet); + messageWaitingForAcknowledgementsMap.put(parsedMessage, waitingForAcknowledgements); } + } + + if (endToEndAcknowledgementsEnabled) { + LOG.debug("Created acknowledgement sets for {} messages.", parsedMessagesToRead.size()); + } + + // Use a separate loop for processing the S3 objects + for (ParsedMessage parsedMessage : parsedMessagesToRead) { + final AcknowledgementSet acknowledgementSet = messageAcknowledgementSetMap.get(parsedMessage); + final List waitingForAcknowledgements = messageWaitingForAcknowledgementsMap.get(parsedMessage); final S3ObjectReference s3ObjectReference = populateS3Reference(parsedMessage.getBucketName(), parsedMessage.getObjectKey()); final Optional deleteMessageBatchRequestEntry = processS3Object(parsedMessage, s3ObjectReference, acknowledgementSet); if (endToEndAcknowledgementsEnabled) { @@ -271,7 +285,7 @@ && isEventBridgeEventTypeCreated(parsedMessage)) { } else { deleteMessageBatchRequestEntry.ifPresent(deleteMessageBatchRequestEntryCollection::add); } - } + } return deleteMessageBatchRequestEntryCollection; } From 6ca73cd634fc106cb46defed3d75cd6a73281c99 Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 23 Aug 2024 16:49:10 -0500 Subject: [PATCH 132/159] Removes the trace-analytics-sample-app from the examples that are provided in the release. (#4867) The motivation for removing this is that the samples pull in dependencies which often trigger CVE reports. It is not likely customers are trying to run this example from a Data Prepper deployment, especially since the example is not made to run from the installed version, but runs from Docker and runs using the latest 2.x version. Signed-off-by: David Venable --- release/archives/build.gradle | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/release/archives/build.gradle b/release/archives/build.gradle index 327a84ec16..b7d751c8b9 100644 --- a/release/archives/build.gradle +++ b/release/archives/build.gradle @@ -154,7 +154,9 @@ CopySpec archiveToTar() { fileMode 0755 } into('examples') { - from("${rootDir}/examples") + from("${rootDir}/examples") { + exclude 'trace-analytics-sample-app' + } dirMode 0750 fileMode 0755 } From 49e4b07514e9c9f1b834855c0fae1b6891255cde Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 23 Aug 2024 16:49:17 -0500 Subject: [PATCH 133/159] Require dnsjava 3.6.1 to resolve CVE-2024-25638. This is a transitive dependency from Hadoop. (#4868) Signed-off-by: David Venable --- build.gradle | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/build.gradle b/build.gradle index 3dccd497cf..83323ae3f0 100644 --- a/build.gradle +++ b/build.gradle @@ -79,6 +79,12 @@ subprojects { testImplementation testLibs.hamcrest testImplementation testLibs.awaitility constraints { + implementation('dnsjava:dnsjava') { + version { + require '3.6.1' + } + because 'Fixes CVE-2023-39410.' + } implementation('org.apache.avro:avro') { version { require '1.11.3' From 0a26f59fe062f8afa9f28a447206884b6a2e1e9a Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 26 Aug 2024 10:30:35 -0500 Subject: [PATCH 134/159] Updates Bouncy Castle to 1.78.1. Update any projects that attempt to use Bouncy Castle jdk15on dependencies with the jdk18on dependency instead. This will prevent any of the older jdk15on dependencies from getting into our classpath. In particular, this was coming from hadoop-common. (#4871) Signed-off-by: David Venable --- build.gradle | 4 ++++ settings.gradle | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 83323ae3f0..10c8e02b7a 100644 --- a/build.gradle +++ b/build.gradle @@ -226,6 +226,10 @@ subprojects { } } else if (details.requested.group == 'log4j' && details.requested.name == 'log4j') { details.useTarget group: 'org.apache.logging.log4j', name: 'log4j-1.2-api', version: '2.17.1' + } else if (details.requested.group == 'org.bouncycastle' && details.requested.name.endsWith('-jdk15on')) { + def java8Name = details.requested.name.replace('-jdk15on', '-jdk18on') + details.useTarget group: 'org.bouncycastle', name: java8Name, version: libs.bouncycastle.bcprov.get().version + details.because 'Use only the Java 8 artifacts of BouncyCastle' } } } diff --git a/settings.gradle b/settings.gradle index 20edb79287..8ca5760be3 100644 --- a/settings.gradle +++ b/settings.gradle @@ -49,7 +49,7 @@ dependencyResolutionManagement { version('spring', '5.3.28') library('spring-core', 'org.springframework', 'spring-core').versionRef('spring') library('spring-context', 'org.springframework', 'spring-context').versionRef('spring') - version('bouncycastle', '1.76') + version('bouncycastle', '1.78.1') library('bouncycastle-bcprov', 'org.bouncycastle', 'bcprov-jdk18on').versionRef('bouncycastle') library('bouncycastle-bcpkix', 'org.bouncycastle', 'bcpkix-jdk18on').versionRef('bouncycastle') version('guava', '32.1.2-jre') From 92448188ad2f975ca1b3c12cd89872453839e794 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 26 Aug 2024 12:34:06 -0500 Subject: [PATCH 135/159] Update Spring to 5.3.39 to fix CVE-2024-38808. Require commons-configuration2 2.11.0 to fix CVE-2024-29131 and CVE-2024-29133. Hadoop pulls this dependency in. (#4874) Signed-off-by: David Venable --- build.gradle | 6 ++++++ settings.gradle | 2 +- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 10c8e02b7a..0efa56a43b 100644 --- a/build.gradle +++ b/build.gradle @@ -91,6 +91,12 @@ subprojects { } because 'Fixes CVE-2023-39410.' } + implementation('org.apache.commons:commons-configuration2') { + version { + require '2.11.0' + } + because 'Fixes CVE-2024-29131 and CVE-2024-29133.' + } implementation('org.apache.httpcomponents:httpclient') { version { require '4.5.14' diff --git a/settings.gradle b/settings.gradle index 8ca5760be3..3524edaee3 100644 --- a/settings.gradle +++ b/settings.gradle @@ -46,7 +46,7 @@ dependencyResolutionManagement { version('opensearch', '1.3.14') library('opensearch-client', 'org.opensearch.client', 'opensearch-rest-client').versionRef('opensearch') library('opensearch-rhlc', 'org.opensearch.client', 'opensearch-rest-high-level-client').versionRef('opensearch') - version('spring', '5.3.28') + version('spring', '5.3.39') library('spring-core', 'org.springframework', 'spring-core').versionRef('spring') library('spring-context', 'org.springframework', 'spring-context').versionRef('spring') version('bouncycastle', '1.78.1') From d6465efc63f98380605b90ddeab90343bb469dbf Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Mon, 26 Aug 2024 17:37:22 -0500 Subject: [PATCH 136/159] Add e2e acknowledgment and checkpointing to RDS source (#4819) * Add acknowledgment and checkpoint for stream Signed-off-by: Hai Yan * Add unit tests for stream checkpoint Signed-off-by: Hai Yan * Add acknowledgment to export Signed-off-by: Hai Yan * Simplify the stream checkpointing workflow Signed-off-by: Hai Yan * A few fixes and cleanups Signed-off-by: Hai Yan * Extend lease while waiting for ack Signed-off-by: Hai Yan * Address review comments Signed-off-by: Hai Yan * Address more review comments Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- .../plugins/source/rds/RdsService.java | 11 +- .../plugins/source/rds/RdsSource.java | 13 +- .../plugins/source/rds/RdsSourceConfig.java | 15 ++ .../state/StreamProgressState.java | 11 - .../source/rds/export/DataFileLoader.java | 60 +++-- .../source/rds/export/DataFileScheduler.java | 76 +++++-- .../source/rds/leader/LeaderScheduler.java | 6 +- .../source/rds/model/BinlogCoordinate.java | 8 + .../rds/stream/BinlogEventListener.java | 205 ++++++++++-------- .../source/rds/stream/ChangeEventStatus.java | 51 +++++ .../rds/stream/StreamCheckpointManager.java | 133 ++++++++++++ .../source/rds/stream/StreamCheckpointer.java | 58 +++++ .../source/rds/stream/StreamScheduler.java | 22 +- .../source/rds/stream/StreamWorker.java | 2 +- .../plugins/source/rds/RdsServiceTest.java | 6 +- .../plugins/source/rds/RdsSourceTest.java | 6 +- .../source/rds/export/DataFileLoaderTest.java | 56 +++-- .../rds/export/DataFileSchedulerTest.java | 23 +- .../rds/stream/BinlogEventListenerTest.java | 13 +- .../stream/StreamCheckpointManagerTest.java | 104 +++++++++ .../rds/stream/StreamCheckpointerTest.java | 83 +++++++ .../rds/stream/StreamSchedulerTest.java | 11 +- .../source/rds/stream/StreamWorkerTest.java | 4 +- 23 files changed, 790 insertions(+), 187 deletions(-) create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ChangeEventStatus.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManager.java create mode 100644 data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManagerTest.java create mode 100644 data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index 005fee88fd..48d6859ba4 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -8,6 +8,7 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import com.github.shyiko.mysql.binlog.network.SSLMode; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventFactory; @@ -50,6 +51,7 @@ public class RdsService { private final EventFactory eventFactory; private final PluginMetrics pluginMetrics; private final RdsSourceConfig sourceConfig; + private final AcknowledgementSetManager acknowledgementSetManager; private ExecutorService executor; private LeaderScheduler leaderScheduler; private ExportScheduler exportScheduler; @@ -60,11 +62,13 @@ public RdsService(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig, final EventFactory eventFactory, final ClientFactory clientFactory, - final PluginMetrics pluginMetrics) { + final PluginMetrics pluginMetrics, + final AcknowledgementSetManager acknowledgementSetManager) { this.sourceCoordinator = sourceCoordinator; this.eventFactory = eventFactory; this.pluginMetrics = pluginMetrics; this.sourceConfig = sourceConfig; + this.acknowledgementSetManager = acknowledgementSetManager; rdsClient = clientFactory.buildRdsClient(); s3Client = clientFactory.buildS3Client(); @@ -94,7 +98,7 @@ public void start(Buffer> buffer) { exportScheduler = new ExportScheduler( sourceCoordinator, snapshotManager, exportTaskManager, s3Client, pluginMetrics); dataFileScheduler = new DataFileScheduler( - sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer, pluginMetrics); + sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer, pluginMetrics, acknowledgementSetManager); runnableList.add(exportScheduler); runnableList.add(dataFileScheduler); } @@ -106,7 +110,8 @@ public void start(Buffer> buffer) { } else { binaryLogClient.setSSLMode(SSLMode.DISABLED); } - streamScheduler = new StreamScheduler(sourceCoordinator, sourceConfig, binaryLogClient, buffer, pluginMetrics); + streamScheduler = new StreamScheduler( + sourceCoordinator, sourceConfig, binaryLogClient, buffer, pluginMetrics, acknowledgementSetManager); runnableList.add(streamScheduler); } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java index 071fc5889b..5c2b08fb5c 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSource.java @@ -7,6 +7,7 @@ import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; import org.opensearch.dataprepper.model.buffer.Buffer; @@ -35,6 +36,7 @@ public class RdsSource implements Source>, UsesEnhancedSourceCoord private final PluginMetrics pluginMetrics; private final RdsSourceConfig sourceConfig; private final EventFactory eventFactory; + private final AcknowledgementSetManager acknowledgementSetManager; private EnhancedSourceCoordinator sourceCoordinator; private RdsService rdsService; @@ -42,10 +44,12 @@ public class RdsSource implements Source>, UsesEnhancedSourceCoord public RdsSource(final PluginMetrics pluginMetrics, final RdsSourceConfig sourceConfig, final EventFactory eventFactory, - final AwsCredentialsSupplier awsCredentialsSupplier) { + final AwsCredentialsSupplier awsCredentialsSupplier, + final AcknowledgementSetManager acknowledgementSetManager) { this.pluginMetrics = pluginMetrics; this.sourceConfig = sourceConfig; this.eventFactory = eventFactory; + this.acknowledgementSetManager = acknowledgementSetManager; clientFactory = new ClientFactory(awsCredentialsSupplier, sourceConfig.getAwsAuthenticationConfig()); } @@ -56,7 +60,7 @@ public void start(Buffer> buffer) { Objects.requireNonNull(sourceCoordinator); sourceCoordinator.createPartition(new LeaderPartition()); - rdsService = new RdsService(sourceCoordinator, sourceConfig, eventFactory, clientFactory, pluginMetrics); + rdsService = new RdsService(sourceCoordinator, sourceConfig, eventFactory, clientFactory, pluginMetrics, acknowledgementSetManager); LOG.info("Start RDS service"); rdsService.start(buffer); @@ -80,4 +84,9 @@ public void setEnhancedSourceCoordinator(EnhancedSourceCoordinator sourceCoordin public Function getPartitionFactory() { return new PartitionFactory(); } + + @Override + public boolean areAcknowledgementsEnabled() { + return sourceConfig.isAcknowledgmentsEnabled(); + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java index 548dc4a2fb..a1cb8c7e2f 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java @@ -14,6 +14,7 @@ import org.opensearch.dataprepper.plugins.source.rds.configuration.StreamConfig; import org.opensearch.dataprepper.plugins.source.rds.configuration.TlsConfig; +import java.time.Duration; import java.util.List; /** @@ -56,6 +57,12 @@ public class RdsSourceConfig { @JsonProperty("acknowledgments") private boolean acknowledgments = false; + @JsonProperty("s3_data_file_acknowledgment_timeout") + private Duration dataFileAcknowledgmentTimeout = Duration.ofMinutes(30); + + @JsonProperty("stream_acknowledgment_timeout") + private Duration streamAcknowledgmentTimeout = Duration.ofMinutes(10); + @JsonProperty("s3_bucket") private String s3Bucket; @@ -106,6 +113,14 @@ public boolean isAcknowledgmentsEnabled() { return acknowledgments; } + public Duration getDataFileAcknowledgmentTimeout() { + return dataFileAcknowledgmentTimeout; + } + + public Duration getStreamAcknowledgmentTimeout() { + return streamAcknowledgmentTimeout; + } + public String getS3Bucket() { return s3Bucket; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java index 21873179da..81a3c7f5ac 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/StreamProgressState.java @@ -10,23 +10,12 @@ public class StreamProgressState { - @JsonProperty("startPosition") - private BinlogCoordinate startPosition; - @JsonProperty("currentPosition") private BinlogCoordinate currentPosition; @JsonProperty("waitForExport") private boolean waitForExport = false; - public BinlogCoordinate getStartPosition() { - return startPosition; - } - - public void setStartPosition(BinlogCoordinate startPosition) { - this.startPosition = startPosition; - } - public BinlogCoordinate getCurrentPosition() { return currentPosition; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java index c6815e37b7..64c613bc43 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -9,9 +9,12 @@ import io.micrometer.core.instrument.DistributionSummary; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.DataFileProgressState; @@ -30,6 +33,8 @@ public class DataFileLoader implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(DataFileLoader.class); static final Duration VERSION_OVERLAP_TIME_FOR_EXPORT = Duration.ofMinutes(5); + static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); + static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; static final String EXPORT_RECORDS_TOTAL_COUNT = "exportRecordsTotal"; static final String EXPORT_RECORDS_PROCESSED_COUNT = "exportRecordsProcessed"; static final String EXPORT_RECORDS_PROCESSING_ERROR_COUNT = "exportRecordsProcessingErrors"; @@ -41,8 +46,11 @@ public class DataFileLoader implements Runnable { private final String objectKey; private final S3ObjectReader objectReader; private final InputCodec codec; - private final BufferAccumulator> bufferAccumulator; + private final Buffer> buffer; private final ExportRecordConverter recordConverter; + private final EnhancedSourceCoordinator sourceCoordinator; + private final AcknowledgementSet acknowledgementSet; + private final Duration acknowledgmentTimeout; private final Counter exportRecordsTotalCounter; private final Counter exportRecordSuccessCounter; private final Counter exportRecordErrorCounter; @@ -51,17 +59,23 @@ public class DataFileLoader implements Runnable { private DataFileLoader(final DataFilePartition dataFilePartition, final InputCodec codec, - final BufferAccumulator> bufferAccumulator, + final Buffer> buffer, final S3ObjectReader objectReader, final ExportRecordConverter recordConverter, - final PluginMetrics pluginMetrics) { + final PluginMetrics pluginMetrics, + final EnhancedSourceCoordinator sourceCoordinator, + final AcknowledgementSet acknowledgementSet, + final Duration acknowledgmentTimeout) { this.dataFilePartition = dataFilePartition; bucket = dataFilePartition.getBucket(); objectKey = dataFilePartition.getKey(); this.objectReader = objectReader; this.codec = codec; - this.bufferAccumulator = bufferAccumulator; + this.buffer = buffer; this.recordConverter = recordConverter; + this.sourceCoordinator = sourceCoordinator; + this.acknowledgementSet = acknowledgementSet; + this.acknowledgmentTimeout = acknowledgmentTimeout; exportRecordsTotalCounter = pluginMetrics.counter(EXPORT_RECORDS_TOTAL_COUNT); exportRecordSuccessCounter = pluginMetrics.counter(EXPORT_RECORDS_PROCESSED_COUNT); @@ -72,17 +86,23 @@ private DataFileLoader(final DataFilePartition dataFilePartition, public static DataFileLoader create(final DataFilePartition dataFilePartition, final InputCodec codec, - final BufferAccumulator> bufferAccumulator, + final Buffer> buffer, final S3ObjectReader objectReader, final ExportRecordConverter recordConverter, - final PluginMetrics pluginMetrics) { - return new DataFileLoader(dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter, pluginMetrics); + final PluginMetrics pluginMetrics, + final EnhancedSourceCoordinator sourceCoordinator, + final AcknowledgementSet acknowledgementSet, + final Duration acknowledgmentTimeout) { + return new DataFileLoader(dataFilePartition, codec, buffer, objectReader, recordConverter, + pluginMetrics, sourceCoordinator, acknowledgementSet, acknowledgmentTimeout); } @Override public void run() { LOG.info(SENSITIVE, "Start loading s3://{}/{}", bucket, objectKey); + final BufferAccumulator> bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); + AtomicLong eventCount = new AtomicLong(); try (InputStream inputStream = objectReader.readFile(bucket, objectKey)) { codec.parse(inputStream, record -> { @@ -100,15 +120,19 @@ public void run() { final long snapshotTime = progressState.getSnapshotTime(); final long eventVersionNumber = snapshotTime - VERSION_OVERLAP_TIME_FOR_EXPORT.toMillis(); - Record transformedRecord = new Record<>( - recordConverter.convert( - record, - progressState.getSourceDatabase(), - progressState.getSourceTable(), - primaryKeys, - snapshotTime, - eventVersionNumber)); - bufferAccumulator.add(transformedRecord); + final Event transformedEvent = recordConverter.convert( + record, + progressState.getSourceDatabase(), + progressState.getSourceTable(), + primaryKeys, + snapshotTime, + eventVersionNumber); + + if (acknowledgementSet != null) { + acknowledgementSet.add(transformedEvent); + } + + bufferAccumulator.add(new Record<>(transformedEvent)); eventCount.getAndIncrement(); bytesProcessedSummary.record(bytes); } catch (Exception e) { @@ -125,6 +149,10 @@ public void run() { try { bufferAccumulator.flush(); + if (acknowledgementSet != null) { + sourceCoordinator.saveProgressStateForPartition(dataFilePartition, acknowledgmentTimeout); + acknowledgementSet.complete(); + } exportRecordSuccessCounter.increment(eventCount.get()); } catch (Exception e) { LOG.error("Failed to write events to buffer", e); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java index f766aec3d2..33c17d9d80 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileScheduler.java @@ -6,8 +6,9 @@ package org.opensearch.dataprepper.plugins.source.rds.export; import io.micrometer.core.instrument.Counter; -import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; @@ -32,6 +33,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; import static org.opensearch.dataprepper.plugins.source.rds.RdsService.DATA_LOADER_MAX_JOB_COUNT; @@ -48,8 +50,7 @@ public class DataFileScheduler implements Runnable { private static final Duration DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT = Duration.ofMinutes(30); - static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); - static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; + static final String EXPORT_S3_OBJECTS_PROCESSED_COUNT = "exportS3ObjectsProcessed"; static final String EXPORT_S3_OBJECTS_ERROR_COUNT = "exportS3ObjectsErrors"; static final String ACTIVE_EXPORT_S3_OBJECT_CONSUMERS_GAUGE = "activeExportS3ObjectConsumers"; @@ -60,9 +61,10 @@ public class DataFileScheduler implements Runnable { private final RdsSourceConfig sourceConfig; private final S3ObjectReader objectReader; private final InputCodec codec; - private final BufferAccumulator> bufferAccumulator; private final ExportRecordConverter recordConverter; + private final Buffer> buffer; private final PluginMetrics pluginMetrics; + private final AcknowledgementSetManager acknowledgementSetManager; private final Counter exportFileSuccessCounter; private final Counter exportFileErrorCounter; @@ -75,15 +77,17 @@ public DataFileScheduler(final EnhancedSourceCoordinator sourceCoordinator, final S3Client s3Client, final EventFactory eventFactory, final Buffer> buffer, - final PluginMetrics pluginMetrics) { + final PluginMetrics pluginMetrics, + final AcknowledgementSetManager acknowledgementSetManager) { this.sourceCoordinator = sourceCoordinator; this.sourceConfig = sourceConfig; codec = new ParquetInputCodec(eventFactory); - bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); objectReader = new S3ObjectReader(s3Client); recordConverter = new ExportRecordConverter(); executor = Executors.newFixedThreadPool(DATA_LOADER_MAX_JOB_COUNT); + this.buffer = buffer; this.pluginMetrics = pluginMetrics; + this.acknowledgementSetManager = acknowledgementSetManager; this.exportFileSuccessCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_PROCESSED_COUNT); this.exportFileErrorCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_ERROR_COUNT); @@ -133,23 +137,39 @@ public void shutdown() { } private void processDataFilePartition(DataFilePartition dataFilePartition) { + // Create AcknowledgmentSet + final boolean isAcknowledgmentsEnabled = sourceConfig.isAcknowledgmentsEnabled(); + AcknowledgementSet acknowledgementSet = null; + if (sourceConfig.isAcknowledgmentsEnabled()) { + acknowledgementSet = acknowledgementSetManager.create((result) -> { + if (result) { + completeDataLoader(dataFilePartition).accept(null, null); + LOG.info("Received acknowledgment of completion from sink for data file {}", dataFilePartition.getKey()); + } else { + exportFileErrorCounter.increment(); + LOG.warn("Negative acknowledgment received for data file {}, retrying", dataFilePartition.getKey()); + sourceCoordinator.giveUpPartition(dataFilePartition); + } + }, sourceConfig.getDataFileAcknowledgmentTimeout()); + } + Runnable loader = DataFileLoader.create( - dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter, pluginMetrics); + dataFilePartition, codec, buffer, objectReader, recordConverter, pluginMetrics, + sourceCoordinator, acknowledgementSet, sourceConfig.getDataFileAcknowledgmentTimeout()); CompletableFuture runLoader = CompletableFuture.runAsync(loader, executor); - runLoader.whenComplete((v, ex) -> { - if (ex == null) { - exportFileSuccessCounter.increment(); - // Update global state so we know if all s3 files have been loaded - updateLoadStatus(dataFilePartition.getExportTaskId(), DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT); - sourceCoordinator.completePartition(dataFilePartition); - } else { - exportFileErrorCounter.increment(); - LOG.error("There was an exception while processing an S3 data file", ex); - sourceCoordinator.giveUpPartition(dataFilePartition); - } - numOfWorkers.decrementAndGet(); - }); + if (isAcknowledgmentsEnabled) { + runLoader.whenComplete((v, ex) -> { + if (ex != null) { + exportFileErrorCounter.increment(); + LOG.error("There was an exception while processing an S3 data file: {}", ex); + sourceCoordinator.giveUpPartition(dataFilePartition); + } + numOfWorkers.decrementAndGet(); + }); + } else { + runLoader.whenComplete(completeDataLoader(dataFilePartition)); + } numOfWorkers.incrementAndGet(); } @@ -183,4 +203,20 @@ private void updateLoadStatus(String exportTaskId, Duration timeout) { } } } + + private BiConsumer completeDataLoader(DataFilePartition dataFilePartition) { + return (v, ex) -> { + if (ex == null) { + exportFileSuccessCounter.increment(); + // Update global state, so we know if all s3 files have been loaded + updateLoadStatus(dataFilePartition.getExportTaskId(), DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT); + sourceCoordinator.completePartition(dataFilePartition); + } else { + exportFileErrorCounter.increment(); + LOG.error("There was an exception while processing an S3 data file", ex); + sourceCoordinator.giveUpPartition(dataFilePartition); + } + numOfWorkers.decrementAndGet(); + }; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index 882fb0242d..4fc00de3a5 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -148,12 +148,14 @@ private Map> getPrimaryKeyMap() { private void createStreamPartition(RdsSourceConfig sourceConfig) { final StreamProgressState progressState = new StreamProgressState(); progressState.setWaitForExport(sourceConfig.isExportEnabled()); - getCurrentBinlogPosition().ifPresent(progressState::setStartPosition); + getCurrentBinlogPosition().ifPresent(progressState::setCurrentPosition); StreamPartition streamPartition = new StreamPartition(sourceConfig.getDbIdentifier(), progressState); sourceCoordinator.createPartition(streamPartition); } private Optional getCurrentBinlogPosition() { - return schemaManager.getCurrentBinaryLogPosition(); + Optional binlogCoordinate = schemaManager.getCurrentBinaryLogPosition(); + LOG.debug("Current binlog position: {}", binlogCoordinate.orElse(null)); + return binlogCoordinate; } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/BinlogCoordinate.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/BinlogCoordinate.java index 6818dabe9b..2b02de14b1 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/BinlogCoordinate.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/BinlogCoordinate.java @@ -30,4 +30,12 @@ public String getBinlogFilename() { public long getBinlogPosition() { return binlogPosition; } + + @Override + public String toString() { + return "BinlogCoordinate{" + + "binlogFilename='" + binlogFilename + '\'' + + ", binlogPosition=" + binlogPosition + + '}'; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java index 487e9aefd0..181716a69a 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java @@ -7,7 +7,9 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import com.github.shyiko.mysql.binlog.event.DeleteRowsEventData; +import com.github.shyiko.mysql.binlog.event.EventHeaderV4; import com.github.shyiko.mysql.binlog.event.EventType; +import com.github.shyiko.mysql.binlog.event.RotateEventData; import com.github.shyiko.mysql.binlog.event.TableMapEventData; import com.github.shyiko.mysql.binlog.event.TableMapEventMetadata; import com.github.shyiko.mysql.binlog.event.UpdateRowsEventData; @@ -16,18 +18,22 @@ import io.micrometer.core.instrument.DistributionSummary; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import org.opensearch.dataprepper.plugins.source.rds.converter.StreamRecordConverter; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; import org.opensearch.dataprepper.plugins.source.rds.model.TableMetadata; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.Serializable; import java.time.Duration; +import java.util.ArrayList; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -52,25 +58,44 @@ public class BinlogEventListener implements BinaryLogClient.EventListener { private final Map tableMetadataMap; private final StreamRecordConverter recordConverter; + private final BinaryLogClient binaryLogClient; private final BufferAccumulator> bufferAccumulator; private final List tableNames; private final String s3Prefix; + private final boolean isAcknowledgmentsEnabled; private final PluginMetrics pluginMetrics; - + private final List pipelineEvents; + private final StreamCheckpointManager streamCheckpointManager; private final Counter changeEventSuccessCounter; private final Counter changeEventErrorCounter; private final DistributionSummary bytesReceivedSummary; private final DistributionSummary bytesProcessedSummary; + /** + * currentBinlogCoordinate is the coordinate where next event will start + */ + private BinlogCoordinate currentBinlogCoordinate; + public BinlogEventListener(final Buffer> buffer, final RdsSourceConfig sourceConfig, - final PluginMetrics pluginMetrics) { + final PluginMetrics pluginMetrics, + final BinaryLogClient binaryLogClient, + final StreamCheckpointer streamCheckpointer, + final AcknowledgementSetManager acknowledgementSetManager) { + this.binaryLogClient = binaryLogClient; tableMetadataMap = new HashMap<>(); recordConverter = new StreamRecordConverter(sourceConfig.getStream().getPartitionCount()); bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); s3Prefix = sourceConfig.getS3Prefix(); tableNames = sourceConfig.getTableNames(); + isAcknowledgmentsEnabled = sourceConfig.isAcknowledgmentsEnabled(); this.pluginMetrics = pluginMetrics; + pipelineEvents = new ArrayList<>(); + + this.streamCheckpointManager = new StreamCheckpointManager( + streamCheckpointer, sourceConfig.isAcknowledgmentsEnabled(), + acknowledgementSetManager, this::stopClient, sourceConfig.getStreamAcknowledgmentTimeout()); + streamCheckpointManager.start(); changeEventSuccessCounter = pluginMetrics.counter(CHANGE_EVENTS_PROCESSED_COUNT); changeEventErrorCounter = pluginMetrics.counter(CHANGE_EVENTS_PROCESSING_ERROR_COUNT); @@ -80,9 +105,12 @@ public BinlogEventListener(final Buffer> buffer, @Override public void onEvent(com.github.shyiko.mysql.binlog.event.Event event) { - EventType eventType = event.getHeader().getEventType(); + final EventType eventType = event.getHeader().getEventType(); switch (eventType) { + case ROTATE: + handleEventAndErrors(event, this::handleRotateEvent); + break; case TABLE_MAP: handleEventAndErrors(event, this::handleTableMapEvent); break; @@ -101,6 +129,27 @@ public void onEvent(com.github.shyiko.mysql.binlog.event.Event event) { } } + public void stopClient() { + try { + binaryLogClient.disconnect(); + LOG.info("Binary log client disconnected."); + } catch (Exception e) { + LOG.error("Binary log client failed to disconnect.", e); + } + } + + void handleRotateEvent(com.github.shyiko.mysql.binlog.event.Event event) { + final RotateEventData data = event.getData(); + currentBinlogCoordinate = new BinlogCoordinate(data.getBinlogFilename(), data.getBinlogPosition()); + + // Trigger a checkpoint update for this rotate when there're no row mutation events being processed + if (streamCheckpointManager.getChangeEventStatuses().isEmpty()) { + ChangeEventStatus changeEventStatus = streamCheckpointManager.saveChangeEventsStatus(currentBinlogCoordinate); + if (isAcknowledgmentsEnabled) { + changeEventStatus.setAcknowledgmentStatus(ChangeEventStatus.AcknowledgmentStatus.POSITIVE_ACK); + } + } + } void handleTableMapEvent(com.github.shyiko.mysql.binlog.event.Event event) { final TableMapEventData data = event.getData(); final TableMapEventMetadata tableMapEventMetadata = data.getEventMetadata(); @@ -116,109 +165,55 @@ void handleTableMapEvent(com.github.shyiko.mysql.binlog.event.Event event) { } void handleInsertEvent(com.github.shyiko.mysql.binlog.event.Event event) { - final long bytes = event.toString().getBytes().length; - bytesReceivedSummary.record(bytes); - LOG.debug("Handling insert event"); final WriteRowsEventData data = event.getData(); - if (!tableMetadataMap.containsKey(data.getTableId())) { - LOG.debug("Cannot find table metadata, the event is likely not from a table of interest or the table metadata was not read"); - return; - } - final TableMetadata tableMetadata = tableMetadataMap.get(data.getTableId()); - final String fullTableName = tableMetadata.getFullTableName(); - if (!isTableOfInterest(fullTableName)) { - LOG.debug("The event is not from a table of interest"); - return; - } - final List columnNames = tableMetadata.getColumnNames(); - final List primaryKeys = tableMetadata.getPrimaryKeys(); - final long eventTimestampMillis = event.getHeader().getTimestamp(); - - // Construct data prepper JacksonEvent - int eventCount = 0; - for (final Object[] rowDataArray : data.getRows()) { - final Map rowDataMap = new HashMap<>(); - for (int i = 0; i < rowDataArray.length; i++) { - rowDataMap.put(columnNames.get(i), rowDataArray[i]); - } - - Event pipelineEvent = recordConverter.convert( - rowDataMap, - tableMetadata.getDatabaseName(), - tableMetadata.getTableName(), - event.getHeader().getEventType(), - OpenSearchBulkActions.INDEX, - primaryKeys, - s3Prefix, - eventTimestampMillis, - eventTimestampMillis); - addToBuffer(new Record<>(pipelineEvent)); - eventCount++; - } - bytesProcessedSummary.record(bytes); - - flushBuffer(eventCount); + handleRowChangeEvent(event, data.getTableId(), data.getRows(), OpenSearchBulkActions.INDEX); } void handleUpdateEvent(com.github.shyiko.mysql.binlog.event.Event event) { - final long bytes = event.toString().getBytes().length; - bytesReceivedSummary.record(bytes); - LOG.debug("Handling update event"); final UpdateRowsEventData data = event.getData(); - if (!tableMetadataMap.containsKey(data.getTableId())) { - return; - } - final TableMetadata tableMetadata = tableMetadataMap.get(data.getTableId()); - final String fullTableName = tableMetadata.getFullTableName(); - if (!isTableOfInterest(fullTableName)) { - LOG.debug("The event is not from a table of interest"); - return; - } - final List columnNames = tableMetadata.getColumnNames(); - final List primaryKeys = tableMetadata.getPrimaryKeys(); - final long eventTimestampMillis = event.getHeader().getTimestamp(); - int eventCount = 0; - for (Map.Entry updatedRow : data.getRows()) { - // updatedRow contains data before update as key and data after update as value - final Object[] rowData = updatedRow.getValue(); + // updatedRow contains data before update as key and data after update as value + final List rows = data.getRows().stream() + .map(Map.Entry::getValue) + .collect(Collectors.toList()); - final Map dataMap = new HashMap<>(); - for (int i = 0; i < rowData.length; i++) { - dataMap.put(columnNames.get(i), rowData[i]); - } + handleRowChangeEvent(event, data.getTableId(), rows, OpenSearchBulkActions.INDEX); + } - final Event pipelineEvent = recordConverter.convert( - dataMap, - tableMetadata.getDatabaseName(), - tableMetadata.getTableName(), - event.getHeader().getEventType(), - OpenSearchBulkActions.INDEX, - primaryKeys, - s3Prefix, - eventTimestampMillis, - eventTimestampMillis); - addToBuffer(new Record<>(pipelineEvent)); - eventCount++; - } - bytesProcessedSummary.record(bytes); + void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { + LOG.debug("Handling delete event"); + final DeleteRowsEventData data = event.getData(); - flushBuffer(eventCount); + handleRowChangeEvent(event, data.getTableId(), data.getRows(), OpenSearchBulkActions.DELETE); } - void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { + private void handleRowChangeEvent(com.github.shyiko.mysql.binlog.event.Event event, + long tableId, + List rows, + OpenSearchBulkActions bulkAction) { + + // Update binlog coordinate after it's first assigned in rotate event handler + if (currentBinlogCoordinate != null) { + final EventHeaderV4 eventHeader = event.getHeader(); + currentBinlogCoordinate = new BinlogCoordinate(currentBinlogCoordinate.getBinlogFilename(), eventHeader.getNextPosition()); + LOG.debug("Current binlog coordinate after receiving a row change event: " + currentBinlogCoordinate); + } + + AcknowledgementSet acknowledgementSet = null; + if (isAcknowledgmentsEnabled) { + acknowledgementSet = streamCheckpointManager.createAcknowledgmentSet(currentBinlogCoordinate); + } + final long bytes = event.toString().getBytes().length; bytesReceivedSummary.record(bytes); - LOG.debug("Handling delete event"); - final DeleteRowsEventData data = event.getData(); - if (!tableMetadataMap.containsKey(data.getTableId())) { + if (!tableMetadataMap.containsKey(tableId)) { LOG.debug("Cannot find table metadata, the event is likely not from a table of interest or the table metadata was not read"); return; } - final TableMetadata tableMetadata = tableMetadataMap.get(data.getTableId()); + final TableMetadata tableMetadata = tableMetadataMap.get(tableId); final String fullTableName = tableMetadata.getFullTableName(); if (!isTableOfInterest(fullTableName)) { LOG.debug("The event is not from a table of interest"); @@ -228,8 +223,7 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = event.getHeader().getTimestamp(); - int eventCount = 0; - for (Object[] rowDataArray : data.getRows()) { + for (Object[] rowDataArray : rows) { final Map rowDataMap = new HashMap<>(); for (int i = 0; i < rowDataArray.length; i++) { rowDataMap.put(columnNames.get(i), rowDataArray[i]); @@ -240,24 +234,41 @@ void handleDeleteEvent(com.github.shyiko.mysql.binlog.event.Event event) { tableMetadata.getDatabaseName(), tableMetadata.getTableName(), event.getHeader().getEventType(), - OpenSearchBulkActions.DELETE, + bulkAction, primaryKeys, s3Prefix, eventTimestampMillis, eventTimestampMillis); - addToBuffer(new Record<>(pipelineEvent)); - eventCount++; + pipelineEvents.add(pipelineEvent); } + + writeToBuffer(acknowledgementSet); bytesProcessedSummary.record(bytes); - flushBuffer(eventCount); + if (isAcknowledgmentsEnabled) { + acknowledgementSet.complete(); + } else { + streamCheckpointManager.saveChangeEventsStatus(currentBinlogCoordinate); + } } private boolean isTableOfInterest(String tableName) { return new HashSet<>(tableNames).contains(tableName); } - private void addToBuffer(final Record record) { + private void writeToBuffer(AcknowledgementSet acknowledgementSet) { + for (Event pipelineEvent : pipelineEvents) { + addToBufferAccumulator(new Record<>(pipelineEvent)); + if (acknowledgementSet != null) { + acknowledgementSet.add(pipelineEvent); + } + } + + flushBufferAccumulator(pipelineEvents.size()); + pipelineEvents.clear(); + } + + private void addToBufferAccumulator(final Record record) { try { bufferAccumulator.add(record); } catch (Exception e) { @@ -265,11 +276,13 @@ private void addToBuffer(final Record record) { } } - private void flushBuffer(int eventCount) { + private void flushBufferAccumulator(int eventCount) { try { bufferAccumulator.flush(); changeEventSuccessCounter.increment(eventCount); } catch (Exception e) { + // this will only happen if writing to buffer gets interrupted from shutdown, + // otherwise bufferAccumulator will keep retrying with backoff LOG.error("Failed to flush buffer", e); changeEventErrorCounter.increment(eventCount); } @@ -280,7 +293,7 @@ private void handleEventAndErrors(com.github.shyiko.mysql.binlog.event.Event eve try { function.accept(event); } catch (Exception e) { - LOG.error("Failed to process change event", e); + LOG.error("Failed to process change event of type {}", event.getHeader().getEventType(), e); } } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ChangeEventStatus.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ChangeEventStatus.java new file mode 100644 index 0000000000..f2b70cbe7b --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ChangeEventStatus.java @@ -0,0 +1,51 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; + +public class ChangeEventStatus { + + private final BinlogCoordinate binlogCoordinate; + private final long timestamp; + private volatile AcknowledgmentStatus acknowledgmentStatus; + + public enum AcknowledgmentStatus { + POSITIVE_ACK, + NEGATIVE_ACK, + NO_ACK + } + + public ChangeEventStatus(final BinlogCoordinate binlogCoordinate, final long timestamp) { + this.binlogCoordinate = binlogCoordinate; + this.timestamp = timestamp; + acknowledgmentStatus = AcknowledgmentStatus.NO_ACK; + } + + public void setAcknowledgmentStatus(final AcknowledgmentStatus acknowledgmentStatus) { + this.acknowledgmentStatus = acknowledgmentStatus; + } + + public AcknowledgmentStatus getAcknowledgmentStatus() { + return acknowledgmentStatus; + } + + public boolean isPositiveAcknowledgment() { + return acknowledgmentStatus == AcknowledgmentStatus.POSITIVE_ACK; + } + + public boolean isNegativeAcknowledgment() { + return acknowledgmentStatus == AcknowledgmentStatus.NEGATIVE_ACK; + } + + public BinlogCoordinate getBinlogCoordinate() { + return binlogCoordinate; + } + + public long getTimestamp() { + return timestamp; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManager.java new file mode 100644 index 0000000000..3827f2b822 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManager.java @@ -0,0 +1,133 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.time.Instant; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +public class StreamCheckpointManager { + private static final Logger LOG = LoggerFactory.getLogger(StreamCheckpointManager.class); + static final int REGULAR_CHECKPOINT_INTERVAL_MILLIS = 60_000; + static final int CHANGE_EVENT_COUNT_PER_CHECKPOINT_BATCH = 1000; + + private final ConcurrentLinkedQueue changeEventStatuses = new ConcurrentLinkedQueue<>(); + private final StreamCheckpointer streamCheckpointer; + private final ExecutorService executorService; + private final Runnable stopStreamRunnable; + private final boolean isAcknowledgmentEnabled; + private final AcknowledgementSetManager acknowledgementSetManager; + private final Duration acknowledgmentTimeout; + + public StreamCheckpointManager(final StreamCheckpointer streamCheckpointer, + final boolean isAcknowledgmentEnabled, + final AcknowledgementSetManager acknowledgementSetManager, + final Runnable stopStreamRunnable, + final Duration acknowledgmentTimeout) { + this.acknowledgementSetManager = acknowledgementSetManager; + this.streamCheckpointer = streamCheckpointer; + this.isAcknowledgmentEnabled = isAcknowledgmentEnabled; + this.stopStreamRunnable = stopStreamRunnable; + this.acknowledgmentTimeout = acknowledgmentTimeout; + executorService = Executors.newSingleThreadExecutor(); + } + + public void start() { + executorService.submit(this::runCheckpointing); + } + + void runCheckpointing() { + ChangeEventStatus currentChangeEventStatus; + + while (!Thread.currentThread().isInterrupted()) { + try { + if (changeEventStatuses.isEmpty()) { + LOG.debug("No records processed. Extend the lease on stream partition."); + streamCheckpointer.extendLease(); + } else { + if (isAcknowledgmentEnabled) { + ChangeEventStatus lastChangeEventStatus = null; + currentChangeEventStatus = changeEventStatuses.peek(); + while (currentChangeEventStatus != null && currentChangeEventStatus.isPositiveAcknowledgment()) { + lastChangeEventStatus = currentChangeEventStatus; + currentChangeEventStatus = changeEventStatuses.poll(); + } + + if (lastChangeEventStatus != null) { + streamCheckpointer.checkpoint(lastChangeEventStatus.getBinlogCoordinate()); + } + + // If negative ack is seen, give up partition and exit loop to stop processing stream + if (currentChangeEventStatus != null && currentChangeEventStatus.isNegativeAcknowledgment()) { + LOG.info("Received negative acknowledgement for change event at {}. Will restart from most recent checkpoint", currentChangeEventStatus.getBinlogCoordinate()); + streamCheckpointer.giveUpPartition(); + break; + } + } else { + int changeEventCount = 0; + do { + currentChangeEventStatus = changeEventStatuses.poll(); + changeEventCount++; + // In case queue are populated faster than the poll, checkpoint when reaching certain count + if (changeEventCount % CHANGE_EVENT_COUNT_PER_CHECKPOINT_BATCH == 0) { + streamCheckpointer.checkpoint(currentChangeEventStatus.getBinlogCoordinate()); + } + } while (!changeEventStatuses.isEmpty()); + streamCheckpointer.checkpoint(currentChangeEventStatus.getBinlogCoordinate()); + } + } + } catch (Exception e) { + LOG.warn("Exception while checkpointing. The stream processing will start from previous checkpoint.", e); + break; + } + + try { + Thread.sleep(REGULAR_CHECKPOINT_INTERVAL_MILLIS); + } catch (InterruptedException ex) { + break; + } + } + + stopStreamRunnable.run(); + stop(); + } + + public void stop() { + executorService.shutdownNow(); + } + + public ChangeEventStatus saveChangeEventsStatus(BinlogCoordinate binlogCoordinate) { + final ChangeEventStatus changeEventStatus = new ChangeEventStatus(binlogCoordinate, Instant.now().toEpochMilli()); + changeEventStatuses.add(changeEventStatus); + return changeEventStatus; + } + + public AcknowledgementSet createAcknowledgmentSet(BinlogCoordinate binlogCoordinate) { + LOG.debug("Create acknowledgment set for events receive prior to {}", binlogCoordinate); + final ChangeEventStatus changeEventStatus = new ChangeEventStatus(binlogCoordinate, Instant.now().toEpochMilli()); + changeEventStatuses.add(changeEventStatus); + return acknowledgementSetManager.create((result) -> { + if (result) { + changeEventStatus.setAcknowledgmentStatus(ChangeEventStatus.AcknowledgmentStatus.POSITIVE_ACK); + } else { + changeEventStatus.setAcknowledgmentStatus(ChangeEventStatus.AcknowledgmentStatus.NEGATIVE_ACK); + } + }, acknowledgmentTimeout); + } + + //VisibleForTesting + ConcurrentLinkedQueue getChangeEventStatuses() { + return changeEventStatuses; + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java new file mode 100644 index 0000000000..b76dbab7c9 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java @@ -0,0 +1,58 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Optional; + +public class StreamCheckpointer { + + private static final Logger LOG = LoggerFactory.getLogger(StreamCheckpointer.class); + + static final Duration CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE = Duration.ofMinutes(5); + static final String CHECKPOINT_COUNT = "checkpoints"; + + private final EnhancedSourceCoordinator sourceCoordinator; + private final StreamPartition streamPartition; + private final PluginMetrics pluginMetrics; + private final Counter checkpointCounter; + + public StreamCheckpointer(final EnhancedSourceCoordinator sourceCoordinator, + final StreamPartition streamPartition, + final PluginMetrics pluginMetrics) { + this.sourceCoordinator = sourceCoordinator; + this.streamPartition = streamPartition; + this.pluginMetrics = pluginMetrics; + checkpointCounter = pluginMetrics.counter(CHECKPOINT_COUNT); + } + + public void checkpoint(final BinlogCoordinate binlogCoordinate) { + LOG.debug("Checkpointing stream partition {} with binlog coordinate {}", streamPartition.getPartitionKey(), binlogCoordinate); + Optional progressState = streamPartition.getProgressState(); + progressState.get().setCurrentPosition(binlogCoordinate); + sourceCoordinator.saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + checkpointCounter.increment(); + } + + public void extendLease() { + LOG.debug("Extending lease of stream partition {}", streamPartition.getPartitionKey()); + sourceCoordinator.saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + } + + public void giveUpPartition() { + LOG.debug("Giving up stream partition {}", streamPartition.getPartitionKey()); + sourceCoordinator.giveUpPartition(streamPartition); + } +} diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java index acb4ea3f85..1886bba451 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java @@ -7,6 +7,7 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; @@ -18,6 +19,8 @@ import org.slf4j.LoggerFactory; import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; public class StreamScheduler implements Runnable { @@ -29,7 +32,10 @@ public class StreamScheduler implements Runnable { private final EnhancedSourceCoordinator sourceCoordinator; private final RdsSourceConfig sourceConfig; private final BinaryLogClient binaryLogClient; + private final Buffer> buffer; private final PluginMetrics pluginMetrics; + private final AcknowledgementSetManager acknowledgementSetManager; + private final ExecutorService executorService; private volatile boolean shutdownRequested = false; @@ -37,13 +43,15 @@ public StreamScheduler(final EnhancedSourceCoordinator sourceCoordinator, final RdsSourceConfig sourceConfig, final BinaryLogClient binaryLogClient, final Buffer> buffer, - final PluginMetrics pluginMetrics) { + final PluginMetrics pluginMetrics, + final AcknowledgementSetManager acknowledgementSetManager) { this.sourceCoordinator = sourceCoordinator; this.sourceConfig = sourceConfig; this.binaryLogClient = binaryLogClient; - this.binaryLogClient.registerEventListener(new BinlogEventListener(buffer, sourceConfig, pluginMetrics)); + this.buffer = buffer; this.pluginMetrics = pluginMetrics; - + this.acknowledgementSetManager = acknowledgementSetManager; + executorService = Executors.newCachedThreadPool(); } @Override @@ -56,12 +64,15 @@ public void run() { LOG.info("Acquired partition to read from stream"); final StreamPartition streamPartition = (StreamPartition) sourcePartition.get(); + final StreamCheckpointer streamCheckpointer = new StreamCheckpointer(sourceCoordinator, streamPartition, pluginMetrics); + binaryLogClient.registerEventListener(new BinlogEventListener( + buffer, sourceConfig, pluginMetrics, binaryLogClient, streamCheckpointer, acknowledgementSetManager)); final StreamWorker streamWorker = StreamWorker.create(sourceCoordinator, binaryLogClient, pluginMetrics); - streamWorker.processStream(streamPartition); + executorService.submit(() -> streamWorker.processStream(streamPartition)); } try { - LOG.debug("Waiting to acquire stream partition."); + LOG.debug("Looping to acquire new stream partition or idle while stream worker is working"); Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); } catch (final InterruptedException e) { LOG.info("The StreamScheduler was interrupted while waiting to retry, stopping processing"); @@ -81,6 +92,7 @@ public void run() { } public void shutdown() { + executorService.shutdownNow(); shutdownRequested = true; } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java index 303ecc2a53..ed88b196b6 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java @@ -84,7 +84,7 @@ private boolean isExportDone(StreamPartition streamPartition) { } private void setStartBinlogPosition(final StreamPartition streamPartition) { - final BinlogCoordinate startBinlogPosition = streamPartition.getProgressState().get().getStartPosition(); + final BinlogCoordinate startBinlogPosition = streamPartition.getProgressState().get().getCurrentPosition(); // set start of binlog stream to current position if exists if (startBinlogPosition != null) { diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java index e0b69746c2..0a814e7fc1 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsServiceTest.java @@ -13,6 +13,7 @@ import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventFactory; @@ -69,6 +70,9 @@ class RdsServiceTest { @Mock private Buffer> buffer; + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + @BeforeEach void setUp() { when(clientFactory.buildRdsClient()).thenReturn(rdsClient); @@ -166,6 +170,6 @@ void test_service_shutdown_calls_executor_shutdownNow() { } private RdsService createObjectUnderTest() { - return new RdsService(sourceCoordinator, sourceConfig, eventFactory, clientFactory, pluginMetrics); + return new RdsService(sourceCoordinator, sourceConfig, eventFactory, clientFactory, pluginMetrics, acknowledgementSetManager); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java index 682f16ed51..f414173b05 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceTest.java @@ -12,6 +12,7 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.plugins.source.rds.configuration.AwsAuthenticationConfig; @@ -37,6 +38,9 @@ class RdsSourceTest { @Mock private AwsAuthenticationConfig awsAuthenticationConfig; + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + @BeforeEach void setUp() { when(sourceConfig.getAwsAuthenticationConfig()).thenReturn(awsAuthenticationConfig); @@ -49,6 +53,6 @@ void test_when_buffer_is_null_then_start_throws_exception() { } private RdsSource createObjectUnderTest() { - return new RdsSource(pluginMetrics, sourceConfig, eventFactory, awsCredentialsSupplier); + return new RdsSource(pluginMetrics, sourceConfig, eventFactory, awsCredentialsSupplier, acknowledgementSetManager); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java index ccb36347fa..c470021c6b 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java @@ -19,6 +19,8 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.BaseEventBuilder; import org.opensearch.dataprepper.model.event.Event; @@ -26,16 +28,20 @@ import org.opensearch.dataprepper.model.event.EventFactory; import org.opensearch.dataprepper.model.io.InputFile; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.codec.parquet.ParquetInputCodec; import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.DataFileProgressState; import java.io.InputStream; +import java.time.Duration; import java.util.Optional; import java.util.UUID; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -56,7 +62,7 @@ class DataFileLoaderTest { private DataFilePartition dataFilePartition; @Mock - private BufferAccumulator> bufferAccumulator; + private Buffer> buffer; @Mock private EventFactory eventFactory; @@ -70,6 +76,15 @@ class DataFileLoaderTest { @Mock private PluginMetrics pluginMetrics; + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + + @Mock + private AcknowledgementSet acknowledgementSet; + + @Mock + private Duration acknowledgmentTimeout; + @Mock private Counter exportRecordsTotalCounter; @@ -115,19 +130,26 @@ void test_run_success() throws Exception { when(eventFactory.eventBuilder(any())).thenReturn(eventBuilder); when(eventBuilder.withEventType(any()).withData(any()).build()).thenReturn(event); when(event.toJsonString()).thenReturn(randomString); + when(recordConverter.convert(any(), any(), any(), any(), anyLong(), anyLong())).thenReturn(event); + + AvroParquetReader.Builder builder = mock(AvroParquetReader.Builder.class); + ParquetReader parquetReader = mock(ParquetReader.class); + BufferAccumulator> bufferAccumulator = mock(BufferAccumulator.class); + when(builder.build()).thenReturn(parquetReader); + when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class); + MockedStatic bufferAccumulatorMockedStatic = mockStatic(BufferAccumulator.class)) { - try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class)) { - ParquetReader parquetReader = mock(ParquetReader.class); - AvroParquetReader.Builder builder = mock(AvroParquetReader.Builder.class); readerMockedStatic.when(() -> AvroParquetReader.builder(any(InputFile.class), any())).thenReturn(builder); - when(builder.build()).thenReturn(parquetReader); - when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + bufferAccumulatorMockedStatic.when(() -> BufferAccumulator.create(any(Buffer.class), anyInt(), any(Duration.class))).thenReturn(bufferAccumulator); dataFileLoader.run(); } verify(bufferAccumulator).add(any(Record.class)); verify(bufferAccumulator).flush(); + verify(acknowledgementSet).add(event); + verify(acknowledgementSet).complete(); verify(exportRecordsTotalCounter).increment(); verify(bytesReceivedSummary).record(sizeBytes); @@ -157,20 +179,27 @@ void test_flush_failure_then_error_metric_updated() throws Exception { when(eventFactory.eventBuilder(any())).thenReturn(eventBuilder); when(eventBuilder.withEventType(any()).withData(any()).build()).thenReturn(event); when(event.toJsonString()).thenReturn(randomString); - doThrow(new RuntimeException("testing")).when(bufferAccumulator).flush(); - try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class)) { - ParquetReader parquetReader = mock(ParquetReader.class); - AvroParquetReader.Builder builder = mock(AvroParquetReader.Builder.class); + when(recordConverter.convert(any(), any(), any(), any(), anyLong(), anyLong())).thenReturn(event); + + ParquetReader parquetReader = mock(ParquetReader.class); + AvroParquetReader.Builder builder = mock(AvroParquetReader.Builder.class); + BufferAccumulator> bufferAccumulator = mock(BufferAccumulator.class); + doThrow(new RuntimeException("testing")).when(bufferAccumulator).flush(); + when(builder.build()).thenReturn(parquetReader); + when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class); + MockedStatic bufferAccumulatorMockedStatic = mockStatic(BufferAccumulator.class)) { readerMockedStatic.when(() -> AvroParquetReader.builder(any(InputFile.class), any())).thenReturn(builder); - when(builder.build()).thenReturn(parquetReader); - when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + bufferAccumulatorMockedStatic.when(() -> BufferAccumulator.create(any(Buffer.class), anyInt(), any(Duration.class))).thenReturn(bufferAccumulator); dataFileLoader.run(); } verify(bufferAccumulator).add(any(Record.class)); verify(bufferAccumulator).flush(); + verify(acknowledgementSet).add(event); + verify(acknowledgementSet, never()).complete(); verify(exportRecordsTotalCounter).increment(); verify(bytesReceivedSummary).record(sizeBytes); @@ -181,6 +210,7 @@ void test_flush_failure_then_error_metric_updated() throws Exception { private DataFileLoader createObjectUnderTest() { final InputCodec codec = new ParquetInputCodec(eventFactory); - return DataFileLoader.create(dataFilePartition, codec, bufferAccumulator, s3ObjectReader, recordConverter, pluginMetrics); + return DataFileLoader.create(dataFilePartition, codec, buffer, s3ObjectReader, recordConverter, + pluginMetrics, sourceCoordinator, acknowledgementSet, acknowledgmentTimeout); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java index 5a5a56c6fd..f249e0b025 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileSchedulerTest.java @@ -12,8 +12,8 @@ import org.mockito.Mock; import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; -import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; @@ -72,6 +72,9 @@ class DataFileSchedulerTest { @Mock private PluginMetrics pluginMetrics; + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + @Mock private DataFilePartition dataFilePartition; @@ -112,7 +115,7 @@ void test_given_no_datafile_partition_then_no_export() throws InterruptedExcepti } @Test - void test_given_available_datafile_partition_then_load_datafile() { + void test_given_available_datafile_partition_then_load_datafile() throws InterruptedException { final String exportTaskId = UUID.randomUUID().toString(); when(dataFilePartition.getExportTaskId()).thenReturn(exportTaskId); @@ -129,9 +132,10 @@ void test_given_available_datafile_partition_then_load_datafile() { // MockedStatic needs to be created on the same thread it's used try (MockedStatic dataFileLoaderMockedStatic = mockStatic(DataFileLoader.class)) { DataFileLoader dataFileLoader = mock(DataFileLoader.class); - dataFileLoaderMockedStatic.when(() -> DataFileLoader.create( - eq(dataFilePartition), any(InputCodec.class), any(BufferAccumulator.class), any(S3ObjectReader.class), - any(ExportRecordConverter.class), any(PluginMetrics.class))) + dataFileLoaderMockedStatic.when(() -> DataFileLoader.create(eq(dataFilePartition), any(InputCodec.class), + any(Buffer.class), any(S3ObjectReader.class), + any(ExportRecordConverter.class), any(PluginMetrics.class), + any(EnhancedSourceCoordinator.class), any(), any(Duration.class))) .thenReturn(dataFileLoader); doNothing().when(dataFileLoader).run(); objectUnderTest.run(); @@ -157,9 +161,10 @@ void test_data_file_loader_throws_exception_then_give_up_partition() { // MockedStatic needs to be created on the same thread it's used try (MockedStatic dataFileLoaderMockedStatic = mockStatic(DataFileLoader.class)) { DataFileLoader dataFileLoader = mock(DataFileLoader.class); - dataFileLoaderMockedStatic.when(() -> DataFileLoader.create( - eq(dataFilePartition), any(InputCodec.class), any(BufferAccumulator.class), any(S3ObjectReader.class), - any(ExportRecordConverter.class), any(PluginMetrics.class))) + dataFileLoaderMockedStatic.when(() -> DataFileLoader.create(eq(dataFilePartition), any(InputCodec.class), + any(Buffer.class), any(S3ObjectReader.class), + any(ExportRecordConverter.class), any(PluginMetrics.class), + any(EnhancedSourceCoordinator.class), any(), any(Duration.class))) .thenReturn(dataFileLoader); doThrow(new RuntimeException()).when(dataFileLoader).run(); objectUnderTest.run(); @@ -187,6 +192,6 @@ void test_shutdown() { } private DataFileScheduler createObjectUnderTest() { - return new DataFileScheduler(sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer, pluginMetrics); + return new DataFileScheduler(sourceCoordinator, sourceConfig, s3Client, eventFactory, buffer, pluginMetrics, acknowledgementSetManager); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java index 30f622c5d7..d33ecd3a70 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugins.source.rds.stream; +import com.github.shyiko.mysql.binlog.BinaryLogClient; import com.github.shyiko.mysql.binlog.event.EventType; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -15,6 +16,7 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; @@ -37,6 +39,15 @@ class BinlogEventListenerTest { @Mock private PluginMetrics pluginMetrics; + @Mock + private BinaryLogClient binaryLogClient; + + @Mock + private StreamCheckpointer streamCheckpointer; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private com.github.shyiko.mysql.binlog.event.Event binlogEvent; @@ -91,6 +102,6 @@ void test_given_DeleteRows_event_then_calls_correct_handler(EventType eventType) } private BinlogEventListener createObjectUnderTest() { - return new BinlogEventListener(buffer, sourceConfig, pluginMetrics); + return new BinlogEventListener(buffer, sourceConfig, pluginMetrics, binaryLogClient, streamCheckpointer, acknowledgementSetManager); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManagerTest.java new file mode 100644 index 0000000000..deddb45e32 --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManagerTest.java @@ -0,0 +1,104 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; + +import java.time.Duration; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.function.Consumer; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; + + +@ExtendWith(MockitoExtension.class) +class StreamCheckpointManagerTest { + + static final Duration ACK_TIMEOUT = Duration.ofMinutes(5); + + @Mock + private StreamCheckpointer streamCheckpointer; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + @Mock + private Runnable stopStreamRunnable; + + private boolean isAcknowledgmentEnabled = false; + + @BeforeEach + void setUp() { + + } + + @Test + void test_start() { + final ExecutorService executorService = mock(ExecutorService.class); + try (MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { + executorsMockedStatic.when(Executors::newSingleThreadExecutor).thenReturn(executorService); + + final StreamCheckpointManager streamCheckpointManager = createObjectUnderTest(); + streamCheckpointManager.start(); + } + verify(executorService).submit(any(Runnable.class)); + } + + @Test + void test_shutdown() { + final ExecutorService executorService = mock(ExecutorService.class); + try (MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { + executorsMockedStatic.when(Executors::newSingleThreadExecutor).thenReturn(executorService); + + final StreamCheckpointManager streamCheckpointManager = createObjectUnderTest(); + streamCheckpointManager.start(); + streamCheckpointManager.stop(); + } + + verify(executorService).shutdownNow(); + } + + @Test + void test_saveChangeEventsStatus() { + final BinlogCoordinate binlogCoordinate = mock(BinlogCoordinate.class); + final StreamCheckpointManager streamCheckpointManager = createObjectUnderTest(); + streamCheckpointManager.saveChangeEventsStatus(binlogCoordinate); + + assertThat(streamCheckpointManager.getChangeEventStatuses().size(), is(1)); + assertThat(streamCheckpointManager.getChangeEventStatuses().peek().getBinlogCoordinate(), is(binlogCoordinate)); + } + + @Test + void test_createAcknowledgmentSet() { + final BinlogCoordinate binlogCoordinate = mock(BinlogCoordinate.class); + final StreamCheckpointManager streamCheckpointManager = createObjectUnderTest(); + streamCheckpointManager.createAcknowledgmentSet(binlogCoordinate); + + assertThat(streamCheckpointManager.getChangeEventStatuses().size(), is(1)); + ChangeEventStatus changeEventStatus = streamCheckpointManager.getChangeEventStatuses().peek(); + assertThat(changeEventStatus.getBinlogCoordinate(), is(binlogCoordinate)); + verify(acknowledgementSetManager).create(any(Consumer.class), eq(ACK_TIMEOUT)); + } + + private StreamCheckpointManager createObjectUnderTest() { + return new StreamCheckpointManager( + streamCheckpointer, isAcknowledgmentEnabled, acknowledgementSetManager, stopStreamRunnable, ACK_TIMEOUT); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java new file mode 100644 index 0000000000..2fdac1065f --- /dev/null +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java @@ -0,0 +1,83 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.rds.stream; + +import io.micrometer.core.instrument.Counter; +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.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; + +import java.util.Optional; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.stream.StreamCheckpointer.CHECKPOINT_COUNT; +import static org.opensearch.dataprepper.plugins.source.rds.stream.StreamCheckpointer.CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE; + + +@ExtendWith(MockitoExtension.class) +class StreamCheckpointerTest { + + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + + @Mock + private StreamPartition streamPartition; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private Counter checkpointCounter; + + private StreamCheckpointer streamCheckpointer; + + + @BeforeEach + void setUp() { + when(pluginMetrics.counter(CHECKPOINT_COUNT)).thenReturn(checkpointCounter); + streamCheckpointer = createObjectUnderTest(); + } + + @Test + void test_checkpoint() { + final BinlogCoordinate binlogCoordinate = mock(BinlogCoordinate.class); + final StreamProgressState streamProgressState = mock(StreamProgressState.class); + when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); + + streamCheckpointer.checkpoint(binlogCoordinate); + + verify(streamProgressState).setCurrentPosition(binlogCoordinate); + verify(sourceCoordinator).saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + verify(checkpointCounter).increment(); + } + + @Test + void test_extendLease() { + streamCheckpointer.extendLease(); + + verify(sourceCoordinator).saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + } + + @Test + void test_giveUpPartition() { + streamCheckpointer.giveUpPartition(); + + verify(sourceCoordinator).giveUpPartition(streamPartition); + } + + private StreamCheckpointer createObjectUnderTest() { + return new StreamCheckpointer(sourceCoordinator, streamPartition, pluginMetrics); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java index 1a152137ee..3c8b70cab2 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java @@ -14,6 +14,7 @@ import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; @@ -27,12 +28,12 @@ import java.util.concurrent.Executors; import static org.awaitility.Awaitility.await; -import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; @@ -53,6 +54,9 @@ class StreamSchedulerTest { @Mock private Buffer> buffer; + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + private StreamScheduler objectUnderTest; @BeforeEach @@ -71,8 +75,7 @@ void test_given_no_stream_partition_then_no_stream_actions() throws InterruptedE Thread.sleep(100); executorService.shutdownNow(); - verify(binaryLogClient).registerEventListener(any(BinlogEventListener.class)); - verifyNoMoreInteractions(binaryLogClient); + verifyNoInteractions(binaryLogClient); } @Test @@ -111,6 +114,6 @@ void test_shutdown() { } private StreamScheduler createObjectUnderTest() { - return new StreamScheduler(sourceCoordinator, sourceConfig, binaryLogClient, buffer, pluginMetrics); + return new StreamScheduler(sourceCoordinator, sourceConfig, binaryLogClient, buffer, pluginMetrics, acknowledgementSetManager); } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java index 1392c852e8..ecc7d86d47 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTest.java @@ -54,7 +54,7 @@ void test_processStream_with_given_binlog_coordinates() throws IOException { final String binlogFilename = UUID.randomUUID().toString(); final long binlogPosition = 100L; when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); - when(streamProgressState.getStartPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); + when(streamProgressState.getCurrentPosition()).thenReturn(new BinlogCoordinate(binlogFilename, binlogPosition)); when(streamProgressState.shouldWaitForExport()).thenReturn(false); streamWorker.processStream(streamPartition); @@ -70,7 +70,7 @@ void test_processStream_without_current_binlog_coordinates() throws IOException when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); final String binlogFilename = "binlog-001"; final Long binlogPosition = 100L; - when(streamProgressState.getStartPosition()).thenReturn(null); + when(streamProgressState.getCurrentPosition()).thenReturn(null); when(streamProgressState.shouldWaitForExport()).thenReturn(false); streamWorker.processStream(streamPartition); From a395bd29d2f17a90ca8902b3a9cb6c98104fe34e Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Tue, 27 Aug 2024 10:57:53 -0400 Subject: [PATCH 137/159] REF: data prepper plugin schema (#4872) Signed-off-by: George Chen --- data-prepper-plugin-schema-cli/build.gradle | 2 +- data-prepper-plugin-schema/build.gradle | 35 +++++++++++++++++++ .../schemas/JsonSchemaConverter.java | 0 .../PluginConfigsJsonSchemaConverter.java | 0 .../schemas/module/CustomJacksonModule.java | 0 .../schemas/JsonSchemaConverterTest.java | 0 .../PluginConfigsJsonSchemaConverterIT.java | 0 .../PluginConfigsJsonSchemaConverterTest.java | 0 settings.gradle | 4 ++- 9 files changed, 39 insertions(+), 2 deletions(-) create mode 100644 data-prepper-plugin-schema/build.gradle rename {data-prepper-plugin-schema-cli => data-prepper-plugin-schema}/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java (100%) rename {data-prepper-plugin-schema-cli => data-prepper-plugin-schema}/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java (100%) rename {data-prepper-plugin-schema-cli => data-prepper-plugin-schema}/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java (100%) rename {data-prepper-plugin-schema-cli => data-prepper-plugin-schema}/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java (100%) rename {data-prepper-plugin-schema-cli => data-prepper-plugin-schema}/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java (100%) rename {data-prepper-plugin-schema-cli => data-prepper-plugin-schema}/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java (100%) diff --git a/data-prepper-plugin-schema-cli/build.gradle b/data-prepper-plugin-schema-cli/build.gradle index 2108fad681..fdcf46cf92 100644 --- a/data-prepper-plugin-schema-cli/build.gradle +++ b/data-prepper-plugin-schema-cli/build.gradle @@ -9,7 +9,7 @@ application { dependencies { implementation project(':data-prepper-plugins') - implementation project(':data-prepper-plugin-framework') + implementation project(':data-prepper-plugin-schema') implementation 'com.fasterxml.jackson.core:jackson-databind' implementation 'org.reflections:reflections:0.10.2' implementation 'com.github.victools:jsonschema-maven-plugin:4.35.0' diff --git a/data-prepper-plugin-schema/build.gradle b/data-prepper-plugin-schema/build.gradle new file mode 100644 index 0000000000..bea3f0f778 --- /dev/null +++ b/data-prepper-plugin-schema/build.gradle @@ -0,0 +1,35 @@ +plugins { + id 'data-prepper.publish' +} + +dependencies { + implementation project(':data-prepper-api') + implementation project(':data-prepper-plugin-framework') + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'org.reflections:reflections:0.10.2' + implementation 'com.github.victools:jsonschema-maven-plugin:4.35.0' + implementation 'com.github.victools:jsonschema-generator:4.35.0' + implementation 'com.github.victools:jsonschema-module-jackson:4.35.0' + implementation 'com.github.victools:jsonschema-module-jakarta-validation:4.35.0' + implementation 'javax.inject:javax.inject:1' + implementation(libs.spring.core) { + exclude group: 'commons-logging', module: 'commons-logging' + } + implementation(libs.spring.context) { + exclude group: 'commons-logging', module: 'commons-logging' + } + testImplementation(platform("org.junit:junit-bom:5.9.1")) + testImplementation("org.junit.jupiter:junit-jupiter") + testImplementation project(':data-prepper-plugins') +} + +jacocoTestCoverageVerification { + dependsOn jacocoTestReport + violationRules { + rule { //in addition to core projects rule + limit { + minimum = 0.90 + } + } + } +} \ No newline at end of file diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java b/data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java similarity index 100% rename from data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java rename to data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java b/data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java similarity index 100% rename from data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java rename to data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverter.java diff --git a/data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java b/data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java similarity index 100% rename from data-prepper-plugin-schema-cli/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java rename to data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/module/CustomJacksonModule.java diff --git a/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java b/data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java similarity index 100% rename from data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java rename to data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java diff --git a/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java b/data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java similarity index 100% rename from data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java rename to data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterIT.java diff --git a/data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java b/data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java similarity index 100% rename from data-prepper-plugin-schema-cli/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java rename to data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/PluginConfigsJsonSchemaConverterTest.java diff --git a/settings.gradle b/settings.gradle index 3524edaee3..97742e8576 100644 --- a/settings.gradle +++ b/settings.gradle @@ -181,4 +181,6 @@ include 'data-prepper-plugins:rds-source' include 'data-prepper-plugins:http-source-common' include 'data-prepper-plugins:http-common' include 'data-prepper-plugins:aws-lambda' -//include 'data-prepper-plugins:dummy-plugin' \ No newline at end of file +//include 'data-prepper-plugins:dummy-plugin' +include 'data-prepper-plugin-schema' + From 9c14d11fbb93fc9e47aae4915123710ce005cace Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Tue, 27 Aug 2024 12:16:55 -0400 Subject: [PATCH 138/159] ENH: Plugin errors consolidator (#4863) * ENH: Plugin errors consolidator Signed-off-by: George Chen --- .../plugin/DefaultPluginFactoryIT.java | 3 ++ .../dataprepper/plugins/ExtensionsIT.java | 5 +++ .../parser/PipelineTransformer.java | 9 +++-- .../config/PipelineParserConfiguration.java | 7 ++-- .../parser/PipelineTransformerTests.java | 36 ++++++++++++++++--- .../PipelineParserConfigurationTest.java | 6 +++- .../LoggingPluginErrorsHandler.java | 26 ++++++++++++++ .../validation/PluginErrorCollector.java | 14 -------- .../validation/PluginErrorsHandler.java | 8 +++++ .../validation/DataPrepperValidateTest.java | 3 -- .../LoggingPluginErrorsHandlerTest.java | 35 ++++++++++++++++++ .../validation/PluginErrorCollectorTest.java | 10 ------ .../dataprepper/plugin/ExtensionLoader.java | 16 ++++++--- .../plugin/ExtensionLoaderTest.java | 23 ++++++++++-- 14 files changed, 157 insertions(+), 44 deletions(-) create mode 100644 data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/LoggingPluginErrorsHandler.java create mode 100644 data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorsHandler.java delete mode 100644 data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/DataPrepperValidateTest.java create mode 100644 data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/LoggingPluginErrorsHandlerTest.java diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java index 89ab07d11d..4c52c614d4 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java @@ -17,7 +17,9 @@ import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.plugins.TestObjectPlugin; import org.opensearch.dataprepper.plugins.test.TestPlugin; +import org.opensearch.dataprepper.validation.LoggingPluginErrorsHandler; import org.opensearch.dataprepper.validation.PluginErrorCollector; +import org.opensearch.dataprepper.validation.PluginErrorsHandler; import org.springframework.context.annotation.AnnotationConfigApplicationContext; import java.util.HashMap; @@ -63,6 +65,7 @@ private DefaultPluginFactory createObjectUnderTest() { coreContext.scan(DefaultPluginFactory.class.getPackage().getName()); coreContext.register(PluginBeanFactoryProvider.class); coreContext.registerBean(PluginErrorCollector.class, PluginErrorCollector::new); + coreContext.registerBean(PluginErrorsHandler.class, LoggingPluginErrorsHandler::new); coreContext.registerBean(ExtensionsConfiguration.class, () -> extensionsConfiguration); coreContext.registerBean(PipelinesDataFlowModel.class, () -> pipelinesDataFlowModel); coreContext.refresh(); diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/ExtensionsIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/ExtensionsIT.java index 1c4dd2f967..53dee72e55 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/ExtensionsIT.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugins/ExtensionsIT.java @@ -31,7 +31,9 @@ import org.opensearch.dataprepper.plugin.TestPluggableInterface; import org.opensearch.dataprepper.plugins.test.TestExtension; import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; +import org.opensearch.dataprepper.validation.LoggingPluginErrorsHandler; import org.opensearch.dataprepper.validation.PluginErrorCollector; +import org.opensearch.dataprepper.validation.PluginErrorsHandler; import org.springframework.context.annotation.AnnotationConfigApplicationContext; import java.util.ArrayList; @@ -70,6 +72,7 @@ public class ExtensionsIT { private AnnotationConfigApplicationContext coreContext; private PluginFactory pluginFactory; private PluginErrorCollector pluginErrorCollector; + private PluginErrorsHandler pluginErrorsHandler; private String pluginName; private String pipelineName; @@ -78,6 +81,7 @@ void setUp() { pluginName = "test_plugin_using_extension"; pipelineName = UUID.randomUUID().toString(); pluginErrorCollector = new PluginErrorCollector(); + pluginErrorsHandler = new LoggingPluginErrorsHandler(); publicContext = new AnnotationConfigApplicationContext(); publicContext.refresh(); @@ -108,6 +112,7 @@ void setUp() { coreContext.registerBean(ObjectMapper.class, () -> new ObjectMapper(new YAMLFactory())); coreContext.register(PipelineParserConfiguration.class); coreContext.registerBean(PluginErrorCollector.class, () -> pluginErrorCollector); + coreContext.registerBean(PluginErrorsHandler.class, () -> pluginErrorsHandler); coreContext.refresh(); pluginFactory = coreContext.getBean(DefaultPluginFactory.class); diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/PipelineTransformer.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/PipelineTransformer.java index a01103c03a..b3f4aede00 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/PipelineTransformer.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/PipelineTransformer.java @@ -34,6 +34,7 @@ import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; import org.opensearch.dataprepper.validation.PluginError; import org.opensearch.dataprepper.validation.PluginErrorCollector; +import org.opensearch.dataprepper.validation.PluginErrorsHandler; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -65,6 +66,7 @@ public class PipelineTransformer { private final AcknowledgementSetManager acknowledgementSetManager; private final SourceCoordinatorFactory sourceCoordinatorFactory; private final PluginErrorCollector pluginErrorCollector; + private final PluginErrorsHandler pluginErrorsHandler; public PipelineTransformer(final PipelinesDataFlowModel pipelinesDataFlowModel, final PluginFactory pluginFactory, @@ -75,7 +77,8 @@ public PipelineTransformer(final PipelinesDataFlowModel pipelinesDataFlowModel, final EventFactory eventFactory, final AcknowledgementSetManager acknowledgementSetManager, final SourceCoordinatorFactory sourceCoordinatorFactory, - final PluginErrorCollector pluginErrorCollector) { + final PluginErrorCollector pluginErrorCollector, + final PluginErrorsHandler pluginErrorsHandler) { this.pipelinesDataFlowModel = pipelinesDataFlowModel; this.pluginFactory = Objects.requireNonNull(pluginFactory); this.peerForwarderProvider = Objects.requireNonNull(peerForwarderProvider); @@ -86,6 +89,7 @@ public PipelineTransformer(final PipelinesDataFlowModel pipelinesDataFlowModel, this.acknowledgementSetManager = acknowledgementSetManager; this.sourceCoordinatorFactory = sourceCoordinatorFactory; this.pluginErrorCollector = pluginErrorCollector; + this.pluginErrorsHandler = pluginErrorsHandler; } public Map transformConfiguration() { @@ -165,9 +169,10 @@ private void buildPipelineFromConfiguration( .stream().filter(pluginError -> pipelineName.equals(pluginError.getPipelineName())) .collect(Collectors.toList()); if (!subPipelinePluginErrors.isEmpty()) { + pluginErrorsHandler.handleErrors(subPipelinePluginErrors); throw new InvalidPluginConfigurationException( String.format("One or more plugins are not configured correctly in the pipeline: %s.\n", - pipelineName) + pluginErrorCollector.getConsolidatedErrorMessage()); + pipelineName)); } final List> decoratedProcessorSets = processorSets.stream() diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/config/PipelineParserConfiguration.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/config/PipelineParserConfiguration.java index 3fbeedf5af..440e618f3b 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/config/PipelineParserConfiguration.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/parser/config/PipelineParserConfiguration.java @@ -22,6 +22,7 @@ import org.opensearch.dataprepper.pipeline.router.RouterFactory; import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; import org.opensearch.dataprepper.validation.PluginErrorCollector; +import org.opensearch.dataprepper.validation.PluginErrorsHandler; import org.springframework.beans.factory.annotation.Qualifier; import org.springframework.context.annotation.Bean; import org.springframework.context.annotation.ComponentScan; @@ -46,7 +47,8 @@ public PipelineTransformer pipelineParser( final EventFactory eventFactory, final AcknowledgementSetManager acknowledgementSetManager, final SourceCoordinatorFactory sourceCoordinatorFactory, - final PluginErrorCollector pluginErrorCollector + final PluginErrorCollector pluginErrorCollector, + final PluginErrorsHandler pluginErrorsHandler ) { return new PipelineTransformer(pipelinesDataFlowModel, pluginFactory, @@ -57,7 +59,8 @@ public PipelineTransformer pipelineParser( eventFactory, acknowledgementSetManager, sourceCoordinatorFactory, - pluginErrorCollector); + pluginErrorCollector, + pluginErrorsHandler); } @Bean diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/PipelineTransformerTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/PipelineTransformerTests.java index 112bb8e93f..13b30965a6 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/PipelineTransformerTests.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/PipelineTransformerTests.java @@ -14,6 +14,8 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.TestDataProvider; @@ -22,6 +24,7 @@ import org.opensearch.dataprepper.core.event.EventFactoryApplicationContextMarker; import org.opensearch.dataprepper.model.breaker.CircuitBreaker; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.configuration.PipelineModel; import org.opensearch.dataprepper.model.configuration.PipelinesDataFlowModel; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.EventFactory; @@ -39,14 +42,18 @@ import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; import org.opensearch.dataprepper.validation.PluginError; import org.opensearch.dataprepper.validation.PluginErrorCollector; +import org.opensearch.dataprepper.validation.PluginErrorsHandler; import org.springframework.context.annotation.AnnotationConfigApplicationContext; import java.time.Duration; +import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Random; +import java.util.Set; import java.util.UUID; import java.util.stream.Stream; @@ -84,6 +91,10 @@ class PipelineTransformerTests { private SourceCoordinatorFactory sourceCoordinatorFactory; @Mock private CircuitBreakerManager circuitBreakerManager; + @Mock + private PluginErrorsHandler pluginErrorsHandler; + @Captor + private ArgumentCaptor> pluginErrorsArgumentCaptor; private PluginErrorCollector pluginErrorCollector; @@ -111,6 +122,7 @@ void setUp() { coreContext.scan(DefaultPluginFactory.class.getPackage().getName()); coreContext.registerBean(PluginErrorCollector.class, () -> pluginErrorCollector); + coreContext.registerBean(PluginErrorsHandler.class, () -> pluginErrorsHandler); coreContext.registerBean(DataPrepperConfiguration.class, () -> dataPrepperConfiguration); coreContext.registerBean(PipelinesDataFlowModel.class, () -> pipelinesDataFlowModel); coreContext.refresh(); @@ -126,9 +138,11 @@ private PipelineTransformer createObjectUnderTest(final String pipelineConfigura final PipelinesDataFlowModel pipelinesDataFlowModel = new PipelinesDataflowModelParser( new PipelineConfigurationFileReader(pipelineConfigurationFileLocation)).parseConfiguration(); - return new PipelineTransformer(pipelinesDataFlowModel, pluginFactory, peerForwarderProvider, - routerFactory, dataPrepperConfiguration, circuitBreakerManager, eventFactory, - acknowledgementSetManager, sourceCoordinatorFactory, pluginErrorCollector); + return new PipelineTransformer( + pipelinesDataFlowModel, pluginFactory, peerForwarderProvider, + routerFactory, dataPrepperConfiguration, circuitBreakerManager, eventFactory, + acknowledgementSetManager, sourceCoordinatorFactory, pluginErrorCollector, + pluginErrorsHandler); } @Test @@ -214,17 +228,29 @@ void parseConfiguration_with_invalid_root_pipeline_creates_empty_pipelinesMap() }) void parseConfiguration_with_incorrect_child_pipeline_returns_empty_pipelinesMap( final String pipelineConfigurationFileLocation) { + pluginErrorCollector.collectPluginError( + PluginError.builder() + .componentType("non-pipeline-plugin") + .pluginName("preexisting-plugin") + .exception(new RuntimeException()) + .build()); mockDataPrepperConfigurationAccesses(); final PipelineTransformer pipelineTransformer = createObjectUnderTest(pipelineConfigurationFileLocation); final Map connectedPipelines = pipelineTransformer.transformConfiguration(); assertThat(connectedPipelines.size(), equalTo(0)); verifyDataPrepperConfigurationAccesses(); verify(dataPrepperConfiguration).getPipelineExtensions(); - assertThat(pluginErrorCollector.getPluginErrors().size(), equalTo(1)); - final PluginError pluginError = pluginErrorCollector.getPluginErrors().get(0); + assertThat(pluginErrorCollector.getPluginErrors().size(), equalTo(2)); + final PluginError pluginError = pluginErrorCollector.getPluginErrors().get(1); assertThat(pluginError.getPipelineName(), equalTo("test-pipeline-2")); assertThat(pluginError.getPluginName(), notNullValue()); assertThat(pluginError.getException(), notNullValue()); + verify(pluginErrorsHandler).handleErrors(pluginErrorsArgumentCaptor.capture()); + final Collection pluginErrorCollection = pluginErrorsArgumentCaptor.getValue(); + assertThat(pluginErrorCollection.size(), equalTo(1)); + final PluginError capturedPluginError = new ArrayList<>(pluginErrorCollection).get(0); + assertThat(Set.of(PipelineModel.PROCESSOR_PLUGIN_TYPE, PipelineModel.SINK_PLUGIN_TYPE) + .contains(capturedPluginError.getComponentType()), is(true)); } @Test diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/config/PipelineParserConfigurationTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/config/PipelineParserConfigurationTest.java index 9b027c6c75..ab657dfc94 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/config/PipelineParserConfigurationTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/parser/config/PipelineParserConfigurationTest.java @@ -20,6 +20,7 @@ import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.sourcecoordination.SourceCoordinatorFactory; import org.opensearch.dataprepper.validation.PluginErrorCollector; +import org.opensearch.dataprepper.validation.PluginErrorsHandler; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.is; @@ -59,12 +60,15 @@ class PipelineParserConfigurationTest { @Mock private PluginErrorCollector pluginErrorCollector; + @Mock + private PluginErrorsHandler pluginErrorsHandler; + @Test void pipelineParser() { final PipelineTransformer pipelineTransformer = pipelineParserConfiguration.pipelineParser( pipelinesDataFlowModel, pluginFactory, peerForwarderProvider, routerFactory, dataPrepperConfiguration, circuitBreakerManager, eventFactory, acknowledgementSetManager, - sourceCoordinatorFactory, pluginErrorCollector); + sourceCoordinatorFactory, pluginErrorCollector, pluginErrorsHandler); assertThat(pipelineTransformer, is(notNullValue())); } diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/LoggingPluginErrorsHandler.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/LoggingPluginErrorsHandler.java new file mode 100644 index 0000000000..879f18823e --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/LoggingPluginErrorsHandler.java @@ -0,0 +1,26 @@ +package org.opensearch.dataprepper.validation; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.inject.Named; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +@Named +public class LoggingPluginErrorsHandler implements PluginErrorsHandler { + private static final Logger LOG = LoggerFactory.getLogger(LoggingPluginErrorsHandler.class); + + @Override + public void handleErrors(final Collection pluginErrors) { + final List allErrorMessages = pluginErrors.stream() + .map(PluginError::getErrorMessage) + .collect(Collectors.toList()); + final String consolidatedErrorMessage = IntStream.range(0, allErrorMessages.size()) + .mapToObj(i -> (i + 1) + ". " + allErrorMessages.get(i)) + .collect(Collectors.joining("\n")); + LOG.error(consolidatedErrorMessage); + } +} diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorCollector.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorCollector.java index 0be5539019..62b6654c0b 100644 --- a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorCollector.java +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorCollector.java @@ -5,8 +5,6 @@ import javax.inject.Named; import java.util.ArrayList; import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.IntStream; @Named @Getter @@ -16,16 +14,4 @@ public class PluginErrorCollector { public void collectPluginError(final PluginError pluginError) { pluginErrors.add(pluginError); } - - public List getAllErrorMessages() { - return pluginErrors.stream().map(PluginError::getErrorMessage).collect(Collectors.toList()); - } - - public String getConsolidatedErrorMessage() { - final List allErrorMessages = getAllErrorMessages(); - - return IntStream.range(0, allErrorMessages.size()) - .mapToObj(i -> (i + 1) + ". " + allErrorMessages.get(i)) - .collect(Collectors.joining("\n")); - } } diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorsHandler.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorsHandler.java new file mode 100644 index 0000000000..3417ff5bfa --- /dev/null +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/validation/PluginErrorsHandler.java @@ -0,0 +1,8 @@ +package org.opensearch.dataprepper.validation; + +import java.util.Collection; + +public interface PluginErrorsHandler { + + public void handleErrors(final Collection pluginErrors); +} diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/DataPrepperValidateTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/DataPrepperValidateTest.java deleted file mode 100644 index 447fb21838..0000000000 --- a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/DataPrepperValidateTest.java +++ /dev/null @@ -1,3 +0,0 @@ -package org.opensearch.dataprepper.validation; -class DataPrepperValidateTest { -} \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/LoggingPluginErrorsHandlerTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/LoggingPluginErrorsHandlerTest.java new file mode 100644 index 0000000000..f4b8fbbec5 --- /dev/null +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/LoggingPluginErrorsHandlerTest.java @@ -0,0 +1,35 @@ +package org.opensearch.dataprepper.validation; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Arrays; +import java.util.Collection; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +class LoggingPluginErrorsHandlerTest { + @Test + void testHandleErrors() { + final Logger mockLogger = mock(Logger.class); + + try (final MockedStatic mockedLoggerFactory = mockStatic(LoggerFactory.class)) { + mockedLoggerFactory.when(() -> LoggerFactory.getLogger(LoggingPluginErrorsHandler.class)) + .thenReturn(mockLogger); + final LoggingPluginErrorsHandler loggingPluginErrorsHandler = new LoggingPluginErrorsHandler(); + final PluginError error1 = mock(PluginError.class); + final PluginError error2 = mock(PluginError.class); + when(error1.getErrorMessage()).thenReturn("Error 1 occurred"); + when(error2.getErrorMessage()).thenReturn("Error 2 occurred"); + final Collection pluginErrors = Arrays.asList(error1, error2); + loggingPluginErrorsHandler.handleErrors(pluginErrors); + final String expectedMessage = "1. Error 1 occurred\n2. Error 2 occurred"; + verify(mockLogger).error(expectedMessage); + } + } +} \ No newline at end of file diff --git a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorCollectorTest.java b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorCollectorTest.java index 324a202fb8..d0b5fbd8b2 100644 --- a/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorCollectorTest.java +++ b/data-prepper-pipeline-parser/src/test/java/org/opensearch/dataprepper/validation/PluginErrorCollectorTest.java @@ -4,27 +4,17 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; class PluginErrorCollectorTest { @Test void testWithPluginErrors() { final PluginErrorCollector objectUnderTest = new PluginErrorCollector(); - final String testErrorMessage1 = "test error message 1"; - final String testErrorMessage2 = "test error message 2"; final PluginError pluginError1 = mock(PluginError.class); - when(pluginError1.getErrorMessage()).thenReturn(testErrorMessage1); final PluginError pluginError2 = mock(PluginError.class); - when(pluginError2.getErrorMessage()).thenReturn(testErrorMessage2); objectUnderTest.collectPluginError(pluginError1); objectUnderTest.collectPluginError(pluginError2); assertThat(objectUnderTest.getPluginErrors().size(), equalTo(2)); - assertThat(objectUnderTest.getAllErrorMessages().size(), equalTo(2)); - assertThat(objectUnderTest.getAllErrorMessages(), contains(testErrorMessage1, testErrorMessage2)); - assertThat(objectUnderTest.getConsolidatedErrorMessage(), equalTo( - String.format("1. %s\n2. %s", testErrorMessage1, testErrorMessage2))); } } \ No newline at end of file diff --git a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionLoader.java b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionLoader.java index f527d835a4..bd895f6e51 100644 --- a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionLoader.java +++ b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionLoader.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.plugin.InvalidPluginDefinitionException; import org.opensearch.dataprepper.validation.PluginError; import org.opensearch.dataprepper.validation.PluginErrorCollector; +import org.opensearch.dataprepper.validation.PluginErrorsHandler; import javax.inject.Inject; import javax.inject.Named; @@ -26,17 +27,20 @@ public class ExtensionLoader { private final ExtensionClassProvider extensionClassProvider; private final PluginCreator extensionPluginCreator; private final PluginErrorCollector pluginErrorCollector; + private final PluginErrorsHandler pluginErrorsHandler; @Inject ExtensionLoader( final ExtensionPluginConfigurationConverter extensionPluginConfigurationConverter, final ExtensionClassProvider extensionClassProvider, @Named("extensionPluginCreator") final PluginCreator extensionPluginCreator, - final PluginErrorCollector pluginErrorCollector) { + final PluginErrorCollector pluginErrorCollector, + final PluginErrorsHandler pluginErrorsHandler) { this.extensionPluginConfigurationConverter = extensionPluginConfigurationConverter; this.extensionClassProvider = extensionClassProvider; this.extensionPluginCreator = extensionPluginCreator; this.pluginErrorCollector = pluginErrorCollector; + this.pluginErrorsHandler = pluginErrorsHandler; } public List loadExtensions() { @@ -59,10 +63,14 @@ public List loadExtensions() { } }) .collect(Collectors.toList()); - if (!pluginErrorCollector.getPluginErrors().isEmpty()) { + final List extensionPluginErrors = pluginErrorCollector.getPluginErrors() + .stream().filter(pluginError -> PipelinesDataFlowModel.EXTENSION_PLUGIN_TYPE + .equals(pluginError.getComponentType())) + .collect(Collectors.toList()); + if (!extensionPluginErrors.isEmpty()) { + pluginErrorsHandler.handleErrors(extensionPluginErrors); throw new InvalidPluginConfigurationException( - "One or more extension plugins are not configured correctly.\n" - + pluginErrorCollector.getConsolidatedErrorMessage()); + "One or more extension plugins are not configured correctly."); } else { return result; } diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionLoaderTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionLoaderTest.java index dc47a4f698..bf46579377 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionLoaderTest.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionLoaderTest.java @@ -18,6 +18,7 @@ import org.mockito.Captor; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.configuration.PipelinesDataFlowModel; import org.opensearch.dataprepper.model.plugin.ExtensionPlugin; import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.plugin.InvalidPluginDefinitionException; @@ -25,6 +26,7 @@ import org.opensearch.dataprepper.plugins.test.TestExtensionWithConfig; import org.opensearch.dataprepper.validation.PluginError; import org.opensearch.dataprepper.validation.PluginErrorCollector; +import org.opensearch.dataprepper.validation.PluginErrorsHandler; import java.util.ArrayList; import java.util.Collection; @@ -57,13 +59,17 @@ class ExtensionLoaderTest { private ExtensionClassProvider extensionClassProvider; @Mock private PluginCreator extensionPluginCreator; + @Mock + private PluginErrorsHandler pluginErrorsHandler; @Captor private ArgumentCaptor pluginArgumentsContextArgumentCaptor; + @Captor + private ArgumentCaptor> pluginErrorsArgumentCaptor; private PluginErrorCollector pluginErrorCollector; private ExtensionLoader createObjectUnderTest() { return new ExtensionLoader(extensionPluginConfigurationConverter, extensionClassProvider, - extensionPluginCreator, pluginErrorCollector); + extensionPluginCreator, pluginErrorCollector, pluginErrorsHandler); } @BeforeEach @@ -105,6 +111,12 @@ void loadExtensions_returns_single_extension_for_single_plugin_class() { @Test void loadExtensions_throws_InvalidPluginConfigurationException_when_invoking_newPluginInstance_throws_exception() { + pluginErrorCollector.collectPluginError( + PluginError.builder() + .componentType("non-extension") + .pluginName("preexisting-plugin") + .exception(new RuntimeException()) + .build()); final Class pluginClass = (Class) mock(ExtensionPlugin.class).getClass(); when(extensionClassProvider.loadExtensionPluginClasses()).thenReturn(Collections.singleton(pluginClass)); @@ -116,11 +128,16 @@ void loadExtensions_throws_InvalidPluginConfigurationException_when_invoking_new .thenThrow(TestException.class); assertThrows(InvalidPluginConfigurationException.class, () -> createObjectUnderTest().loadExtensions()); - assertThat(pluginErrorCollector.getPluginErrors().size(), equalTo(1)); - final PluginError pluginError = pluginErrorCollector.getPluginErrors().get(0); + assertThat(pluginErrorCollector.getPluginErrors().size(), equalTo(2)); + final PluginError pluginError = pluginErrorCollector.getPluginErrors().get(1); assertThat(pluginError.getPipelineName(), nullValue()); assertThat(pluginError.getPluginName(), CoreMatchers.startsWith("extension_plugin")); assertThat(pluginError.getException(), instanceOf(TestException.class)); + verify(pluginErrorsHandler).handleErrors(pluginErrorsArgumentCaptor.capture()); + final Collection pluginErrorCollection = pluginErrorsArgumentCaptor.getValue(); + assertThat(pluginErrorCollection.size(), equalTo(1)); + final PluginError capturedPluginError = new ArrayList<>(pluginErrorCollection).get(0); + assertThat(capturedPluginError.getComponentType(), equalTo(PipelinesDataFlowModel.EXTENSION_PLUGIN_TYPE)); } @ParameterizedTest From 14770850328a2188945515ad820a7106c91f5833 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 28 Aug 2024 10:20:02 -0500 Subject: [PATCH 139/159] Creates the release notes for Data Prepper 2.9.0 (#4879) Signed-off-by: David Venable --- .../data-prepper.release-notes-2.9.0.md | 57 +++++++++++++++++++ 1 file changed, 57 insertions(+) create mode 100644 release/release-notes/data-prepper.release-notes-2.9.0.md diff --git a/release/release-notes/data-prepper.release-notes-2.9.0.md b/release/release-notes/data-prepper.release-notes-2.9.0.md new file mode 100644 index 0000000000..1c45b7e2d6 --- /dev/null +++ b/release/release-notes/data-prepper.release-notes-2.9.0.md @@ -0,0 +1,57 @@ +## 2024-08-28 Version 2.9.0 + +--- + + +### Features + +* Support sets and set operations in Data Prepper expressions ([#3854](https://github.com/opensearch-project/data-prepper/issues/3854)) +* Add startsWith expression function ([#4840](https://github.com/opensearch-project/data-prepper/pull/4840)) +* Support default route option for Events that match no other route ([#4615](https://github.com/opensearch-project/data-prepper/issues/4615)) +* Delete input for processors which expand the event ([#3968](https://github.com/opensearch-project/data-prepper/issues/3968)) +* Dynamic Rule Detection ([#4600](https://github.com/opensearch-project/data-prepper/issues/4600)) +* Kafka Source should support message headers ([#4565](https://github.com/opensearch-project/data-prepper/issues/4565)) +* Aggregate processor : add option to allow raw events ([#4598](https://github.com/opensearch-project/data-prepper/pull/4598)) +* Add support for start and end times in count and histogram aggregate actions ([#4614](https://github.com/opensearch-project/data-prepper/pull/4614)) +* Add an option to count unique values of specified key(s) to CountAggregateAction ([#4644](https://github.com/opensearch-project/data-prepper/issues/4644)) +* Flatten processor: option for keys wihout brackets ([#4616](https://github.com/opensearch-project/data-prepper/issues/4616)) +* Modify Key Value processor to support string literal grouping ([#4599](https://github.com/opensearch-project/data-prepper/pull/4599)) +* Make AWS credential management available in data-prepper-config.yaml ([#2570](https://github.com/opensearch-project/data-prepper/issues/2570)) + +### Enhancements + +* Support enhanced configuration of the Kafka source and buffer loggers ([#4126](https://github.com/opensearch-project/data-prepper/issues/4126)) +* Update the rename_keys and delete_entries processors to use EventKey ([#4636](https://github.com/opensearch-project/data-prepper/pull/4636)) +* Update the mutate string processors to use the EventKey. ([#4649](https://github.com/opensearch-project/data-prepper/pull/4649)) +* OpenSearch Sink add support for sending pipeline parameter in BulkRequest ([#4609](https://github.com/opensearch-project/data-prepper/pull/4609)) +* Add support for Kafka headers and timestamp in the Kafka Source ([#4566](https://github.com/opensearch-project/data-prepper/pull/4566)) + + +### Bug Fixes + +* [BUG] Visibility duplication protection fails when using S3 source for large files and receiving 10 messages from SQS queue ([#4812](https://github.com/opensearch-project/data-prepper/issues/4812)) +* [BUG] ChangeVisibilityTimeout call failure during pipeline shutdown. ([#4575](https://github.com/opensearch-project/data-prepper/issues/4575)) +* [BUG] Service-map relationship should be created regardless of missing traceGroupName ([#4821](https://github.com/opensearch-project/data-prepper/issues/4821)) +* [BUG] Unable to create stateful processors with multiple workers. ([#4660](https://github.com/opensearch-project/data-prepper/issues/4660)) +* [BUG] Routes: regex doesn't work ([#4763](https://github.com/opensearch-project/data-prepper/issues/4763)) +* [BUG] Grok plugin CLOUDFRONT_ACCESS_LOG pattern does not compile ([#4604](https://github.com/opensearch-project/data-prepper/issues/4604)) +* [BUG] The user_agent processor throws exceptions with multiple threads. ([#4618](https://github.com/opensearch-project/data-prepper/issues/4618)) +* [BUG] DynamoDB source export converts Numbers ending in 0 to scientific notation ([#3840](https://github.com/opensearch-project/data-prepper/issues/3840)) +* Fix null document in DLQ object ([#4814](https://github.com/opensearch-project/data-prepper/pull/4814)) +* Fix KeyValue Processor value grouping bug ([#4606](https://github.com/opensearch-project/data-prepper/pull/4606)) + + +### Security + +* CVE-2024-6345 (High) detected in setuptools-68.0.0-py3-none-any.whl ([#4738](https://github.com/opensearch-project/data-prepper/issues/4738)) +* CVE-2024-39689 (High) detected in certifi-2023.7.22-py3-none-any.whl ([#4715](https://github.com/opensearch-project/data-prepper/issues/4715)) +* CVE-2024-5569 (Low) detected in zipp-3.15.0-py3-none-any.whl ([#4714](https://github.com/opensearch-project/data-prepper/issues/4714)) +* CVE-2024-3651 (High) detected in idna-3.3-py3-none-any.whl ([#4713](https://github.com/opensearch-project/data-prepper/issues/4713)) +* CVE-2024-35195 (Medium) detected in requests-2.31.0-py3-none-any.whl ([#4562](https://github.com/opensearch-project/data-prepper/issues/4562)) +* CVE-2024-37891 (Medium) detected in urllib3-2.0.7-py3-none-any.whl ([#4641](https://github.com/opensearch-project/data-prepper/issues/4641)) + + +### Maintenance + +* Removes Zookeeper from Data Prepper ([#4707](https://github.com/opensearch-project/data-prepper/pull/4707)) +* Tools to generate User Agent strings in the performance-test project ([#4620](https://github.com/opensearch-project/data-prepper/pull/4620)) From b5358b59edbbe638b8f86155dc4deee7346b8723 Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 29 Aug 2024 10:44:17 -0500 Subject: [PATCH 140/159] Adds a script to help generate the Thank You text for the release blogs. (#4884) Adds a script to help generate the Thank You text for the release blogs. Use en dashes in the Thank You text to meet the standards of the project-website. When there is no name for a GitHub user, don't include a None. Signed-off-by: David Venable Co-authored-by: Hai Yan --- release/script/blog/README.md | 18 +++++++++++++++ .../script/blog/format-release-thank-you.py | 22 +++++++++++++++++++ 2 files changed, 40 insertions(+) create mode 100644 release/script/blog/README.md create mode 100755 release/script/blog/format-release-thank-you.py diff --git a/release/script/blog/README.md b/release/script/blog/README.md new file mode 100644 index 0000000000..93d4819f9d --- /dev/null +++ b/release/script/blog/README.md @@ -0,0 +1,18 @@ +# Generate Thank You text for the blog + +The Data Prepper release blogs include text thanking contributors to the project. +This script makes this easier to accomplish. + +You need two tools installed: + +* [GitHub CLI](https://cli.github.com/) +* Python 3 + + +Copy the following command and paste into your CLI. +Modify the dates for `since` and `until` to include when work for this release start through when the the release work ended. + + +``` +gh api --paginate '/repos/opensearch-project/data-prepper/commits?since=2024-05-16&until=2024-08-26' | jq -r '.[].author.login' | sort | uniq | ./format-release-thank-you.py +``` diff --git a/release/script/blog/format-release-thank-you.py b/release/script/blog/format-release-thank-you.py new file mode 100755 index 0000000000..29e4e506e5 --- /dev/null +++ b/release/script/blog/format-release-thank-you.py @@ -0,0 +1,22 @@ +#!python3 + +# +# Copyright OpenSearch Contributors +# SPDX-License-Identifier: Apache-2.0 +# + +import json +import os +import sys + +authors = [] +for author in sys.stdin: + authors.append(author) + + +for author in sorted(authors, key=str.lower): + user = json.loads(os.popen(f"gh api users/{author}").read()) + if user['name'] != None: + print(f"* [{user['login']}]({user['html_url']}) -- {user['name']}") + else: + print(f"* [{user['login']}]({user['html_url']})") From 8eef2f68a8c54ed2aee8eb50e03fcf7fcc784d4b Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Thu, 29 Aug 2024 13:20:47 -0500 Subject: [PATCH 141/159] Minor updates and bug fixes for RDS source (#4887) * Minor updates and bug fixes to prepare for performance testing Signed-off-by: Hai Yan * Address comments Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- .../plugins/source/rds/RdsService.java | 4 ++-- .../plugins/source/rds/RdsSourceConfig.java | 11 +++++++++++ .../rds/converter/StreamRecordConverter.java | 6 ++++++ .../source/rds/model/ExportObjectKey.java | 4 +++- .../source/rds/stream/StreamScheduler.java | 19 +++++++++++++++++-- .../rds/export/ExportSchedulerTest.java | 4 ++-- .../source/rds/model/ExportObjectKeyTest.java | 2 +- 7 files changed, 42 insertions(+), 8 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java index 48d6859ba4..0431b39372 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsService.java @@ -105,7 +105,7 @@ public void start(Buffer> buffer) { if (sourceConfig.isStreamEnabled()) { BinaryLogClient binaryLogClient = new BinlogClientFactory(sourceConfig, rdsClient, dbMetadata).create(); - if (sourceConfig.getTlsConfig() == null || !sourceConfig.getTlsConfig().isInsecure()) { + if (sourceConfig.isTlsEnabled()) { binaryLogClient.setSSLMode(SSLMode.REQUIRED); } else { binaryLogClient.setSSLMode(SSLMode.DISABLED); @@ -146,7 +146,7 @@ private SchemaManager getSchemaManager(final RdsSourceConfig sourceConfig, final dbMetadata.getPort(), sourceConfig.getAuthenticationConfig().getUsername(), sourceConfig.getAuthenticationConfig().getPassword(), - !sourceConfig.getTlsConfig().isInsecure()); + sourceConfig.isTlsEnabled()); return new SchemaManager(connectionManager); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java index a1cb8c7e2f..65a65a4fde 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/RdsSourceConfig.java @@ -85,6 +85,9 @@ public class RdsSourceConfig { @JsonProperty("tls") private TlsConfig tlsConfig; + @JsonProperty("disable_s3_read_for_leader") + private boolean disableS3ReadForLeader = false; + public String getDbIdentifier() { return dbIdentifier; } @@ -153,6 +156,14 @@ public TlsConfig getTlsConfig() { return tlsConfig; } + public boolean isTlsEnabled() { + return tlsConfig == null || !tlsConfig.isInsecure(); + } + + public boolean isDisableS3ReadForLeader() { + return disableS3ReadForLeader; + } + public AuthenticationConfig getAuthenticationConfig() { return this.authenticationConfig; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java index 51cce6541e..cc1f897bea 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/converter/StreamRecordConverter.java @@ -16,6 +16,7 @@ import java.nio.ByteBuffer; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; +import java.time.Instant; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -63,6 +64,11 @@ public Event convert(final Map rowData, EventMetadata eventMetadata = event.getMetadata(); + // Only set external origination time for stream events, not export + final Instant externalOriginationTime = Instant.ofEpochMilli(eventCreateTimeEpochMillis); + event.getEventHandle().setExternalOriginationTime(externalOriginationTime); + eventMetadata.setExternalOriginationTime(externalOriginationTime); + eventMetadata.setAttribute(EVENT_DATABASE_NAME_METADATA_ATTRIBUTE, databaseName); eventMetadata.setAttribute(EVENT_TABLE_NAME_METADATA_ATTRIBUTE, tableName); eventMetadata.setAttribute(BULK_ACTION_METADATA_ATTRIBUTE, bulkAction.toString()); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java index c69dcc7651..feba4555b2 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java @@ -36,7 +36,9 @@ public static ExportObjectKey fromString(final String objectKeyString) { final String prefix = parts[0]; final String exportTaskId = parts[1]; final String databaseName = parts[2]; - final String tableName = parts[3]; + // fullTableName is in the format of "databaseName.tableName" + final String fullTableName = parts[3]; + final String tableName = fullTableName.split("\\.")[1]; final String numberedFolder = parts[4]; final String fileName = parts[5]; return new ExportObjectKey(prefix, exportTaskId, databaseName, tableName, numberedFolder, fileName); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java index 1886bba451..14d61e6626 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamScheduler.java @@ -22,6 +22,8 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import static org.opensearch.dataprepper.model.source.s3.S3ScanEnvironmentVariables.STOP_S3_SCAN_PROCESSING_PROPERTY; + public class StreamScheduler implements Runnable { @@ -57,18 +59,24 @@ public StreamScheduler(final EnhancedSourceCoordinator sourceCoordinator, @Override public void run() { LOG.debug("Start running Stream Scheduler"); + StreamPartition streamPartition = null; while (!shutdownRequested && !Thread.currentThread().isInterrupted()) { try { final Optional sourcePartition = sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE); if (sourcePartition.isPresent()) { LOG.info("Acquired partition to read from stream"); - final StreamPartition streamPartition = (StreamPartition) sourcePartition.get(); + if (sourceConfig.isDisableS3ReadForLeader()) { + // Primary node that acquires the stream partition will not perform work on the S3 buffer + System.setProperty(STOP_S3_SCAN_PROCESSING_PROPERTY, "true"); + } + + streamPartition = (StreamPartition) sourcePartition.get(); final StreamCheckpointer streamCheckpointer = new StreamCheckpointer(sourceCoordinator, streamPartition, pluginMetrics); binaryLogClient.registerEventListener(new BinlogEventListener( buffer, sourceConfig, pluginMetrics, binaryLogClient, streamCheckpointer, acknowledgementSetManager)); final StreamWorker streamWorker = StreamWorker.create(sourceCoordinator, binaryLogClient, pluginMetrics); - executorService.submit(() -> streamWorker.processStream(streamPartition)); + executorService.submit(() -> streamWorker.processStream((StreamPartition) sourcePartition.get())); } try { @@ -81,6 +89,13 @@ public void run() { } catch (Exception e) { LOG.error("Received an exception during stream processing, backing off and retrying", e); + if (streamPartition != null) { + if (sourceConfig.isDisableS3ReadForLeader()) { + System.clearProperty(STOP_S3_SCAN_PROCESSING_PROPERTY); + } + sourceCoordinator.giveUpPartition(streamPartition); + } + try { Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); } catch (final InterruptedException ex) { diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java index f5036e8890..43f08ff3fc 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportSchedulerTest.java @@ -124,7 +124,7 @@ void test_given_export_partition_and_export_task_id_then_complete_export() throw String tableName = UUID.randomUUID().toString(); // objectKey needs to have this structure: "{prefix}/{export task ID}/{database name}/{table name}/{numbered folder}/{file name}" S3Object s3Object = S3Object.builder() - .key("prefix/" + exportTaskId + "/my_db/" + tableName + "/1/file1" + PARQUET_SUFFIX) + .key("prefix/" + exportTaskId + "/my_db/my_db." + tableName + "/1/file1" + PARQUET_SUFFIX) .build(); when(listObjectsV2Response.contents()).thenReturn(List.of(s3Object)); when(listObjectsV2Response.isTruncated()).thenReturn(false); @@ -185,7 +185,7 @@ void test_given_export_partition_without_export_task_id_then_start_and_complete_ String tableName = UUID.randomUUID().toString(); // objectKey needs to have this structure: "{prefix}/{export task ID}/{database name}/{table name}/{numbered folder}/{file name}" S3Object s3Object = S3Object.builder() - .key("prefix/" + exportTaskId + "/my_db/" + tableName + "/1/file1" + PARQUET_SUFFIX) + .key("prefix/" + exportTaskId + "/my_db/my_db." + tableName + "/1/file1" + PARQUET_SUFFIX) .build(); when(listObjectsV2Response.contents()).thenReturn(List.of(s3Object)); when(listObjectsV2Response.isTruncated()).thenReturn(false); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java index 7056114572..697d721c9b 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java @@ -16,7 +16,7 @@ class ExportObjectKeyTest { @Test void test_fromString_with_valid_input_string() { - final String objectKeyString = "prefix/export-task-id/db-name/table-name/1/file-name.parquet"; + final String objectKeyString = "prefix/export-task-id/db-name/db-name.table-name/1/file-name.parquet"; final ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKeyString); assertThat(exportObjectKey.getPrefix(), equalTo("prefix")); From 2c034a00a8a0bd96ada6e441eb4fc1e239073af4 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Tue, 3 Sep 2024 11:38:08 -0400 Subject: [PATCH 142/159] ENH: respect JsonProperty defaultValue in JsonSchemaConverter (#4889) * ENH: respect JsonProperty defaultValue in JsonSchemaConverter Signed-off-by: George Chen --- .../dataprepper/schemas/JsonSchemaConverter.java | 10 ++++++++++ .../dataprepper/schemas/JsonSchemaConverterTest.java | 12 ++++++++++++ 2 files changed, 22 insertions(+) diff --git a/data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java b/data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java index fe08825af4..0985485952 100644 --- a/data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java +++ b/data-prepper-plugin-schema/src/main/java/org/opensearch/dataprepper/schemas/JsonSchemaConverter.java @@ -1,5 +1,6 @@ package org.opensearch.dataprepper.schemas; +import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.node.ObjectNode; import com.github.victools.jsonschema.generator.FieldScope; @@ -29,6 +30,7 @@ public ObjectNode convertIntoJsonSchema( loadJsonSchemaGeneratorModules(configBuilder); final SchemaGeneratorConfigPart scopeSchemaGeneratorConfigPart = configBuilder.forFields(); overrideInstanceAttributeWithDeprecated(scopeSchemaGeneratorConfigPart); + resolveDefaultValueFromJsonProperty(scopeSchemaGeneratorConfigPart); final SchemaGeneratorConfig config = configBuilder.build(); final SchemaGenerator generator = new SchemaGenerator(config); @@ -49,4 +51,12 @@ private void overrideInstanceAttributeWithDeprecated( } }); } + + private void resolveDefaultValueFromJsonProperty( + final SchemaGeneratorConfigPart scopeSchemaGeneratorConfigPart) { + scopeSchemaGeneratorConfigPart.withDefaultResolver(field -> { + final JsonProperty annotation = field.getAnnotationConsideringFieldAndGetter(JsonProperty.class); + return annotation == null || annotation.defaultValue().isEmpty() ? null : annotation.defaultValue(); + }); + } } diff --git a/data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java b/data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java index d5d172f8c0..67cf0ac527 100644 --- a/data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java +++ b/data-prepper-plugin-schema/src/test/java/org/opensearch/dataprepper/schemas/JsonSchemaConverterTest.java @@ -5,6 +5,7 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.ObjectNode; +import com.fasterxml.jackson.databind.node.TextNode; import com.github.victools.jsonschema.generator.Module; import com.github.victools.jsonschema.generator.OptionPreset; import com.github.victools.jsonschema.generator.SchemaVersion; @@ -14,6 +15,7 @@ import java.util.Collections; import java.util.List; +import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -30,6 +32,13 @@ void testConvertIntoJsonSchemaWithDefaultModules() throws JsonProcessingExceptio final ObjectNode jsonSchemaNode = jsonSchemaConverter.convertIntoJsonSchema( SchemaVersion.DRAFT_2020_12, OptionPreset.PLAIN_JSON, TestConfig.class); assertThat(jsonSchemaNode, instanceOf(ObjectNode.class)); + final JsonNode propertiesNode = jsonSchemaNode.at("/properties"); + assertThat(propertiesNode, instanceOf(ObjectNode.class)); + assertThat(propertiesNode.has("testAttributeWithDefaultValue"), is(true)); + final JsonNode testAttributeWithDefaultValueNode = propertiesNode.at("/testAttributeWithDefaultValue"); + assertThat(testAttributeWithDefaultValueNode, instanceOf(ObjectNode.class)); + assertThat(testAttributeWithDefaultValueNode.has("default"), is(true)); + assertThat(testAttributeWithDefaultValueNode.get("default"), equalTo(TextNode.valueOf("default_value"))); } @Test @@ -53,6 +62,9 @@ static class TestConfig { @JsonProperty("custom_test_attribute") private String testAttributeWithJsonPropertyAnnotation; + @JsonProperty(defaultValue = "default_value") + private String testAttributeWithDefaultValue; + public String getTestAttributeWithGetter() { return testAttributeWithGetter; } From cf9584a37cb96cf802ecae5b5e1351094f5a9a2a Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 4 Sep 2024 14:24:25 -0500 Subject: [PATCH 143/159] Updating the instructions for releasing a new version of Data Prepper. (#4878) This adds instructions for the release setup so that maintainers can have a consistent set of instructions to follow. Specifically, it adds steps for setting up the branch, updating version numbers, creating the release notes and change log. Additionally, this fixes a broken fragment link to the Backporting section of the Developer Guide. Signed-off-by: David Venable --- RELEASING.md | 149 +++++++++++++++++++++++++++++++++++++--- docs/developer_guide.md | 2 +- 2 files changed, 140 insertions(+), 11 deletions(-) diff --git a/RELEASING.md b/RELEASING.md index c8da921782..2d51eb42db 100644 --- a/RELEASING.md +++ b/RELEASING.md @@ -1,18 +1,122 @@ # Releasing This document outlines the process for releasing Data Prepper. +It is a guide for maintainers of the Data Prepper project to release a new version. -## Release prerequisites +## Overview -Be sure you have: +This document has three broad categories of steps to follow: -* Created a release branch. -* Updated the version in the release branch. -* Updated the THIRD-PARTY file. -* Created the release notes file -* Created the changelog file +1. [Release setup](#release-setup) +2. [Performing a release](#performing-a-release) +3. [Post-release](#post-release) -## Performing a release + +## Release setup + +### Branch setup + +Data Prepper uses a release branch for releasing. +The [Developer Guide](docs/developer_guide.md#backporting) discusses this in detail. + +The repository has a release branch for a major/minor version. +Patch versions will continue on the same branch. +For example, Data Prepper `2.6.0` was released from the `2.6` branch. +Additionally, Data Prepper `2.6.1` and `2.6.2` were also released from the `2.6` branch. + +If you are creating a new major/minor release, then you will need to create the release branch. +Use GitHub to create a new branch. + +Steps: +1. Go to the [branches page here](https://github.com/opensearch-project/data-prepper/branches). +2. Select `New branch` +3. Name the branch `{major}.{minor}`. For example, `2.6`. +4. Select `main` as the source. + +Create a backport label for the release branch. + +Steps: +1. Go to the [labels page here](https://github.com/opensearch-project/data-prepper/labels). +2. Select `New label` +3. Name the branch `backport {major}.minor`. For example, `backport 2.6` + +### Update versions + +The Data Prepper version is defined in the [`gradle.properties`](https://github.com/opensearch-project/data-prepper/blob/main/gradle.properties) file. +We must update this whenever we create a new release. +We will need two PRs to update it. + +#### Update on release branch + +For the current release branch, update the version to the new version. +You may either need to update by removing the `-SNAPSHOT` or by updating the patch version. + +For example, when we released `2.6.2`, the property was set as `version=2.6.2`. +You can see the [exact commit here](https://github.com/opensearch-project/data-prepper/blob/2.6.2/gradle.properties#L8). + +Create a PR that targets the release branch with the change. +Do not create a PR to `main` for this. + +#### Update on the main branch + +If you have just created a release branch, you should also create a PR on the `main` branch to bump the version. + +For example, if you have started the `2.7` branch, you will need to update the `main` branch from `2.6.0-SNAPSHOT` to `2.7.0-SNAPSHOT`. + +#### Update the version of DataPrepperVersion + +If you have just created a release branch, you should also create a PR on the `main` branch to bump the version in `DataPrepperVersion`. + +Steps: +1. Modify the `DataPrepperVersion` class to update `CURRENT_VERSION` to the next version. +2. Create a PR targeting `main` + +Note: This step can be automated through [#4877](https://github.com/opensearch-project/data-prepper/issues/4877). + +### Update the THIRD-PARTY file + +We should update the `THIRD-PARTY` file for every release. +Data Prepper has a GitHub action that will generate this and create a PR with the updated file. + +Steps: +* Go the [Third Party Generate action](https://github.com/opensearch-project/data-prepper/actions/workflows/third-party-generate.yml) +* Select `Run workflow` +* Choose the branch you are releasing. e.g. `2.6` +* Press `Run workflow` +* Wait for a new PR to be created +* Spot check the PR, approve and merge + + +### Prepare release notes + +Prepare release notes and check them into the `main` branch in the [`release-notes` directory](https://github.com/opensearch-project/data-prepper/tree/main/release/release-notes). +The format for the release notes file is `data-prepper.release-notes.{major}.{minor}.{patch}.md`. + +You can use a script to help you generate these. +See the [README](release/script/release-notes/README.md) for the script for instructions. + +Once the release notes are ready, create a PR to merge them into `main`. +Also tag this with the `backport {major}.{minor}` to create a PR that you can merge into your release branch. + +### Create changelog + +You can create a changelog using [git-release-notes](https://github.com/ariatemplates/git-release-notes). + +``` +git fetch upstream +git switch {major}.{minor} +git fetch upstream --tags +git pull +git-release-notes {previousMajor}.{previousMinor}.{previousPatch}..HEAD markdown > release/release-notes/data-prepper.change-log-{major}.{minor}.{patch}.md +git switch main +``` + +Once the change log ready, create a PR to merge it into `main`. +Also tag this with the `backport {major}.{minor}` to create a PR that you can merge into your release branch. + + + +## Performing a release This section outlines how to perform a Data Prepper release using GitHub Actions and the OpenSearch build infrastructure. The audience for this section are Data Prepper maintainers. @@ -27,14 +131,14 @@ Select the "Run workflow" option from the GitHub Actions UI. GitHub will prompt #### Use workflow for Select the release branch which you are releasing for. -Typically, this will be a branch such as `2.4`. +Typically, this will be a branch such as `2.6`. However, you may select `main` for testing. #### Whether to create major tag of docker image or not. This will create a tag such as `2` which points to this version -All releases have a full version tag. For example, `2.4.0`. +All releases have a full version tag. For example, `2.6.0`. The latest release on a major series can also have a tag applied such as `2`. Check this option if you are releasing the latest version withing that series of major versions. This value can be true for old major series as well such as the 1.x series. @@ -69,3 +173,28 @@ You can also deny a release by using _deny_ or _denied_ in the comment. ### Further details For more details on the release build, or to setup your own GitHub repository, see [release/README.md](release/README.md). + + + +## Post release + +After the release, there are a few other steps to clean up the repository. + +### Update the release notes + +The release process will have created a draft release for the new version. +The next step is to update the draft release with the release notes created before the release. + +Steps: +* Go to the [releases page](https://github.com/opensearch-project/data-prepper/releases) +* Find the new draft release. It should be at the top. +* Replace the auto-generated release notes with the release notes created previous to the release. +* Release it + +### Close the GitHub milestone + +Steps: +* Go to the [milestones](https://github.com/opensearch-project/data-prepper/milestones) page. +* Find the milestone for the release. +* Make sure there are no issues. If there are any triage them by closing, or changing the milestone. +* Click the "Close" button diff --git a/docs/developer_guide.md b/docs/developer_guide.md index 1a6587d5b3..098ee5a0d3 100644 --- a/docs/developer_guide.md +++ b/docs/developer_guide.md @@ -134,7 +134,7 @@ Near the time of the next release, we create a release branch for that upcoming release (e.g. `1.2`). We perform our release builds from this branch. Any patch releases also build from that release branch. -### Backporting +### Backporting When you create a PR which targets `main` and need this change as a patch to a previous version of Data Prepper, use the auto backport GitHub Action. All you need to do is add the label From 7f7d2dd00ec48939ea3735c8f55fee56a5aed70a Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 4 Sep 2024 14:24:52 -0500 Subject: [PATCH 144/159] Adds a .gitignore for Python virtual environments. (#4881) Some projects such as the trace-analytics-sample-app and the current smoke tests use Python. Using Python virtual environments lets developers use these without affecting their local Python environment. Ignore .venv directories which Python virtual environments use. Signed-off-by: David Venable --- .gitignore | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index 086c830d15..6bf57412d2 100644 --- a/.gitignore +++ b/.gitignore @@ -13,3 +13,6 @@ gradle/tools # Ignore config file generated by test data-prepper-main/src/test/resources/logstash-filter.yaml data-prepper-main/src/test/resources/logstash-conf/logstash-filter.yaml + +# Python virtual environments +.venv From 82d811ab76b4a268616fd1080c85079aa1116120 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 4 Sep 2024 16:18:09 -0500 Subject: [PATCH 145/159] Use awaitility to read data in Kafka buffer tests to fix flakiness (#4703) Use awaitility to read data in KafkaBufferIT to promote stability and speed of execution. Contributes toward #4168 Signed-off-by: David Venable --- .../kafka-plugin-integration-tests.yml | 2 +- .../plugins/kafka/KafkaStartIT.java | 2 +- .../plugins/kafka/buffer/KafkaBufferIT.java | 23 ++++++--------- .../kafka/buffer/KafkaBuffer_KmsIT.java | 5 ++-- .../kafka/buffer/ReadBufferHelper.java | 29 +++++++++++++++++++ 5 files changed, 43 insertions(+), 18 deletions(-) create mode 100644 data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/ReadBufferHelper.java diff --git a/.github/workflows/kafka-plugin-integration-tests.yml b/.github/workflows/kafka-plugin-integration-tests.yml index 507ca1d435..72d6645d53 100644 --- a/.github/workflows/kafka-plugin-integration-tests.yml +++ b/.github/workflows/kafka-plugin-integration-tests.yml @@ -41,7 +41,7 @@ jobs: run: | echo 'KAFKA_VERSION=${{ matrix.kafka }}' > data-prepper-plugins/kafka-plugins/src/integrationTest/resources/kafka/.env docker compose --project-directory data-prepper-plugins/kafka-plugins/src/integrationTest/resources/kafka/zookeeper --env-file data-prepper-plugins/kafka-plugins/src/integrationTest/resources/kafka/.env up -d - sleep 10 + sleep 2 - name: Wait for Kafka run: | ./gradlew data-prepper-plugins:kafka-plugins:integrationTest -Dtests.kafka.bootstrap_servers=localhost:9092 -Dtests.kafka.authconfig.username=admin -Dtests.kafka.authconfig.password=admin --tests KafkaStartIT diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/KafkaStartIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/KafkaStartIT.java index 1963f8910f..e8d6ce7000 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/KafkaStartIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/KafkaStartIT.java @@ -29,7 +29,7 @@ void waitForKafka() { try (AdminClient adminClient = AdminClient.create(props)) { await().atMost(Duration.ofMinutes(3)) .pollDelay(Duration.ofSeconds(2)) - .untilAsserted(() -> adminClient.listTopics().names().get()); + .until(() -> adminClient.listTopics().names().get() != null); } } } diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBufferIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBufferIT.java index 261008e050..b44355810b 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBufferIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBufferIT.java @@ -25,12 +25,10 @@ import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; -import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.kafka.configuration.KafkaProducerProperties; import org.opensearch.dataprepper.plugins.kafka.util.TestConsumer; import org.opensearch.dataprepper.plugins.kafka.util.TestProducer; -import org.opensearch.dataprepper.model.codec.JsonDecoder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -64,6 +62,7 @@ import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.kafka.buffer.ReadBufferHelper.awaitRead; import static org.opensearch.dataprepper.test.helper.ReflectivelySetField.setField; @ExtendWith(MockitoExtension.class) @@ -74,8 +73,6 @@ public class KafkaBufferIT { private KafkaBufferConfig kafkaBufferConfig; @Mock - private PluginFactory pluginFactory; - @Mock private AcknowledgementSetManager acknowledgementSetManager; @Mock private AcknowledgementSet acknowledgementSet; @@ -95,9 +92,7 @@ void setUp() { random = new Random(); acknowledgementSetManager = mock(AcknowledgementSetManager.class); acknowledgementSet = mock(AcknowledgementSet.class); - lenient().doAnswer((a) -> { - return null; - }).when(acknowledgementSet).complete(); + lenient().doAnswer((a) -> null).when(acknowledgementSet).complete(); lenient().when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); objectMapper = new ObjectMapper().registerModule(new JavaTimeModule()); @@ -142,7 +137,7 @@ void write_and_read() throws TimeoutException { Record record = createRecord(); objectUnderTest.write(record, 1_000); - Map.Entry>, CheckpointState> readResult = objectUnderTest.read(10_000); + final Map.Entry>, CheckpointState> readResult = awaitRead(objectUnderTest); assertThat(readResult, notNullValue()); assertThat(readResult.getKey(), notNullValue()); @@ -183,7 +178,7 @@ void write_and_read_max_request_test() throws TimeoutException, NoSuchFieldExcep Record record = createLargeRecord(); objectUnderTest.write(record, 1_000); - Map.Entry>, CheckpointState> readResult = objectUnderTest.read(10_000); + Map.Entry>, CheckpointState> readResult = awaitRead(objectUnderTest); assertThat(readResult, notNullValue()); assertThat(readResult.getKey(), notNullValue()); @@ -213,7 +208,7 @@ void writeBigJson_and_read() throws Exception { inputJson += "]"; objectUnderTest.writeBytes(inputJson.getBytes(), null, 1_000); - Map.Entry>, CheckpointState> readResult = objectUnderTest.read(10_000); + final Map.Entry>, CheckpointState> readResult = awaitRead(objectUnderTest); assertThat(readResult, notNullValue()); assertThat(readResult.getKey(), notNullValue()); @@ -243,7 +238,7 @@ void writeMultipleSmallJson_and_read() throws Exception { objectUnderTest.writeBytes(inputJson.getBytes(), null, 1_000); } - Map.Entry>, CheckpointState> readResult = objectUnderTest.read(10_000); + final Map.Entry>, CheckpointState> readResult = awaitRead(objectUnderTest); assertThat(readResult, notNullValue()); assertThat(readResult.getKey(), notNullValue()); @@ -273,7 +268,7 @@ void writeBytes_and_read() throws Exception { final String key = UUID.randomUUID().toString(); objectUnderTest.writeBytes(bytes, key, 1_000); - Map.Entry>, CheckpointState> readResult = objectUnderTest.read(10_000); + final Map.Entry>, CheckpointState> readResult = awaitRead(objectUnderTest); assertThat(readResult, notNullValue()); assertThat(readResult.getKey(), notNullValue()); @@ -398,7 +393,7 @@ void write_and_read_encrypted() throws TimeoutException { Record record = createRecord(); objectUnderTest.write(record, 1_000); - Map.Entry>, CheckpointState> readResult = objectUnderTest.read(10_000); + final Map.Entry>, CheckpointState> readResult = awaitRead(objectUnderTest); assertThat(readResult, notNullValue()); assertThat(readResult.getKey(), notNullValue()); @@ -526,7 +521,7 @@ void read_decrypts_data_from_the_predefined_key() throws IllegalBlockSizeExcepti testProducer.publishRecord(keyData.toByteArray(), bufferedData.toByteArray()); - final Map.Entry>, CheckpointState> readResult = objectUnderTest.read(10_000); + final Map.Entry>, CheckpointState> readResult = awaitRead(objectUnderTest); assertThat(readResult, notNullValue()); assertThat(readResult.getKey(), notNullValue()); diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer_KmsIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer_KmsIT.java index 464d7966e3..8a57765efb 100644 --- a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer_KmsIT.java +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer_KmsIT.java @@ -59,6 +59,7 @@ import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.kafka.buffer.ReadBufferHelper.awaitRead; @ExtendWith(MockitoExtension.class) public class KafkaBuffer_KmsIT { @@ -177,7 +178,7 @@ void write_and_read_encrypted() throws TimeoutException { Record record = createRecord(); objectUnderTest.write(record, 1_000); - Map.Entry>, CheckpointState> readResult = objectUnderTest.read(10_000); + final Map.Entry>, CheckpointState> readResult = awaitRead(objectUnderTest); assertThat(readResult, notNullValue()); assertThat(readResult.getKey(), notNullValue()); @@ -216,7 +217,7 @@ void read_decrypts_data_from_the_predefined_key() throws IllegalBlockSizeExcepti testProducer.publishRecord(keyData.toByteArray(), bufferedData.toByteArray()); - final Map.Entry>, CheckpointState> readResult = objectUnderTest.read(10_000); + final Map.Entry>, CheckpointState> readResult = awaitRead(objectUnderTest); assertThat(readResult, notNullValue()); assertThat(readResult.getKey(), notNullValue()); diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/ReadBufferHelper.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/ReadBufferHelper.java new file mode 100644 index 0000000000..7c325a18b6 --- /dev/null +++ b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/buffer/ReadBufferHelper.java @@ -0,0 +1,29 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.kafka.buffer; + +import org.opensearch.dataprepper.model.CheckpointState; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; + +import java.time.Duration; +import java.util.Collection; +import java.util.Map; + +import static org.awaitility.Awaitility.await; + +class ReadBufferHelper { + static Map.Entry>, CheckpointState> awaitRead(final KafkaBuffer objectUnderTest) { + final Map.Entry>, CheckpointState>[] lastReadResult = new Map.Entry[1]; + await() + .atMost(Duration.ofSeconds(30)) + .until(() -> { + lastReadResult[0] = objectUnderTest.read(500); + return lastReadResult[0] != null && lastReadResult[0].getKey() != null && lastReadResult[0].getKey().size() >= 1; + }); + return lastReadResult[0]; + } +} From 608fd158629184733fe7847d74abc49fe3185d4b Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 4 Sep 2024 16:30:28 -0500 Subject: [PATCH 146/159] Corrects the S3SinkStack for AWS testing (#4913) Corrects the S3SinkStack for AWS testing. The S3SinkStack was not in use and didn't quite work. This corrects it so that we can use it to automate the tests for the S3 sink in GitHub. Signed-off-by: David Venable --- testing/aws-testing-cdk/bin/aws-testing-cdk.ts | 5 +++++ testing/aws-testing-cdk/lib/s3/S3SinkStack.ts | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/testing/aws-testing-cdk/bin/aws-testing-cdk.ts b/testing/aws-testing-cdk/bin/aws-testing-cdk.ts index 13452ce0f7..fcdd4914ae 100644 --- a/testing/aws-testing-cdk/bin/aws-testing-cdk.ts +++ b/testing/aws-testing-cdk/bin/aws-testing-cdk.ts @@ -4,6 +4,7 @@ import * as cdk from 'aws-cdk-lib'; import {GitHubAccessStack} from '../lib/common/GitHubAccessStack'; import {SecretsManagerStack} from '../lib/aws-secrets-manager/SecretsManagerStack'; import {KmsStack} from '../lib/common/KmsStack'; +import {S3SinkStack} from '../lib/s3/S3SinkStack'; const app = new cdk.App(); @@ -16,3 +17,7 @@ new KmsStack(app, 'CommonKmsStack', { new SecretsManagerStack(app, 'SecretsManagerStack', { testingRole: githubStack.gitHubActionsTestingRole }); + +new S3SinkStack(app, 'S3SinkStack', { + testingRole: githubStack.gitHubActionsTestingRole +}); diff --git a/testing/aws-testing-cdk/lib/s3/S3SinkStack.ts b/testing/aws-testing-cdk/lib/s3/S3SinkStack.ts index ea6d22615f..89fe05d836 100644 --- a/testing/aws-testing-cdk/lib/s3/S3SinkStack.ts +++ b/testing/aws-testing-cdk/lib/s3/S3SinkStack.ts @@ -21,7 +21,7 @@ export class S3SinkStack extends Stack { constructor(scope: Construct, id: string, props: S3SinkStackProps) { super(scope, id, props); - new Bucket(this, 'MyBucket', { + this.bucket = new Bucket(this, 'DataPrepperSinkTest', { removalPolicy: RemovalPolicy.DESTROY, lifecycleRules: [ { From 6bde685312557ecde4ee506e0d74417b6bf0f6c5 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Thu, 5 Sep 2024 10:58:59 -0500 Subject: [PATCH 147/159] Move event processing to separate threads and add event processing timer in RDS source (#4914) * Move event processing to separate threads and add event processing timer Signed-off-by: Hai Yan * Address comments Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- .../rds/configuration/StreamConfig.java | 10 ++++ .../rds/stream/BinlogEventListener.java | 47 +++++++++++++------ .../rds/stream/BinlogEventListenerTest.java | 46 +++++++++++++++++- .../rds/stream/StreamSchedulerTest.java | 3 ++ 4 files changed, 90 insertions(+), 16 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/StreamConfig.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/StreamConfig.java index c246e56b45..f2b5af11a1 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/StreamConfig.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/StreamConfig.java @@ -12,13 +12,23 @@ public class StreamConfig { private static final int DEFAULT_S3_FOLDER_PARTITION_COUNT = 100; + private static final int DEFAULT_NUM_WORKERS = 1; @JsonProperty("partition_count") @Min(1) @Max(1000) private int s3FolderPartitionCount = DEFAULT_S3_FOLDER_PARTITION_COUNT; + @JsonProperty("workers") + @Min(1) + @Max(1000) + private int numWorkers = DEFAULT_NUM_WORKERS; + public int getPartitionCount() { return s3FolderPartitionCount; } + + public int getNumWorkers() { + return numWorkers; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java index 181716a69a..33defb42b7 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java @@ -16,7 +16,9 @@ import com.github.shyiko.mysql.binlog.event.WriteRowsEventData; import io.micrometer.core.instrument.Counter; import io.micrometer.core.instrument.DistributionSummary; +import io.micrometer.core.instrument.Timer; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.common.concurrent.BackgroundThreadFactory; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; @@ -38,6 +40,8 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -51,6 +55,7 @@ public class BinlogEventListener implements BinaryLogClient.EventListener { static final String CHANGE_EVENTS_PROCESSING_ERROR_COUNT = "changeEventsProcessingErrors"; static final String BYTES_RECEIVED = "bytesReceived"; static final String BYTES_PROCESSED = "bytesProcessed"; + static final String REPLICATION_LOG_EVENT_PROCESSING_TIME = "replicationLogEntryProcessingTime"; /** * TableId to TableMetadata mapping @@ -59,17 +64,20 @@ public class BinlogEventListener implements BinaryLogClient.EventListener { private final StreamRecordConverter recordConverter; private final BinaryLogClient binaryLogClient; - private final BufferAccumulator> bufferAccumulator; + private final Buffer> buffer; private final List tableNames; private final String s3Prefix; private final boolean isAcknowledgmentsEnabled; private final PluginMetrics pluginMetrics; private final List pipelineEvents; private final StreamCheckpointManager streamCheckpointManager; + private final ExecutorService binlogEventExecutorService; private final Counter changeEventSuccessCounter; private final Counter changeEventErrorCounter; private final DistributionSummary bytesReceivedSummary; private final DistributionSummary bytesProcessedSummary; + private final Timer eventProcessingTimer; + /** * currentBinlogCoordinate is the coordinate where next event will start @@ -82,15 +90,17 @@ public BinlogEventListener(final Buffer> buffer, final BinaryLogClient binaryLogClient, final StreamCheckpointer streamCheckpointer, final AcknowledgementSetManager acknowledgementSetManager) { + this.buffer = buffer; this.binaryLogClient = binaryLogClient; tableMetadataMap = new HashMap<>(); recordConverter = new StreamRecordConverter(sourceConfig.getStream().getPartitionCount()); - bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); s3Prefix = sourceConfig.getS3Prefix(); tableNames = sourceConfig.getTableNames(); isAcknowledgmentsEnabled = sourceConfig.isAcknowledgmentsEnabled(); this.pluginMetrics = pluginMetrics; pipelineEvents = new ArrayList<>(); + binlogEventExecutorService = Executors.newFixedThreadPool( + sourceConfig.getStream().getNumWorkers(), BackgroundThreadFactory.defaultExecutorThreadFactory("rds-source-binlog-processor")); this.streamCheckpointManager = new StreamCheckpointManager( streamCheckpointer, sourceConfig.isAcknowledgmentsEnabled(), @@ -101,6 +111,7 @@ public BinlogEventListener(final Buffer> buffer, changeEventErrorCounter = pluginMetrics.counter(CHANGE_EVENTS_PROCESSING_ERROR_COUNT); bytesReceivedSummary = pluginMetrics.summary(BYTES_RECEIVED); bytesProcessedSummary = pluginMetrics.summary(BYTES_PROCESSED); + eventProcessingTimer = pluginMetrics.timer(REPLICATION_LOG_EVENT_PROCESSING_TIME); } @Override @@ -109,22 +120,22 @@ public void onEvent(com.github.shyiko.mysql.binlog.event.Event event) { switch (eventType) { case ROTATE: - handleEventAndErrors(event, this::handleRotateEvent); + processEvent(event, this::handleRotateEvent); break; case TABLE_MAP: - handleEventAndErrors(event, this::handleTableMapEvent); + processEvent(event, this::handleTableMapEvent); break; case WRITE_ROWS: case EXT_WRITE_ROWS: - handleEventAndErrors(event, this::handleInsertEvent); + processEvent(event, this::handleInsertEvent); break; case UPDATE_ROWS: case EXT_UPDATE_ROWS: - handleEventAndErrors(event, this::handleUpdateEvent); + processEvent(event, this::handleUpdateEvent); break; case DELETE_ROWS: case EXT_DELETE_ROWS: - handleEventAndErrors(event, this::handleDeleteEvent); + processEvent(event, this::handleDeleteEvent); break; } } @@ -132,6 +143,7 @@ public void onEvent(com.github.shyiko.mysql.binlog.event.Event event) { public void stopClient() { try { binaryLogClient.disconnect(); + binlogEventExecutorService.shutdownNow(); LOG.info("Binary log client disconnected."); } catch (Exception e) { LOG.error("Binary log client failed to disconnect.", e); @@ -150,6 +162,7 @@ void handleRotateEvent(com.github.shyiko.mysql.binlog.event.Event event) { } } } + void handleTableMapEvent(com.github.shyiko.mysql.binlog.event.Event event) { final TableMapEventData data = event.getData(); final TableMapEventMetadata tableMapEventMetadata = data.getEventMetadata(); @@ -223,6 +236,8 @@ private void handleRowChangeEvent(com.github.shyiko.mysql.binlog.event.Event eve final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = event.getHeader().getTimestamp(); + final BufferAccumulator> bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); + for (Object[] rowDataArray : rows) { final Map rowDataMap = new HashMap<>(); for (int i = 0; i < rowDataArray.length; i++) { @@ -242,7 +257,7 @@ private void handleRowChangeEvent(com.github.shyiko.mysql.binlog.event.Event eve pipelineEvents.add(pipelineEvent); } - writeToBuffer(acknowledgementSet); + writeToBuffer(bufferAccumulator, acknowledgementSet); bytesProcessedSummary.record(bytes); if (isAcknowledgmentsEnabled) { @@ -256,19 +271,19 @@ private boolean isTableOfInterest(String tableName) { return new HashSet<>(tableNames).contains(tableName); } - private void writeToBuffer(AcknowledgementSet acknowledgementSet) { + private void writeToBuffer(BufferAccumulator> bufferAccumulator, AcknowledgementSet acknowledgementSet) { for (Event pipelineEvent : pipelineEvents) { - addToBufferAccumulator(new Record<>(pipelineEvent)); + addToBufferAccumulator(bufferAccumulator, new Record<>(pipelineEvent)); if (acknowledgementSet != null) { acknowledgementSet.add(pipelineEvent); } } - flushBufferAccumulator(pipelineEvents.size()); + flushBufferAccumulator(bufferAccumulator, pipelineEvents.size()); pipelineEvents.clear(); } - private void addToBufferAccumulator(final Record record) { + private void addToBufferAccumulator(final BufferAccumulator> bufferAccumulator, final Record record) { try { bufferAccumulator.add(record); } catch (Exception e) { @@ -276,7 +291,7 @@ private void addToBufferAccumulator(final Record record) { } } - private void flushBufferAccumulator(int eventCount) { + private void flushBufferAccumulator(BufferAccumulator> bufferAccumulator, int eventCount) { try { bufferAccumulator.flush(); changeEventSuccessCounter.increment(eventCount); @@ -288,10 +303,14 @@ private void flushBufferAccumulator(int eventCount) { } } + private void processEvent(com.github.shyiko.mysql.binlog.event.Event event, Consumer function) { + binlogEventExecutorService.submit(() -> handleEventAndErrors(event, function)); + } + private void handleEventAndErrors(com.github.shyiko.mysql.binlog.event.Event event, Consumer function) { try { - function.accept(event); + eventProcessingTimer.record(() -> function.accept(event)); } catch (Exception e) { LOG.error("Failed to process change event of type {}", event.getHeader().getEventType(), e); } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java index d33ecd3a70..c287ec2f00 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListenerTest.java @@ -7,13 +7,17 @@ import com.github.shyiko.mysql.binlog.BinaryLogClient; import com.github.shyiko.mysql.binlog.event.EventType; +import io.micrometer.core.instrument.Metrics; +import io.micrometer.core.instrument.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; import org.mockito.Answers; +import org.mockito.ArgumentCaptor; import org.mockito.Mock; +import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; @@ -22,10 +26,18 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.rds.stream.BinlogEventListener.REPLICATION_LOG_EVENT_PROCESSING_TIME; @ExtendWith(MockitoExtension.class) class BinlogEventListenerTest { @@ -48,14 +60,32 @@ class BinlogEventListenerTest { @Mock private AcknowledgementSetManager acknowledgementSetManager; + @Mock + private ExecutorService eventListnerExecutorService; + + @Mock + private ExecutorService checkpointManagerExecutorService; + + @Mock + private ThreadFactory threadFactory; + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private com.github.shyiko.mysql.binlog.event.Event binlogEvent; - private static BinlogEventListener objectUnderTest; + private BinlogEventListener objectUnderTest; + + private Timer eventProcessingTimer; @BeforeEach void setUp() { - objectUnderTest = spy(createObjectUnderTest()); + eventProcessingTimer = Metrics.timer("test-timer"); + when(pluginMetrics.timer(REPLICATION_LOG_EVENT_PROCESSING_TIME)).thenReturn(eventProcessingTimer); + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { + executorsMockedStatic.when(() -> Executors.newFixedThreadPool(anyInt(), any(ThreadFactory.class))).thenReturn(eventListnerExecutorService); + executorsMockedStatic.when(Executors::newSingleThreadExecutor).thenReturn(checkpointManagerExecutorService); + executorsMockedStatic.when(Executors::defaultThreadFactory).thenReturn(threadFactory); + objectUnderTest = spy(createObjectUnderTest()); + } } @Test @@ -65,6 +95,7 @@ void test_given_TableMap_event_then_calls_correct_handler() { objectUnderTest.onEvent(binlogEvent); + verifyHandlerCallHelper(); verify(objectUnderTest).handleTableMapEvent(binlogEvent); } @@ -76,6 +107,7 @@ void test_given_WriteRows_event_then_calls_correct_handler(EventType eventType) objectUnderTest.onEvent(binlogEvent); + verifyHandlerCallHelper(); verify(objectUnderTest).handleInsertEvent(binlogEvent); } @@ -87,6 +119,7 @@ void test_given_UpdateRows_event_then_calls_correct_handler(EventType eventType) objectUnderTest.onEvent(binlogEvent); + verifyHandlerCallHelper(); verify(objectUnderTest).handleUpdateEvent(binlogEvent); } @@ -98,10 +131,19 @@ void test_given_DeleteRows_event_then_calls_correct_handler(EventType eventType) objectUnderTest.onEvent(binlogEvent); + verifyHandlerCallHelper(); verify(objectUnderTest).handleDeleteEvent(binlogEvent); } private BinlogEventListener createObjectUnderTest() { return new BinlogEventListener(buffer, sourceConfig, pluginMetrics, binaryLogClient, streamCheckpointer, acknowledgementSetManager); } + + private void verifyHandlerCallHelper() { + ArgumentCaptor runnableArgumentCaptor = ArgumentCaptor.forClass(Runnable.class); + verify(eventListnerExecutorService).submit(runnableArgumentCaptor.capture()); + + Runnable capturedRunnable = runnableArgumentCaptor.getValue(); + capturedRunnable.run(); + } } \ No newline at end of file diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java index 3c8b70cab2..0414495f6c 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamSchedulerTest.java @@ -42,6 +42,7 @@ class StreamSchedulerTest { @Mock private EnhancedSourceCoordinator sourceCoordinator; + @Mock(answer = Answers.RETURNS_DEEP_STUBS) private RdsSourceConfig sourceConfig; @@ -85,6 +86,8 @@ void test_given_stream_partition_then_start_stream() throws InterruptedException StreamWorker streamWorker = mock(StreamWorker.class); doNothing().when(streamWorker).processStream(streamPartition); + when(sourceConfig.getStream().getNumWorkers()).thenReturn(1); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(() -> { try (MockedStatic streamWorkerMockedStatic = mockStatic(StreamWorker.class)) { From 3dfad0b88bca23d6c09b97702f3a19285bef6ad0 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Fri, 6 Sep 2024 14:24:03 -0400 Subject: [PATCH 148/159] MAINT: add data prepper plugin schema module into build resource (#4920) Signed-off-by: George Chen --- build-resources.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/build-resources.gradle b/build-resources.gradle index 446f1d97b2..7572abf5c7 100644 --- a/build-resources.gradle +++ b/build-resources.gradle @@ -15,5 +15,6 @@ ext.coreProjects = [ project(':data-prepper-test-common'), project(':data-prepper-test-event'), project(':data-prepper-plugin-framework'), + project(':data-prepper-plugin-schema'), project(':data-prepper-plugin-schema-cli') ] From cf24b895b8c4a42fbbe4596e5836c714336244b0 Mon Sep 17 00:00:00 2001 From: David Venable Date: Mon, 9 Sep 2024 11:27:44 -0500 Subject: [PATCH 149/159] Updates to the code for HTTP chunking. (#4919) This refactors the code by placing all logic for serializing the data into the Codec itself. In so doing, it allows for improved testing such as symmetric testing. It also decouples the serialization format from the HTTP server. This also uses the Jackson library for the serialization which yields more accurate JSON. Signed-off-by: David Venable --- .../dataprepper/http/codec/Codec.java | 30 ++- .../dataprepper/http/codec/JsonCodec.java | 122 +++++++--- .../dataprepper/http/codec/JsonCodecTest.java | 213 +++++++++++++++--- .../source/loghttp/LogHTTPService.java | 73 +++--- 4 files changed, 327 insertions(+), 111 deletions(-) diff --git a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/Codec.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/Codec.java index 3a71abbd3d..49b28b9aa2 100644 --- a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/Codec.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/Codec.java @@ -8,6 +8,7 @@ import com.linecorp.armeria.common.HttpData; import java.io.IOException; +import java.util.function.Consumer; /** * Codec parses the content of HTTP request into custom Java type. @@ -22,7 +23,32 @@ public interface Codec { */ T parse(HttpData httpData) throws IOException; - default T parse(HttpData httpData, int maxSize) throws IOException { - return parse(httpData); + /** + * Serializes parsed data back into a UTF-8 string. + *

+ * This API will split into multiple bodies based on splitLength. Note that if a single + * item is larger than this, it will be output and exceed that length. + * + * @param parsedData The parsed data + * @param serializedBodyConsumer A {@link Consumer} to accept each serialized body + * @param splitLength The length at which to split serialized bodies. + * @throws IOException A failure writing data. + */ + void serialize(final T parsedData, + final Consumer serializedBodyConsumer, + final int splitLength) throws IOException; + + + /** + * Serializes parsed data back into a UTF-8 string. + *

+ * This API will not split the data into chunks. + * + * @param parsedData The parsed data + * @param serializedBodyConsumer A {@link Consumer} to accept the serialized body + * @throws IOException A failure writing data. + */ + default void serialize(final T parsedData, final Consumer serializedBodyConsumer) throws IOException { + serialize(parsedData, serializedBodyConsumer, Integer.MAX_VALUE); } } diff --git a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java index 9a74cdb767..378af9c2d9 100644 --- a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java @@ -5,68 +5,126 @@ package org.opensearch.dataprepper.http.codec; +import com.fasterxml.jackson.core.JsonEncoding; +import com.fasterxml.jackson.core.JsonFactory; +import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.io.CountingOutputStream; import com.linecorp.armeria.common.HttpData; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.function.Consumer; /** * JsonCodec parses the json array format HTTP data into List<{@link String}>. * TODO: replace output List<String> with List<InternalModel> type *

*/ -public class JsonCodec implements Codec>> { - // To account for "[" and "]" when the list is converted to String - private static final String OVERHEAD_CHARACTERS = "[]"; - // To account for "," when the list is converted to String - private static final int COMMA_OVERHEAD_LENGTH = 1; +public class JsonCodec implements Codec> { private static final ObjectMapper mapper = new ObjectMapper(); private static final TypeReference>> LIST_OF_MAP_TYPE_REFERENCE = - new TypeReference>>() {}; + new TypeReference>>() { + }; + @Override - public List> parse(HttpData httpData) throws IOException { - List jsonList = new ArrayList<>(); + public List parse(final HttpData httpData) throws IOException { + final List jsonList = new ArrayList<>(); final List> logList = mapper.readValue(httpData.toInputStream(), LIST_OF_MAP_TYPE_REFERENCE); - for (final Map log: logList) { + for (final Map log : logList) { final String recordString = mapper.writeValueAsString(log); jsonList.add(recordString); } - return List.of(jsonList); + return jsonList; } - @Override - public List> parse(HttpData httpData, int maxSize) throws IOException { - List> jsonList = new ArrayList<>(); - final List> logList = mapper.readValue(httpData.toInputStream(), - LIST_OF_MAP_TYPE_REFERENCE); - List innerJsonList = new ArrayList<>(); - int size = OVERHEAD_CHARACTERS.length(); - for (Map log: logList) { - final String recordString = mapper.writeValueAsString(log); - final int nextRecordLength = recordString.getBytes(Charset.defaultCharset()).length; - // It is possible that the first record is larger than maxSize, then - // innerJsonList size would be zero. - if (size + nextRecordLength > maxSize && !innerJsonList.isEmpty()) { - jsonList.add(innerJsonList); - innerJsonList = new ArrayList<>(); - size = OVERHEAD_CHARACTERS.length(); + public void serialize(final List jsonList, + final Consumer serializedBodyConsumer, + final int splitLength) throws IOException { + if (splitLength < 0) + throw new IllegalArgumentException("The splitLength must be greater than or equal to 0."); + + if (splitLength == 0) { + performSerialization(jsonList, serializedBodyConsumer, Integer.MAX_VALUE); + } else { + performSerialization(jsonList, serializedBodyConsumer, splitLength); + } + } + + private void performSerialization(final List jsonList, + final Consumer serializedBodyConsumer, + final int splitLength) throws IOException { + + JsonArrayWriter jsonArrayWriter = new JsonArrayWriter(splitLength, serializedBodyConsumer); + + for (final String individualJsonLine : jsonList) { + if (jsonArrayWriter.willExceedByWriting(individualJsonLine)) { + jsonArrayWriter.close(); + + jsonArrayWriter = new JsonArrayWriter(splitLength, serializedBodyConsumer); + } - // The following may result in a innerJsonList with larger than "maxSize" length recordString - innerJsonList.add(recordString); - size += nextRecordLength + COMMA_OVERHEAD_LENGTH; + jsonArrayWriter.write(individualJsonLine); } - if (size > OVERHEAD_CHARACTERS.length()) { - jsonList.add(innerJsonList); + + jsonArrayWriter.close(); + } + + private static class JsonArrayWriter { + private static final JsonFactory JSON_FACTORY = new JsonFactory().setCodec(mapper); + private static final int BUFFER_SIZE = 16 * 1024; + private static final String NECESSARY_CHARACTERS_TO_WRITE = ",]"; + private final CountingOutputStream countingOutputStream; + private final ByteArrayOutputStream outputStream; + private final int splitLength; + private final Consumer serializedBodyConsumer; + private final JsonGenerator generator; + private boolean hasItem = false; + + JsonArrayWriter(final int splitLength, final Consumer serializedBodyConsumer) throws IOException { + outputStream = new ByteArrayOutputStream(Math.min(splitLength, BUFFER_SIZE)); + countingOutputStream = new CountingOutputStream(outputStream); + this.splitLength = splitLength; + this.serializedBodyConsumer = serializedBodyConsumer; + generator = JSON_FACTORY.createGenerator(countingOutputStream, JsonEncoding.UTF8); + generator.writeStartArray(); } - return jsonList; + boolean willExceedByWriting(final String individualJsonLine) { + final int lengthToWrite = individualJsonLine.getBytes(StandardCharsets.UTF_8).length; + final long lengthOfDataWritten = countingOutputStream.getCount(); + return lengthToWrite + lengthOfDataWritten + NECESSARY_CHARACTERS_TO_WRITE.length() > splitLength; + } + + void write(final String individualJsonLine) throws IOException { + final JsonNode jsonNode = mapper.readTree(individualJsonLine); + generator.writeTree(jsonNode); + generator.flush(); + hasItem = true; + } + + void close() throws IOException { + if (hasItem) { + generator.writeEndArray(); + generator.flush(); + final String resultJson = outputStream.toString(Charset.defaultCharset()); + + serializedBodyConsumer.accept(resultJson); + } + + generator.close(); + outputStream.close(); + } } + } diff --git a/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java index ec095e04e2..b58f9e6cde 100644 --- a/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java +++ b/data-prepper-plugins/http-source-common/src/test/java/org/opensearch/dataprepper/http/codec/JsonCodecTest.java @@ -12,11 +12,14 @@ 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.ValueSource; +import org.mockito.ArgumentCaptor; import java.io.IOException; import java.nio.charset.Charset; +import java.nio.charset.StandardCharsets; import java.util.List; -import java.util.stream.Collectors; +import java.util.function.Consumer; import java.util.stream.Stream; import static org.hamcrest.CoreMatchers.equalTo; @@ -24,15 +27,30 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.params.provider.Arguments.arguments; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; class JsonCodecTest { - private final HttpData goodTestData = HttpData.ofUtf8("[{\"a\":\"b\"}, {\"c\":\"d\"}]"); - private final HttpData goodLargeTestData = HttpData.ofUtf8("[{\"a1\":\"b1\"}, {\"a2\":\"b2\"}, {\"a3\":\"b3\"}, {\"a4\":\"b4\"}, {\"a5\":\"b5\"}]"); - private final HttpData goodLargeTestDataUnicode = HttpData.ofUtf8("[{\"ὊὊὊ1\":\"ὊὊὊ1\"}, {\"ὊὊὊ2\":\"ὊὊὊ2\"}, {\"a3\":\"b3\"}, {\"ὊὊὊ4\":\"ὊὊὊ4\"}]"); + private static final HttpData GOOD_TEST_DATA = HttpData.ofUtf8("[{\"a\":\"b\"}, {\"c\":\"d\"}]"); + private static final HttpData GOOD_LARGE_TEST_DATA = HttpData.ofUtf8("[{\"a1\":\"b1\"}, {\"a2\":\"b2\"}, {\"a3\":\"b3\"}, {\"a4\":\"b4\"}, {\"a5\":\"b5\"}]"); + private static final HttpData GOOD_LARGE_TEST_DATA_UNICODE = HttpData.ofUtf8("[{\"ὊὊὊ1\":\"ὊὊὊ1\"}, {\"ὊὊὊ2\":\"ὊὊὊ2\"}, {\"a3\":\"b3\"}, {\"ὊὊὊ4\":\"ὊὊὊ4\"}]"); + public static final List JSON_BODIES_LIST = List.of( + "{\"a1\":\"b1\"}", + "{\"a2\":\"b2\"}", + "{\"a3\":\"b3\"}", + "{\"a4\":\"b4\"}", + "{\"a5\":\"b5\"}" + ); + public static final List JSON_BODIES_UNICODE_MIXED_LIST = List.of( + "{\"ὊὊὊ1\":\"ὊὊὊ1\"}", + "{\"ὊὊὊ2\":\"ὊὊὊ2\"}", + "{\"a3\":\"b3\"}", + "{\"ὊὊὊ4\":\"ὊὊὊ4\"}" + ); private final HttpData badTestDataJsonLine = HttpData.ofUtf8("{\"a\":\"b\"}"); private final HttpData badTestDataMultiJsonLines = HttpData.ofUtf8("{\"a\":\"b\"}{\"c\":\"d\"}"); private final HttpData badTestDataNonJson = HttpData.ofUtf8("non json content"); @@ -40,56 +58,170 @@ class JsonCodecTest { @Test public void testParseSuccess() throws IOException { - // When - List> res = objectUnderTest.parse(goodTestData); + List res = objectUnderTest.parse(GOOD_TEST_DATA); // Then - assertEquals(1, res.size()); - assertEquals(2, res.get(0).size()); - assertEquals("{\"a\":\"b\"}", res.get(0).get(0)); + assertEquals(2, res.size()); + assertEquals("{\"a\":\"b\"}", res.get(0)); + assertEquals("{\"c\":\"d\"}", res.get(1)); } @Test public void testParseSuccessWithMaxSize() throws IOException { // When - List> res = objectUnderTest.parse(goodLargeTestData, 30); + List res = objectUnderTest.parse(GOOD_LARGE_TEST_DATA); - assertEquals(3, res.size()); + assertEquals(5, res.size()); // Then - assertEquals(2, res.get(0).size()); - assertEquals("{\"a1\":\"b1\"}", res.get(0).get(0)); - assertEquals("{\"a2\":\"b2\"}", res.get(0).get(1)); - assertEquals(2, res.get(1).size()); - assertEquals("{\"a3\":\"b3\"}", res.get(1).get(0)); - assertEquals("{\"a4\":\"b4\"}", res.get(1).get(1)); - assertEquals(1, res.get(2).size()); - assertEquals("{\"a5\":\"b5\"}", res.get(2).get(0)); + assertEquals("{\"a1\":\"b1\"}", res.get(0)); + assertEquals("{\"a2\":\"b2\"}", res.get(1)); + assertEquals("{\"a3\":\"b3\"}", res.get(2)); + assertEquals("{\"a4\":\"b4\"}", res.get(3)); + assertEquals("{\"a5\":\"b5\"}", res.get(4)); + } + + + @ParameterizedTest + @ValueSource(ints = {-1, -2, Integer.MIN_VALUE}) + void serialize_with_invalid_splitLength(final int splitLength) { + final Consumer serializedBodyConsumer = mock(Consumer.class); + assertThrows(IllegalArgumentException.class, () -> objectUnderTest.serialize(JSON_BODIES_LIST, serializedBodyConsumer, splitLength)); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 24}) + void serialize_with_split_length_leading_to_groups_of_one(final int splitLength) throws IOException { + final Consumer serializedBodyConsumer = mock(Consumer.class); + objectUnderTest.serialize(JSON_BODIES_LIST, serializedBodyConsumer, splitLength); + + final ArgumentCaptor actualSerializedBodyCaptor = ArgumentCaptor.forClass(String.class); + verify(serializedBodyConsumer, times(5)).accept(actualSerializedBodyCaptor.capture()); + + final List allActualSerializedBodies = actualSerializedBodyCaptor.getAllValues(); + assertThat(allActualSerializedBodies.size(), equalTo(5)); + assertThat(allActualSerializedBodies.get(0), equalTo("[{\"a1\":\"b1\"}]")); + assertThat(allActualSerializedBodies.get(1), equalTo("[{\"a2\":\"b2\"}]")); + assertThat(allActualSerializedBodies.get(2), equalTo("[{\"a3\":\"b3\"}]")); + assertThat(allActualSerializedBodies.get(3), equalTo("[{\"a4\":\"b4\"}]")); + assertThat(allActualSerializedBodies.get(4), equalTo("[{\"a5\":\"b5\"}]")); + } + + @ParameterizedTest + @ValueSource(ints = {25, 30, 36}) + void serialize_with_split_length_leading_to_groups_of_two(final int splitLength) throws IOException { + final Consumer serializedBodyConsumer = mock(Consumer.class); + objectUnderTest.serialize(JSON_BODIES_LIST, serializedBodyConsumer, splitLength); + + final ArgumentCaptor actualSerializedBodyCaptor = ArgumentCaptor.forClass(String.class); + verify(serializedBodyConsumer, times(3)).accept(actualSerializedBodyCaptor.capture()); + + final List allActualSerializedBodies = actualSerializedBodyCaptor.getAllValues(); + assertThat(allActualSerializedBodies.size(), equalTo(3)); + assertThat(allActualSerializedBodies.get(0), equalTo("[{\"a1\":\"b1\"},{\"a2\":\"b2\"}]")); + assertThat(allActualSerializedBodies.get(1), equalTo("[{\"a3\":\"b3\"},{\"a4\":\"b4\"}]")); + assertThat(allActualSerializedBodies.get(2), equalTo("[{\"a5\":\"b5\"}]")); + + assertThat(allActualSerializedBodies.get(0).getBytes(StandardCharsets.UTF_8).length, lessThanOrEqualTo(splitLength)); + assertThat(allActualSerializedBodies.get(1).getBytes(StandardCharsets.UTF_8).length, lessThanOrEqualTo(splitLength)); + assertThat(allActualSerializedBodies.get(2).getBytes(StandardCharsets.UTF_8).length, lessThanOrEqualTo(splitLength)); + } + + @ParameterizedTest + @ValueSource(ints = {37, 48}) + void serialize_with_split_length_leading_to_groups_up_to_three(final int splitLength) throws IOException { + final Consumer serializedBodyConsumer = mock(Consumer.class); + objectUnderTest.serialize(JSON_BODIES_LIST, serializedBodyConsumer, splitLength); + + final ArgumentCaptor actualSerializedBodyCaptor = ArgumentCaptor.forClass(String.class); + verify(serializedBodyConsumer, times(2)).accept(actualSerializedBodyCaptor.capture()); + + final List allActualSerializedBodies = actualSerializedBodyCaptor.getAllValues(); + assertThat(allActualSerializedBodies.size(), equalTo(2)); + assertThat(allActualSerializedBodies.get(0), equalTo("[{\"a1\":\"b1\"},{\"a2\":\"b2\"},{\"a3\":\"b3\"}]")); + assertThat(allActualSerializedBodies.get(1), equalTo("[{\"a4\":\"b4\"},{\"a5\":\"b5\"}]")); + + assertThat(allActualSerializedBodies.get(0).getBytes(StandardCharsets.UTF_8).length, lessThanOrEqualTo(splitLength)); + assertThat(allActualSerializedBodies.get(1).getBytes(StandardCharsets.UTF_8).length, lessThanOrEqualTo(splitLength)); + } + + @ParameterizedTest + @ValueSource(ints = {0, Integer.MAX_VALUE}) + void serialize_with_split_size_that_does_not_split(final int splitLength) throws IOException { + final Consumer serializedBodyConsumer = mock(Consumer.class); + objectUnderTest.serialize(JSON_BODIES_LIST, serializedBodyConsumer, splitLength); + + final ArgumentCaptor actualSerializedBodyCaptor = ArgumentCaptor.forClass(String.class); + verify(serializedBodyConsumer, times(1)).accept(actualSerializedBodyCaptor.capture()); + + final String actualSerializedBody = actualSerializedBodyCaptor.getValue(); + assertThat(actualSerializedBody, equalTo("[{\"a1\":\"b1\"},{\"a2\":\"b2\"},{\"a3\":\"b3\"},{\"a4\":\"b4\"},{\"a5\":\"b5\"}]")); } + @ParameterizedTest + @ValueSource(ints = {58, 68}) + void serialize_with_split_length_unicode(final int splitLength) throws IOException { + final Consumer serializedBodyConsumer = mock(Consumer.class); + objectUnderTest.serialize(JSON_BODIES_UNICODE_MIXED_LIST, serializedBodyConsumer, splitLength); + + final ArgumentCaptor actualSerializedBodyCaptor = ArgumentCaptor.forClass(String.class); + verify(serializedBodyConsumer, times(2)).accept(actualSerializedBodyCaptor.capture()); + + final List allActualSerializedBodies = actualSerializedBodyCaptor.getAllValues(); + assertThat(allActualSerializedBodies.size(), equalTo(2)); + assertThat(allActualSerializedBodies.get(0), equalTo("[{\"ὊὊὊ1\":\"ὊὊὊ1\"},{\"ὊὊὊ2\":\"ὊὊὊ2\"}]")); + assertThat(allActualSerializedBodies.get(1), equalTo("[{\"a3\":\"b3\"},{\"ὊὊὊ4\":\"ὊὊὊ4\"}]")); + + assertThat(allActualSerializedBodies.get(0).getBytes(StandardCharsets.UTF_8).length, lessThanOrEqualTo(splitLength)); + assertThat(allActualSerializedBodies.get(1).getBytes(StandardCharsets.UTF_8).length, lessThanOrEqualTo(splitLength)); + } + + @ParameterizedTest + @ArgumentsSource(GoodTestData.class) + void parse_and_serialize_symmetry(final HttpData httpData) throws IOException { + final List parsedList = objectUnderTest.parse(httpData); + + final Consumer serializedBodyConsumer = mock(Consumer.class); + objectUnderTest.serialize(parsedList, serializedBodyConsumer); + final ArgumentCaptor actualSerializedBodyCaptor = ArgumentCaptor.forClass(String.class); + verify(serializedBodyConsumer, times(1)).accept(actualSerializedBodyCaptor.capture()); + final String actualString = actualSerializedBodyCaptor.getValue(); + + final String expectedJsonString = httpData.toStringUtf8().replace(" ", ""); + assertThat(actualString, equalTo(expectedJsonString)); + } + + @ParameterizedTest @ArgumentsSource(JsonArrayWithKnownFirstArgumentsProvider.class) public void parse_should_return_lists_smaller_than_provided_length( final String inputJsonArray, final String knownFirstPart, final int maxSize, final List> expectedChunks, final List exceedsMaxSize) throws IOException { - final int knownSingleBodySize = knownFirstPart.getBytes(Charset.defaultCharset()).length; - final List> chunkedBodies = objectUnderTest.parse(HttpData.ofUtf8(inputJsonArray), - maxSize); + List individualJsonLines = objectUnderTest.parse(HttpData.ofUtf8(inputJsonArray)); + + Consumer serializedBodyConsumer = mock(Consumer.class); + objectUnderTest.serialize(individualJsonLines, serializedBodyConsumer, maxSize); + ArgumentCaptor actualSerializedBodyCaptor = ArgumentCaptor.forClass(String.class); + verify(serializedBodyConsumer, times(expectedChunks.size())).accept(actualSerializedBodyCaptor.capture()); + + List chunkedBodies = actualSerializedBodyCaptor.getAllValues(); assertThat(chunkedBodies, notNullValue()); assertThat(chunkedBodies.size(), equalTo(expectedChunks.size())); for (int i = 0; i < expectedChunks.size(); i++) { - final String reconstructed = chunkedBodies.get(i).stream().collect(Collectors.joining(",", "[", "]")); + final String reconstructed = chunkedBodies.get(i); if (exceedsMaxSize.get(i)) { assertThat(reconstructed.getBytes(Charset.defaultCharset()).length, - greaterThanOrEqualTo(maxSize)); + greaterThanOrEqualTo(maxSize)); } else { assertThat(reconstructed.getBytes(Charset.defaultCharset()).length, - lessThanOrEqualTo(maxSize)); + lessThanOrEqualTo(maxSize)); } - + + List reParsedToCompare = objectUnderTest.parse(HttpData.ofUtf8(reconstructed)); + for (int j = 0; j < expectedChunks.get(i).size(); j++) { - assertThat(chunkedBodies.get(i).get(j), equalTo(expectedChunks.get(i).get(j))); + assertThat(reParsedToCompare.get(j), equalTo(expectedChunks.get(i).get(j))); } } } @@ -109,6 +241,17 @@ public void testParseNonJsonFailure() { assertThrows(IOException.class, () -> objectUnderTest.parse(badTestDataNonJson)); } + static class GoodTestData implements ArgumentsProvider { + @Override + public Stream provideArguments(final ExtensionContext extensionContext) { + return Stream.of( + arguments(GOOD_TEST_DATA), + arguments(GOOD_LARGE_TEST_DATA), + arguments(GOOD_LARGE_TEST_DATA_UNICODE) + ); + } + } + static class JsonArrayWithKnownFirstArgumentsProvider implements ArgumentsProvider { @Override public Stream provideArguments(ExtensionContext extensionContext) throws Exception { @@ -126,7 +269,7 @@ public Stream provideArguments(ExtensionContext extensionCo String chunk31 = "{\"ὊὊὊ1\":\"ὊὊὊ01\"}"; String chunk32 = "{\"ὊὊὊ2\":\"ὊὊὊO2\"}"; String chunk33 = "{\"ὊὊὊ3\":\"ὊὊὊO3\"}"; - String chunk34 = "{\"ὊὊὊ4\":\"ὊὊὊO4\"}"; + String chunk34 = "{\"ὊὊὊ4\":\"ὊὊὊO4\"}"; // Fourth test, only first chunk larger than maxSize, output has 3 lists, with first chunk larger than maxSize and others smaller String chunk41 = "{\"aaaaaaaaaaa1\":\"aaaaaaaaaaa1\"}"; String chunk42 = "{\"aaa2\":\"aaa2\"}"; @@ -149,12 +292,12 @@ public Stream provideArguments(ExtensionContext extensionCo final int maxSize5 = chunk51.getBytes(Charset.defaultCharset()).length * 2 + 3; final int maxSize6 = chunk61.getBytes(Charset.defaultCharset()).length * 2 + 3; return Stream.of( - arguments("["+chunk11+","+chunk12+","+chunk13+","+chunk14+"]", chunk11, maxSize1, List.of(List.of(chunk11), List.of(chunk12, chunk13), List.of(chunk14)), List.of(false, false, false)), - arguments("["+chunk21+","+chunk22+","+chunk23+","+chunk24+"]", chunk21, maxSize2, List.of(List.of(chunk21, chunk22), List.of(chunk23, chunk24)), List.of(false, false)), - arguments("["+chunk31+","+chunk32+","+chunk33+","+chunk34+"]", chunk31, maxSize3, List.of(List.of(chunk31), List.of(chunk32), List.of(chunk33), List.of(chunk34)), List.of(true, true, true, true)), - arguments("["+chunk41+","+chunk42+","+chunk43+","+chunk44+"]", chunk41, maxSize4, List.of(List.of(chunk41), List.of(chunk42, chunk43), List.of(chunk44)), List.of(true, false, false)), - arguments("["+chunk51+","+chunk52+","+chunk53+","+chunk54+"]", chunk51, maxSize5, List.of(List.of(chunk51), List.of(chunk52), List.of(chunk53,chunk54)), List.of(false, true, false)), - arguments("["+chunk61+","+chunk62+","+chunk63+","+chunk64+"]", chunk61, maxSize6, List.of(List.of(chunk61,chunk62), List.of(chunk63), List.of(chunk64)), List.of(false, false, true)) + arguments("[" + chunk11 + "," + chunk12 + "," + chunk13 + "," + chunk14 + "]", chunk11, maxSize1, List.of(List.of(chunk11), List.of(chunk12, chunk13), List.of(chunk14)), List.of(false, false, false)), + arguments("[" + chunk21 + "," + chunk22 + "," + chunk23 + "," + chunk24 + "]", chunk21, maxSize2, List.of(List.of(chunk21, chunk22), List.of(chunk23, chunk24)), List.of(false, false)), + arguments("[" + chunk31 + "," + chunk32 + "," + chunk33 + "," + chunk34 + "]", chunk31, maxSize3, List.of(List.of(chunk31), List.of(chunk32), List.of(chunk33), List.of(chunk34)), List.of(true, true, true, true)), + arguments("[" + chunk41 + "," + chunk42 + "," + chunk43 + "," + chunk44 + "]", chunk41, maxSize4, List.of(List.of(chunk41), List.of(chunk42, chunk43), List.of(chunk44)), List.of(true, false, false)), + arguments("[" + chunk51 + "," + chunk52 + "," + chunk53 + "," + chunk54 + "]", chunk51, maxSize5, List.of(List.of(chunk51), List.of(chunk52), List.of(chunk53, chunk54)), List.of(false, true, false)), + arguments("[" + chunk61 + "," + chunk62 + "," + chunk63 + "," + chunk64 + "]", chunk61, maxSize6, List.of(List.of(chunk61, chunk62), List.of(chunk63), List.of(chunk64)), List.of(false, false, true)) ); } } diff --git a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java index 1eac99ed92..2163f322ef 100644 --- a/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java +++ b/data-prepper-plugins/http-source/src/main/java/org/opensearch/dataprepper/plugins/source/loghttp/LogHTTPService.java @@ -56,8 +56,8 @@ public class LogHTTPService { private final Counter requestsOverMaximumSizeCounter; private final DistributionSummary payloadSizeSummary; private final Timer requestProcessDuration; - private Integer maxRequestLength; - private Integer optimalRequestLength; + private Integer bufferMaxRequestLength; + private Integer bufferOptimalRequestLength; public LogHTTPService(final int bufferWriteTimeoutInMillis, final Buffer> buffer, @@ -65,8 +65,8 @@ public LogHTTPService(final int bufferWriteTimeoutInMillis, final PluginMetrics pluginMetrics) { this.buffer = buffer; this.bufferWriteTimeoutInMillis = bufferWriteTimeoutInMillis; - this.maxRequestLength = buffer.getMaxRequestSize().isPresent() ? buffer.getMaxRequestSize().get(): null; - this.optimalRequestLength = buffer.getOptimalRequestSize().isPresent() ? buffer.getOptimalRequestSize().get(): null; + this.bufferMaxRequestLength = buffer.getMaxRequestSize().isPresent() ? buffer.getMaxRequestSize().get(): null; + this.bufferOptimalRequestLength = buffer.getOptimalRequestSize().isPresent() ? buffer.getOptimalRequestSize().get(): null; requestsReceivedCounter = pluginMetrics.counter(REQUESTS_RECEIVED); successRequestsCounter = pluginMetrics.counter(SUCCESS_REQUESTS); requestsOverOptimalSizeCounter = pluginMetrics.counter(REQUESTS_OVER_OPTIMAL_SIZE); @@ -87,60 +87,30 @@ public HttpResponse doPost(final ServiceRequestContext serviceRequestContext, fi return requestProcessDuration.recordCallable(() -> processRequest(aggregatedHttpRequest)); } - private void sendJsonList(List jsonList) throws Exception { - StringBuilder sb = new StringBuilder(maxRequestLength); - sb.append("["); - String comma = ""; - String key = UUID.randomUUID().toString(); - for (final String json: jsonList) { - sb.append(comma); - sb.append(json); - comma = ","; - } - sb.append("]"); - if (sb.toString().getBytes().length > maxRequestLength) { - requestsOverMaximumSizeCounter.increment(); - throw new RuntimeException("Request length "+ sb.toString().getBytes().length + " exceeds maxRequestLength "+ maxRequestLength); - } else if (sb.toString().getBytes().length > optimalRequestLength) { - requestsOverOptimalSizeCounter.increment(); - } - buffer.writeBytes(sb.toString().getBytes(), key, bufferWriteTimeoutInMillis); - } - HttpResponse processRequest(final AggregatedHttpRequest aggregatedHttpRequest) throws Exception { final HttpData content = aggregatedHttpRequest.content(); - List> jsonList; - boolean isJsonListSplit = false; + final List jsonList; try { - if (buffer.isByteBuffer() && maxRequestLength != null && optimalRequestLength != null) { - jsonList = jsonCodec.parse(content, optimalRequestLength - SERIALIZATION_OVERHEAD); - isJsonListSplit = true; - } else { - jsonList = jsonCodec.parse(content); - } + jsonList = jsonCodec.parse(content); } catch (IOException e) { LOG.error("Failed to parse the request of size {} due to: {}", content.length(), e.getMessage()); throw new IOException("Bad request data format. Needs to be json array.", e.getCause()); } try { if (buffer.isByteBuffer()) { - if (isJsonListSplit && content.array().length > optimalRequestLength) { - for (final List innerJsonList: jsonList) { - sendJsonList(innerJsonList); - } + if (bufferMaxRequestLength != null && bufferOptimalRequestLength != null && content.array().length > bufferOptimalRequestLength) { + jsonCodec.serialize(jsonList, this::writeChunkedBody, bufferOptimalRequestLength - SERIALIZATION_OVERHEAD); } else { // jsonList is ignored in this path but parse() was done to make // sure that the data is in the expected json format buffer.writeBytes(content.array(), null, bufferWriteTimeoutInMillis); } } else { - for (final List innerJsonList: jsonList) { - final List> records = innerJsonList.stream() - .map(this::buildRecordLog) - .collect(Collectors.toList()); - buffer.writeAll(records, bufferWriteTimeoutInMillis); - } + final List> records = jsonList.stream() + .map(this::buildRecordLog) + .collect(Collectors.toList()); + buffer.writeAll(records, bufferWriteTimeoutInMillis); } } catch (Exception e) { LOG.error("Failed to write the request of size {} due to: {}", content.length(), e.getMessage()); @@ -150,6 +120,25 @@ HttpResponse processRequest(final AggregatedHttpRequest aggregatedHttpRequest) t return HttpResponse.of(HttpStatus.OK); } + private void writeChunkedBody(final String chunk) { + final byte[] chunkBytes = chunk.getBytes(); + + if (chunkBytes.length > bufferMaxRequestLength) { + requestsOverMaximumSizeCounter.increment(); + LOG.error("Unable to write chunked bytes of size {} as it exceeds the maximum buffer size of {}", chunkBytes.length, bufferMaxRequestLength); + return; + } else if (chunkBytes.length > bufferOptimalRequestLength) { + requestsOverOptimalSizeCounter.increment(); + } + + final String key = UUID.randomUUID().toString(); + try { + buffer.writeBytes(chunkBytes, key, bufferWriteTimeoutInMillis); + } catch (final Exception e) { + LOG.error("Failed to write chunked bytes of size {} due to: {}", chunkBytes.length, e.getMessage()); + } + } + private Record buildRecordLog(String json) { final JacksonLog log = JacksonLog.builder() From 0d121bc75e7eb3430471ad9527a84ae55cd3c31c Mon Sep 17 00:00:00 2001 From: Jayesh Parmar <89792517+jayeshjeh@users.noreply.github.com> Date: Tue, 10 Sep 2024 01:37:40 +0530 Subject: [PATCH 150/159] Updated README.md according to previous changes of docker files. (#4845) * Updated README.md for updated /examples/log-ingestion files Signed-off-by: jayeshjeh * Updated README.md for updated /examples/log-ingestion files Signed-off-by: jayeshjeh * correction Signed-off-by: jayeshjeh --------- Signed-off-by: jayeshjeh --- examples/log-ingestion/README.md | 49 ++++++++++++++++++-------------- 1 file changed, 28 insertions(+), 21 deletions(-) diff --git a/examples/log-ingestion/README.md b/examples/log-ingestion/README.md index 33e6a336e7..7576bcda2b 100644 --- a/examples/log-ingestion/README.md +++ b/examples/log-ingestion/README.md @@ -8,10 +8,10 @@ Fluent Bit → Data Prepper → OpenSearch. This log ingestion flow is shown in ## List of Components -- An OpenSearch domain running through Docker -- A FluentBit agent running through Docker -- Data Prepper, which includes a `log_pipeline.yaml` -- An Apache Log Generator in the form of a python script +- An OpenSearch domain running through Docker. +- A FluentBit agent running through Docker using `fluent-bit.conf`. +- Data Prepper, which includes a `log_pipeline.yaml` and `data-prepper-config.yaml`for data-prepper server configuration running through Docker. +- An Apache Log Generator in the form of a python script. ### FluentBit And OpenSearch Setup @@ -29,35 +29,38 @@ FluentBit is tailing to collect logs from. 4. Now that you understand a bit more about how FluentBit and OpenSearch are set up, run them with: ``` -docker-compose --project-name data-prepper up +docker compose --project-name data-prepper up ``` +This we can verify using http://127.0.0.1:5601/ -### Data Prepper Setup - -1. Pull down the latest Data Prepper Docker image. +Once we are able to access our opensearch-dashboard we can run data-prepper. -``` -docker pull opensearchproject/data-prepper:2 -``` +### Data Prepper Setup -2. Take a look at [log_pipeline.yaml](log_pipeline.yaml). This configuration will take logs sent to the [http source](../../data-prepper-plugins/http-source), +1. Take a look at [log_pipeline.yaml](log_pipeline.yaml). This configuration will take logs sent to the [http source](../../data-prepper-plugins/http-source), process them with the [Grok Processor](../../data-prepper-plugins/grok-prepper) by matching against the `COMMONAPACHELOG` pattern, and send the processed logs to a local [OpenSearch sink](../../data-prepper-plugins/opensearch) to an index named `apache_logs`. +2. And [data-prepper-config.yaml](data-prepper-config.yaml) is also mounted in [docker-compose-dataprepper.yaml](docker-compose-dataprepper.yaml) which will help us to configure our data-prepper server. -3. Run the Data Prepper docker image with the `log_pipeline.yaml` from step 2 passed in. This command attaches the Data Prepper Docker image to the Docker network `log-ingestion_opensearch_net` so that -FluentBit is able to send logs to the http source of Data Prepper. + +3. Run the Data Prepper docker compose file where we are using `log_pipeline.yaml`. Now FluentBit is able to send logs to the http source of Data Prepper. Run the following to start Data Prepper: ``` -docker run --name data-prepper -v ${PWD}/log_pipeline.yaml:/usr/share/data-prepper/pipelines/log_pipeline.yaml --network "data-prepper_opensearch-net" opensearchproject/data-prepper:2 +docker compose -f docker-compose-dataprepper.yaml up ``` If Data Prepper is running correctly, you should see something similar to the following line as the latest output in your terminal. ``` -INFO org.opensearch.dataprepper.pipeline.server.DataPrepperServer - Data Prepper server running at :4900 +INFO org.opensearch.dataprepper.plugins.sink.opensearch.OpenSearchSink - Initialized OpenSearch sink +INFO org.opensearch.dataprepper.pipeline.Pipeline - Pipeline [log-pipeline] Sink is ready, starting source... + + +INFO org.opensearch.dataprepper.plugins.source.loghttp.HTTPSource - Started http source on port 2021... +INFO org.opensearch.dataprepper.pipeline.Pipeline - Pipeline [log-pipeline] - Submitting request to initiate the pipeline processing ``` ### Apache Log Generator @@ -88,7 +91,7 @@ Additionally, if you just want to test a single log, you can send it to `test.lo echo '63.173.168.120 - - [04/Nov/2021:15:07:25 -0500] "GET /search/tag/list HTTP/1.0" 200 5003' >> test.log ``` -In order to simulate an application generating logs, a simple python script will be used. This script only runs with python 2. You can download this script by running +In order to simulate an application generating logs, a simple python script will be used. This script only runs with python 2. You can download this script by running. ``` git clone https://github.com/graytaylor0/Fake-Apache-Log-Generator.git @@ -109,11 +112,15 @@ python apache-fake-log-gen.py -n 0 -s 2 -l "CLF" -o "LOG" -f "/full/path/to/test You should now be able to check your terminal output for FluentBit and Data Prepper to verify that they are processing logs. -The following FluentBit ouptut means that FluentBit was able to forward logs to the Data Prepper http source +The following FluentBit ouptut means that FluentBit was able to forward logs to the Data Prepper http source. ``` -fluent-bit | [2021/10/30 17:16:39] [ info] [output:http:http.0] host.docker.internal:2021, HTTP status=200 +fluent-bit | [ info] [output:http:http.0] data-prepper:2021, HTTP status=200 +200 OK ``` -Finally, head into OpenSearch Dashboards ([http://localhost:5601](http://localhost:5601)) to view your processed logs. -You will need to create an index pattern for the index provided in your `pipeline.yaml` in order to see them. You can do this by selecting the `Manage` menu with the gear icon at the top of the home page and then the `Index Patterns` menu on the left side of the page. Select the `Create index pattern` button and then start typing in the name of the index you sent logs to in the `Index pattern name` field (in this guide it was `apache_logs`). You should see that the index pattern matches 1 source. Click `Next Step` and then `Create index pattern`. After, you should be able to go to the `Discover` page with a link on the menu to the left, and see your processed logs. \ No newline at end of file +Finally, head into OpenSearch Dashboards ([http://localhost:5601](http://localhost:5601)) (login with credentials) to view your processed logs. +You will need to create an index pattern for the index provided in your `pipeline.yaml` (i.e. `apache_logs`) in order to see them. You can do this by selecting the `Manage` menu with the gear icon at the top of the home page and then the `Index Patterns` menu on the left side of the page. Select the `Create index pattern` button and then start typing in the name of the index you sent logs to in the `Index pattern name` field (in this guide it was `apache_logs`). You should see that the index pattern matches 1 source (This will only be seen if data-prepper is working well with the opensource). + +Click `Next Step` and then `Create index pattern`. After, you should be able to go to the `Discover` page with a link on the menu to the left, and see your processed logs. + From af9cab8c0f6553c180d92486f56c463114b2bb23 Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Tue, 10 Sep 2024 11:35:51 -0400 Subject: [PATCH 151/159] ENH: add shutdown into extension plugin (#4924) * ENH: add shutdown into extension plugin Signed-off-by: George Chen --- .../model/plugin/ExtensionPlugin.java | 6 ++++ .../model/plugin/ExtensionPluginTest.java | 20 +++++++++++ .../dataprepper/plugin/ExtensionsApplier.java | 12 +++++-- .../plugin/ExtensionsApplierTest.java | 10 ++++++ .../plugins/aws/AwsSecretPlugin.java | 4 +-- .../plugins/aws/AwsSecretPluginIT.java | 35 ++++--------------- 6 files changed, 54 insertions(+), 33 deletions(-) create mode 100644 data-prepper-api/src/test/java/org/opensearch/dataprepper/model/plugin/ExtensionPluginTest.java diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/plugin/ExtensionPlugin.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/plugin/ExtensionPlugin.java index 9edf55c454..51339ca9c1 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/plugin/ExtensionPlugin.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/plugin/ExtensionPlugin.java @@ -20,4 +20,10 @@ public interface ExtensionPlugin { * @param extensionPoints The {@link ExtensionPoints} wherein the extension can extend behaviors. */ void apply(ExtensionPoints extensionPoints); + + /** + * Close resources used by the extension. + */ + default void shutdown() { + }; } diff --git a/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/plugin/ExtensionPluginTest.java b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/plugin/ExtensionPluginTest.java new file mode 100644 index 0000000000..10c3b0102e --- /dev/null +++ b/data-prepper-api/src/test/java/org/opensearch/dataprepper/model/plugin/ExtensionPluginTest.java @@ -0,0 +1,20 @@ +package org.opensearch.dataprepper.model.plugin; + +import org.junit.jupiter.api.Test; + +public class ExtensionPluginTest { + + @Test + void testShutdown() { + final ExtensionPlugin extensionPlugin = new ExtensionPluginTestImpl(); + extensionPlugin.shutdown(); + } + + static class ExtensionPluginTestImpl implements ExtensionPlugin { + + @Override + public void apply(ExtensionPoints extensionPoints) { + + } + } +} diff --git a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionsApplier.java b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionsApplier.java index b2bd9ffa6f..a55445003f 100644 --- a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionsApplier.java +++ b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/ExtensionsApplier.java @@ -8,14 +8,17 @@ import org.opensearch.dataprepper.model.plugin.ExtensionPlugin; import javax.annotation.PostConstruct; +import javax.annotation.PreDestroy; import javax.inject.Inject; import javax.inject.Named; +import java.util.Collections; import java.util.List; @Named("extensionsApplier") class ExtensionsApplier { private final DataPrepperExtensionPoints dataPrepperExtensionPoints; private final ExtensionLoader extensionLoader; + private List loadedExtensionPlugins = Collections.emptyList(); @Inject ExtensionsApplier( @@ -27,10 +30,15 @@ class ExtensionsApplier { @PostConstruct void applyExtensions() { - final List extensionPlugins = extensionLoader.loadExtensions(); + loadedExtensionPlugins = extensionLoader.loadExtensions(); - for (ExtensionPlugin extensionPlugin : extensionPlugins) { + for (ExtensionPlugin extensionPlugin : loadedExtensionPlugins) { extensionPlugin.apply(dataPrepperExtensionPoints); } } + + @PreDestroy + public void shutdownExtensions() { + loadedExtensionPlugins.forEach(ExtensionPlugin::shutdown); + } } diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionsApplierTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionsApplierTest.java index 06c9ff2809..d3eb9a39cd 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionsApplierTest.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/ExtensionsApplierTest.java @@ -55,4 +55,14 @@ void applyExtensions_with_empty_extensions_is_ok() { createObjectUnderTest().applyExtensions(); } + + @Test + void shutDownExtensions_invokes_extension_plugin_shutdown() { + final ExtensionPlugin extensionPlugin = mock(ExtensionPlugin.class); + when(extensionLoader.loadExtensions()).thenReturn((List) List.of(extensionPlugin)); + final ExtensionsApplier objectUnderTest = createObjectUnderTest(); + objectUnderTest.applyExtensions(); + objectUnderTest.shutdownExtensions(); + verify(extensionPlugin).shutdown(); + } } \ No newline at end of file diff --git a/data-prepper-plugins/aws-plugin/src/main/java/org/opensearch/dataprepper/plugins/aws/AwsSecretPlugin.java b/data-prepper-plugins/aws-plugin/src/main/java/org/opensearch/dataprepper/plugins/aws/AwsSecretPlugin.java index 552106adb9..288f6d6e2b 100644 --- a/data-prepper-plugins/aws-plugin/src/main/java/org/opensearch/dataprepper/plugins/aws/AwsSecretPlugin.java +++ b/data-prepper-plugins/aws-plugin/src/main/java/org/opensearch/dataprepper/plugins/aws/AwsSecretPlugin.java @@ -43,7 +43,6 @@ public AwsSecretPlugin(final AwsSecretPluginConfig awsSecretPluginConfig) { scheduledExecutorService = Executors.newSingleThreadScheduledExecutor(); pluginMetrics = PluginMetrics.fromNames("secrets", "aws"); submitSecretsRefreshJobs(awsSecretPluginConfig, secretsSupplier); - Runtime.getRuntime().addShutdownHook(new Thread(this::shutdown)); } else { pluginConfigValueTranslator = null; } @@ -70,7 +69,8 @@ private void submitSecretsRefreshJobs(final AwsSecretPluginConfig awsSecretPlugi }); } - void shutdown() { + @Override + public void shutdown() { if (scheduledExecutorService != null) { LOG.info("Shutting down secrets refreshing tasks."); scheduledExecutorService.shutdown(); diff --git a/data-prepper-plugins/aws-plugin/src/test/java/org/opensearch/dataprepper/plugins/aws/AwsSecretPluginIT.java b/data-prepper-plugins/aws-plugin/src/test/java/org/opensearch/dataprepper/plugins/aws/AwsSecretPluginIT.java index 2f624611a9..a9f434d3e8 100644 --- a/data-prepper-plugins/aws-plugin/src/test/java/org/opensearch/dataprepper/plugins/aws/AwsSecretPluginIT.java +++ b/data-prepper-plugins/aws-plugin/src/test/java/org/opensearch/dataprepper/plugins/aws/AwsSecretPluginIT.java @@ -71,9 +71,6 @@ class AwsSecretPluginIT { @Mock private ScheduledExecutorService scheduledExecutorService; - @Mock - private Runtime runtime; - @Captor private ArgumentCaptor initialDelayCaptor; @@ -92,12 +89,9 @@ void testInitializationWithNonNullConfig() { when(awsSecretManagerConfiguration.createGetSecretValueRequest()).thenReturn(getSecretValueRequest); when(secretsManagerClient.getSecretValue(eq(getSecretValueRequest))).thenReturn(getSecretValueResponse); when(getSecretValueResponse.secretString()).thenReturn(UUID.randomUUID().toString()); - try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class); - final MockedStatic runtimeMockedStatic = mockStatic(Runtime.class) - ) { + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(Executors::newSingleThreadScheduledExecutor) .thenReturn(scheduledExecutorService); - runtimeMockedStatic.when(Runtime::getRuntime).thenReturn(runtime); objectUnderTest = new AwsSecretPlugin(awsSecretPluginConfig); objectUnderTest.apply(extensionPoints); } @@ -118,7 +112,6 @@ void testInitializationWithNonNullConfig() { any(), initialDelayCaptor.capture(), periodCaptor.capture(), eq(TimeUnit.SECONDS)); assertThat(initialDelayCaptor.getValue() >= testInterval.toSeconds(), is(true)); assertThat(periodCaptor.getValue(), equalTo(testInterval.toSeconds())); - verify(runtime).addShutdownHook(any()); } @Test @@ -130,12 +123,9 @@ void testInitializationWithDisableRefresh() { when(awsSecretManagerConfiguration.createGetSecretValueRequest()).thenReturn(getSecretValueRequest); when(secretsManagerClient.getSecretValue(eq(getSecretValueRequest))).thenReturn(getSecretValueResponse); when(getSecretValueResponse.secretString()).thenReturn(UUID.randomUUID().toString()); - try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class); - final MockedStatic runtimeMockedStatic = mockStatic(Runtime.class) - ) { + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(Executors::newSingleThreadScheduledExecutor) .thenReturn(scheduledExecutorService); - runtimeMockedStatic.when(Runtime::getRuntime).thenReturn(runtime); objectUnderTest = new AwsSecretPlugin(awsSecretPluginConfig); objectUnderTest.apply(extensionPoints); } @@ -153,7 +143,6 @@ void testInitializationWithDisableRefresh() { actualExtensionProviders.get(1).provideInstance(context); assertThat(optionalPluginConfigPublisher.isPresent(), is(true)); verifyNoInteractions(scheduledExecutorService); - verify(runtime).addShutdownHook(any()); } @Test @@ -178,12 +167,9 @@ void testInitializationWithNullConfig() { void testShutdownAwaitTerminationSuccess() throws InterruptedException { when(awsSecretPluginConfig.getAwsSecretManagerConfigurationMap()).thenReturn( Collections.emptyMap()); - try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class); - final MockedStatic runtimeMockedStatic = mockStatic(Runtime.class) - ) { + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(Executors::newSingleThreadScheduledExecutor) .thenReturn(scheduledExecutorService); - runtimeMockedStatic.when(Runtime::getRuntime).thenReturn(runtime); objectUnderTest = new AwsSecretPlugin(awsSecretPluginConfig); } when(scheduledExecutorService.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(true); @@ -199,12 +185,9 @@ void testShutdownAwaitTerminationSuccess() throws InterruptedException { void testShutdownAwaitTerminationTimeout() throws InterruptedException { when(awsSecretPluginConfig.getAwsSecretManagerConfigurationMap()).thenReturn( Collections.emptyMap()); - try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class); - final MockedStatic runtimeMockedStatic = mockStatic(Runtime.class) - ) { + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(Executors::newSingleThreadScheduledExecutor) .thenReturn(scheduledExecutorService); - runtimeMockedStatic.when(Runtime::getRuntime).thenReturn(runtime); objectUnderTest = new AwsSecretPlugin(awsSecretPluginConfig); } when(scheduledExecutorService.awaitTermination(anyLong(), any(TimeUnit.class))).thenReturn(false); @@ -220,12 +203,9 @@ void testShutdownAwaitTerminationTimeout() throws InterruptedException { void testShutdownAwaitTerminationInterrupted() throws InterruptedException { when(awsSecretPluginConfig.getAwsSecretManagerConfigurationMap()).thenReturn( Collections.emptyMap()); - try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class); - final MockedStatic runtimeMockedStatic = mockStatic(Runtime.class) - ) { + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(Executors::newSingleThreadScheduledExecutor) .thenReturn(scheduledExecutorService); - runtimeMockedStatic.when(Runtime::getRuntime).thenReturn(runtime); objectUnderTest = new AwsSecretPlugin(awsSecretPluginConfig); } when(scheduledExecutorService.awaitTermination(anyLong(), any(TimeUnit.class))) @@ -240,12 +220,9 @@ void testShutdownAwaitTerminationInterrupted() throws InterruptedException { @Test void testShutdownWithNullScheduledExecutorService() { - try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class); - final MockedStatic runtimeMockedStatic = mockStatic(Runtime.class) - ) { + try (final MockedStatic executorsMockedStatic = mockStatic(Executors.class)) { executorsMockedStatic.when(Executors::newSingleThreadScheduledExecutor) .thenReturn(scheduledExecutorService); - runtimeMockedStatic.when(Runtime::getRuntime).thenReturn(runtime); objectUnderTest = new AwsSecretPlugin(null); } objectUnderTest.shutdown(); From 7e9866ad8779109c8742cd1cd94704709cce6585 Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Tue, 10 Sep 2024 11:14:00 -0500 Subject: [PATCH 152/159] Add progress check callback to update partition ownership in S3 scan source (#4918) Signed-off-by: Taylor Gray --- .../source/coordinator/SourceCoordinator.java | 7 ++ .../LeaseBasedSourceCoordinator.java | 30 ++++++-- .../LeaseBasedSourceCoordinatorTest.java | 23 ++++++ .../plugins/source/s3/S3ObjectWorker.java | 2 +- .../plugins/source/s3/ScanObjectWorker.java | 23 +++++- .../source/s3/S3ScanObjectWorkerTest.java | 73 +++++++++++++++++++ 6 files changed, 149 insertions(+), 9 deletions(-) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/source/coordinator/SourceCoordinator.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/source/coordinator/SourceCoordinator.java index 71c0232669..ef00db1cb8 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/source/coordinator/SourceCoordinator.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/source/coordinator/SourceCoordinator.java @@ -139,5 +139,12 @@ public interface SourceCoordinator { */ void updatePartitionForAcknowledgmentWait(final String partitionKey, final Duration ackowledgmentTimeout); + /** + * Should be called by the source to keep ownership of the partition + * before another instance of Data Prepper can pick it up for processing. + * @param partitionKey - the partition to renew ownership for + */ + void renewPartitionOwnership(final String partitionKey); + void deletePartition(final String partitionKey); } diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/sourcecoordination/LeaseBasedSourceCoordinator.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/sourcecoordination/LeaseBasedSourceCoordinator.java index af0d8578f9..b66edb496a 100644 --- a/data-prepper-core/src/main/java/org/opensearch/dataprepper/sourcecoordination/LeaseBasedSourceCoordinator.java +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/sourcecoordination/LeaseBasedSourceCoordinator.java @@ -59,7 +59,6 @@ public class LeaseBasedSourceCoordinator implements SourceCoordinator { static final String PARTITION_NOT_FOUND_ERROR_COUNT = "partitionNotFoundErrors"; static final String PARTITION_NOT_OWNED_ERROR_COUNT = "partitionNotOwnedErrors"; static final String PARTITION_UPDATE_ERROR_COUNT = "PartitionUpdateErrors"; - static final Duration DEFAULT_LEASE_TIMEOUT = Duration.ofMinutes(10); private static final String hostName; @@ -91,7 +90,6 @@ public class LeaseBasedSourceCoordinator implements SourceCoordinator { private final Counter saveStatePartitionUpdateErrorCounter; private final Counter closePartitionUpdateErrorCounter; private final Counter completePartitionUpdateErrorCounter; - private final Counter partitionsDeleted; private final ReentrantLock lock; @@ -302,7 +300,8 @@ public void saveProgressStateForPartition(final String partitionKe try { sourceCoordinationStore.tryUpdateSourcePartitionItem(itemToUpdate); - } catch (final PartitionUpdateException e) { + } catch (final Exception e) { + LOG.error("Exception while saving state for the partition {}: {}", partitionKey, e.getMessage()); saveStatePartitionUpdateErrorCounter.increment(); throw e; } @@ -315,12 +314,29 @@ public void saveProgressStateForPartition(final String partitionKe @Override public void updatePartitionForAcknowledgmentWait(final String partitionKey, final Duration ackowledgmentTimeout) { - validateIsInitialized(); + try { + updatePartitionOwnership(partitionKey, ackowledgmentTimeout); + } catch (final Exception e) { + LOG.error("Exception while updating acknowledgment wait for the partition {}: {}", partitionKey, e.getMessage()); + throw e; + } + } + + @Override + public void renewPartitionOwnership(final String partitionKey) { + try { + updatePartitionOwnership(partitionKey, DEFAULT_LEASE_TIMEOUT); + } catch (final Exception e) { + LOG.error("Exception while renewing partition ownership for the partition {}: {}", partitionKey, e.getMessage()); + throw e; + } + } - final SourcePartitionStoreItem itemToUpdate = getSourcePartitionStoreItem(partitionKey, "update for ack wait"); + private void updatePartitionOwnership(final String partitionKey, final Duration ownershipRenewalTime) { + final SourcePartitionStoreItem itemToUpdate = getSourcePartitionStoreItem(partitionKey, "update partition ownership"); validatePartitionOwnership(itemToUpdate); - itemToUpdate.setPartitionOwnershipTimeout(Instant.now().plus(ackowledgmentTimeout)); + itemToUpdate.setPartitionOwnershipTimeout(Instant.now().plus(ownershipRenewalTime)); sourceCoordinationStore.tryUpdateSourcePartitionItem(itemToUpdate); } @@ -371,7 +387,7 @@ public void deletePartition(final String partitionKey) { try { sourceCoordinationStore.tryDeletePartitionItem(deleteItem); } catch (final PartitionUpdateException e) { - LOG.info("Unable to delete partition {}: {}.", deleteItem.getSourcePartitionKey(), e.getMessage()); + LOG.error("Unable to delete partition {}: {}.", deleteItem.getSourcePartitionKey(), e.getMessage()); return; } diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/sourcecoordination/LeaseBasedSourceCoordinatorTest.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/sourcecoordination/LeaseBasedSourceCoordinatorTest.java index 2e95671f78..965a0c8786 100644 --- a/data-prepper-core/src/test/java/org/opensearch/dataprepper/sourcecoordination/LeaseBasedSourceCoordinatorTest.java +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/sourcecoordination/LeaseBasedSourceCoordinatorTest.java @@ -796,6 +796,29 @@ void updatePartitionForAckWait_updates_partition_ownership_and_removes_active_pa assertThat(newPartitionOwnershipTimeout.isAfter(beforeSave.plus(ackTimeout)), equalTo(true)); } + @Test + void renewPartitionOwnership_updates_partition_ownership() throws UnknownHostException { + final SourcePartition sourcePartition = SourcePartition.builder(String.class) + .withPartitionKey(UUID.randomUUID().toString()) + .withPartitionState(null) + .build(); + + final Instant beforeSave = Instant.now(); + + given(sourcePartitionStoreItem.getPartitionOwner()).willReturn(sourceIdentifierWithPartitionPrefix + ":" + InetAddress.getLocalHost().getHostName()); + given(sourceCoordinationStore.getSourcePartitionItem(fullSourceIdentifierForPartition, sourcePartition.getPartitionKey())).willReturn(Optional.of(sourcePartitionStoreItem)); + + doNothing().when(sourceCoordinationStore).tryUpdateSourcePartitionItem(sourcePartitionStoreItem); + + final Duration ackTimeout = Duration.ofSeconds(10); + createObjectUnderTest().renewPartitionOwnership(sourcePartition.getPartitionKey()); + + final ArgumentCaptor argumentCaptorForPartitionOwnershipTimeout = ArgumentCaptor.forClass(Instant.class); + verify(sourcePartitionStoreItem).setPartitionOwnershipTimeout(argumentCaptorForPartitionOwnershipTimeout.capture()); + final Instant newPartitionOwnershipTimeout = argumentCaptorForPartitionOwnershipTimeout.getValue(); + assertThat(newPartitionOwnershipTimeout.isAfter(beforeSave.plus(ackTimeout)), equalTo(true)); + } + @Test void giveUpPartitions_with_active_partitionKey_that_does_not_exist_in_the_store_removes_the_active_partition() { final SourcePartition sourcePartition = SourcePartition.builder(String.class) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ObjectWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ObjectWorker.java index 0397183877..6750ada0f3 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ObjectWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ObjectWorker.java @@ -108,7 +108,7 @@ private void doParseObject(final AcknowledgementSet acknowledgementSet, } bufferAccumulator.add(record); - if (sourceCoordinator != null && partitionKey != null && + if (acknowledgementSet != null && sourceCoordinator != null && partitionKey != null && (System.currentTimeMillis() - lastCheckpointTime.get() > DEFAULT_CHECKPOINT_INTERVAL_MILLS)) { LOG.debug("Renew partition ownership for the object {}", partitionKey); sourceCoordinator.saveProgressStateForPartition(partitionKey, null); diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java index 471a0efa3d..00172701cc 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/ScanObjectWorker.java @@ -52,6 +52,8 @@ public class ScanObjectWorker implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(ScanObjectWorker.class); private static final Integer MAX_OBJECTS_PER_ACKNOWLEDGMENT_SET = 1; + static final Duration CHECKPOINT_OWNERSHIP_INTERVAL = Duration.ofMinutes(2); + static final Duration NO_OBJECTS_FOUND_BEFORE_PARTITION_DELETION_DURATION = Duration.ofHours(1); private static final int RETRY_BACKOFF_ON_EXCEPTION_MILLIS = 5_000; @@ -60,6 +62,7 @@ public class ScanObjectWorker implements Runnable { static final String NO_OBJECTS_FOUND_FOR_FOLDER_PARTITION = "folderPartitionNoObjectsFound"; + static final String PARTITION_OWNERSHIP_UPDATE_ERRORS = "partitionOwnershipUpdateErrors"; private final S3Client s3Client; private final List scanOptionsBuilderList; @@ -85,6 +88,8 @@ public class ScanObjectWorker implements Runnable { private final Counter acknowledgementSetCallbackCounter; private final Counter folderPartitionNoObjectsFound; + + private final Counter partitionOwnershipUpdateFailures; private final long backOffMs; private final List partitionKeys; @@ -118,6 +123,7 @@ public ScanObjectWorker(final S3Client s3Client, this.pluginMetrics = pluginMetrics; acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME); this.folderPartitionNoObjectsFound = pluginMetrics.counter(NO_OBJECTS_FOUND_FOR_FOLDER_PARTITION); + this.partitionOwnershipUpdateFailures = pluginMetrics.counter(PARTITION_OWNERSHIP_UPDATE_ERRORS); this.sourceCoordinator.initialize(); this.partitionKeys = new ArrayList<>(); this.folderPartitioningOptions = s3SourceConfig.getS3ScanScanOptions().getPartitioningOptions(); @@ -209,6 +215,17 @@ private void startProcessingObject(final long waitTimeMillis) { } partitionKeys.remove(objectToProcess.get().getPartitionKey()); }, ACKNOWLEDGEMENT_SET_TIMEOUT); + + acknowledgementSet.addProgressCheck( + (ratio) -> { + try { + sourceCoordinator.renewPartitionOwnership(objectToProcess.get().getPartitionKey()); + } catch (final PartitionUpdateException | PartitionNotOwnedException | PartitionNotFoundException e) { + LOG.debug("Failed to update partition ownership for {} in the acknowledgment progress check", objectToProcess.get().getPartitionKey()); + partitionOwnershipUpdateFailures.increment(); + } + }, + CHECKPOINT_OWNERSHIP_INTERVAL); } @@ -217,7 +234,11 @@ private void startProcessingObject(final long waitTimeMillis) { if (endToEndAcknowledgementsEnabled) { deleteObjectRequest.ifPresent(deleteRequest -> objectsToDeleteForAcknowledgmentSets.put(objectToProcess.get().getPartitionKey(), Set.of(deleteRequest))); - sourceCoordinator.updatePartitionForAcknowledgmentWait(objectToProcess.get().getPartitionKey(), ACKNOWLEDGEMENT_SET_TIMEOUT); + try { + sourceCoordinator.updatePartitionForAcknowledgmentWait(objectToProcess.get().getPartitionKey(), ACKNOWLEDGEMENT_SET_TIMEOUT); + } catch (final PartitionUpdateException e) { + LOG.debug("Failed to update the partition for the acknowledgment wait."); + } acknowledgementSet.complete(); } else { sourceCoordinator.completePartition(objectToProcess.get().getPartitionKey(), false); diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanObjectWorkerTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanObjectWorkerTest.java index fa1645db8d..3e71510bf4 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanObjectWorkerTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanObjectWorkerTest.java @@ -18,6 +18,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.acknowledgements.ProgressCheck; import org.opensearch.dataprepper.model.source.coordinator.SourceCoordinator; import org.opensearch.dataprepper.model.source.coordinator.SourcePartition; import org.opensearch.dataprepper.model.source.coordinator.exceptions.PartitionNotFoundException; @@ -69,8 +70,10 @@ import static org.opensearch.dataprepper.model.source.s3.S3ScanEnvironmentVariables.STOP_S3_SCAN_PROCESSING_PROPERTY; import static org.opensearch.dataprepper.plugins.source.s3.ScanObjectWorker.ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME; import static org.opensearch.dataprepper.plugins.source.s3.ScanObjectWorker.ACKNOWLEDGEMENT_SET_TIMEOUT; +import static org.opensearch.dataprepper.plugins.source.s3.ScanObjectWorker.CHECKPOINT_OWNERSHIP_INTERVAL; import static org.opensearch.dataprepper.plugins.source.s3.ScanObjectWorker.NO_OBJECTS_FOUND_BEFORE_PARTITION_DELETION_DURATION; import static org.opensearch.dataprepper.plugins.source.s3.ScanObjectWorker.NO_OBJECTS_FOUND_FOR_FOLDER_PARTITION; +import static org.opensearch.dataprepper.plugins.source.s3.ScanObjectWorker.PARTITION_OWNERSHIP_UPDATE_ERRORS; @ExtendWith(MockitoExtension.class) class S3ScanObjectWorkerTest { @@ -114,6 +117,9 @@ class S3ScanObjectWorkerTest { @Mock private Counter counter; + @Mock + private Counter partitionOwnershipUpdateErrorCounter; + @Mock private Counter noObjectsFoundForFolderPartitionCounter; @@ -130,6 +136,7 @@ private ScanObjectWorker createObjectUnderTest() { when(s3SourceConfig.getS3ScanScanOptions()).thenReturn(s3ScanScanOptions); when(pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME)).thenReturn(counter); when(pluginMetrics.counter(NO_OBJECTS_FOUND_FOR_FOLDER_PARTITION)).thenReturn(noObjectsFoundForFolderPartitionCounter); + when(pluginMetrics.counter(PARTITION_OWNERSHIP_UPDATE_ERRORS)).thenReturn(partitionOwnershipUpdateErrorCounter); final ScanObjectWorker objectUnderTest = new ScanObjectWorker(s3Client, scanOptionsList, s3ObjectHandler, bucketOwnerProvider, sourceCoordinator, s3SourceConfig, acknowledgementSetManager, s3ObjectDeleteWorker, 30000, pluginMetrics); verify(sourceCoordinator).initialize(); @@ -207,12 +214,76 @@ void buildDeleteObjectRequest_should_be_invoked_after_processing_when_deleteS3Ob final ScanObjectWorker scanObjectWorker = createObjectUnderTest(); when(acknowledgementSetManager.create(any(Consumer.class), any(Duration.class))).thenReturn(acknowledgementSet); + doNothing().when(acknowledgementSet).addProgressCheck(any(Consumer.class), any(Duration.class)); scanObjectWorker.runWithoutInfiniteLoop(); final ArgumentCaptor consumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); verify(acknowledgementSetManager).create(consumerArgumentCaptor.capture(), any(Duration.class)); + final ArgumentCaptor progressCheckArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSet).addProgressCheck(progressCheckArgumentCaptor.capture(), eq(CHECKPOINT_OWNERSHIP_INTERVAL)); + + final Consumer progressCheckConsumer = progressCheckArgumentCaptor.getValue(); + progressCheckConsumer.accept(mock(ProgressCheck.class)); + + final Consumer ackCallback = consumerArgumentCaptor.getValue(); + ackCallback.accept(true); + + final InOrder inOrder = inOrder(sourceCoordinator, acknowledgementSet, s3ObjectDeleteWorker); + inOrder.verify(s3ObjectDeleteWorker).buildDeleteObjectRequest(bucket, objectKey); + inOrder.verify(sourceCoordinator).updatePartitionForAcknowledgmentWait(partitionKey, ACKNOWLEDGEMENT_SET_TIMEOUT); + inOrder.verify(acknowledgementSet).complete(); + inOrder.verify(sourceCoordinator).renewPartitionOwnership(partitionKey); + inOrder.verify(sourceCoordinator).completePartition(partitionKey, true); + + verify(counter).increment(); + + final S3ObjectReference processedObject = objectReferenceArgumentCaptor.getValue(); + assertThat(processedObject.getBucketName(), equalTo(bucket)); + assertThat(processedObject.getKey(), equalTo(objectKey)); + } + + @ParameterizedTest + @MethodSource("exceptionProvider") + void acknowledgment_progress_check_increments_ownership_error_metric_when_partition_fails_to_update(final Class exception) throws IOException { + final String bucket = UUID.randomUUID().toString(); + final String objectKey = UUID.randomUUID().toString(); + final String partitionKey = bucket + "|" + objectKey; + + + when(s3SourceConfig.getAcknowledgements()).thenReturn(true); + when(s3SourceConfig.isDeleteS3ObjectsOnRead()).thenReturn(true); + when(s3ObjectDeleteWorker.buildDeleteObjectRequest(bucket, objectKey)).thenReturn(deleteObjectRequest); + + final SourcePartition partitionToProcess = SourcePartition.builder(S3SourceProgressState.class) + .withPartitionKey(partitionKey) + .withPartitionClosedCount(0L) + .build(); + + given(sourceCoordinator.getNextPartition(any(Function.class), eq(false))).willReturn(Optional.of(partitionToProcess)); + + final ArgumentCaptor objectReferenceArgumentCaptor = ArgumentCaptor.forClass(S3ObjectReference.class); + doNothing().when(s3ObjectHandler).parseS3Object(objectReferenceArgumentCaptor.capture(), eq(acknowledgementSet), eq(sourceCoordinator), eq(partitionKey)); + doNothing().when(sourceCoordinator).completePartition(anyString(), eq(true)); + + final ScanObjectWorker scanObjectWorker = createObjectUnderTest(); + + when(acknowledgementSetManager.create(any(Consumer.class), any(Duration.class))).thenReturn(acknowledgementSet); + doNothing().when(acknowledgementSet).addProgressCheck(any(Consumer.class), any(Duration.class)); + + scanObjectWorker.runWithoutInfiniteLoop(); + + final ArgumentCaptor consumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSetManager).create(consumerArgumentCaptor.capture(), any(Duration.class)); + + final ArgumentCaptor progressCheckArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSet).addProgressCheck(progressCheckArgumentCaptor.capture(), eq(CHECKPOINT_OWNERSHIP_INTERVAL)); + + final Consumer progressCheckConsumer = progressCheckArgumentCaptor.getValue(); + doThrow(exception).when(sourceCoordinator).renewPartitionOwnership(partitionKey); + progressCheckConsumer.accept(mock(ProgressCheck.class)); + final Consumer ackCallback = consumerArgumentCaptor.getValue(); ackCallback.accept(true); @@ -220,9 +291,11 @@ void buildDeleteObjectRequest_should_be_invoked_after_processing_when_deleteS3Ob inOrder.verify(s3ObjectDeleteWorker).buildDeleteObjectRequest(bucket, objectKey); inOrder.verify(sourceCoordinator).updatePartitionForAcknowledgmentWait(partitionKey, ACKNOWLEDGEMENT_SET_TIMEOUT); inOrder.verify(acknowledgementSet).complete(); + inOrder.verify(sourceCoordinator).renewPartitionOwnership(partitionKey); inOrder.verify(sourceCoordinator).completePartition(partitionKey, true); verify(counter).increment(); + verify(partitionOwnershipUpdateErrorCounter).increment(); final S3ObjectReference processedObject = objectReferenceArgumentCaptor.getValue(); assertThat(processedObject.getBucketName(), equalTo(bucket)); From 76d96404bd3b2f81fb4d587741fd434939d73e21 Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Tue, 10 Sep 2024 13:53:16 -0500 Subject: [PATCH 153/159] Support start_time or range options for the first scan of scheduled s3 scan (#4929) Signed-off-by: Taylor Gray --- .../s3/S3ScanPartitionCreationSupplier.java | 8 ++- .../s3/configuration/S3ScanScanOptions.java | 9 ++- .../S3ScanPartitionCreationSupplierTest.java | 63 +++++++++++++++++++ .../configuration/S3ScanScanOptionsTest.java | 56 +++++++++++++++++ 4 files changed, 131 insertions(+), 5 deletions(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplier.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplier.java index 66a0df271c..1d4bf6ea81 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplier.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplier.java @@ -114,6 +114,7 @@ private List listFilteredS3ObjectsForBucket(final List globalStateMap) { final Instant previousScanTime = globalStateMap.get(bucket) != null ? Instant.parse((String) globalStateMap.get(bucket)) : null; + final boolean isFirstScan = previousScanTime == null; final List allPartitionIdentifiers = new ArrayList<>(); ListObjectsV2Response listObjectsV2Response = null; do { @@ -124,7 +125,7 @@ private List listFilteredS3ObjectsForBucket(final List !keyTimestampPair.left().endsWith("/")) .filter(keyTimestampPair -> excludeKeyPaths.stream() .noneMatch(excludeItem -> keyTimestampPair.left().endsWith(excludeItem))) - .filter(keyTimestampPair -> isKeyMatchedBetweenTimeRange(keyTimestampPair.right(), startDateTime, endDateTime)) + .filter(keyTimestampPair -> isKeyMatchedBetweenTimeRange(keyTimestampPair.right(), startDateTime, endDateTime, isFirstScan)) .map(Pair::left) .map(objectKey -> PartitionIdentifier.builder().withPartitionKey(String.format(BUCKET_OBJECT_PARTITION_KEY_FORMAT, bucket, objectKey)).build()) .collect(Collectors.toList())); @@ -166,8 +167,9 @@ private LocalDateTime instantToLocalDateTime(final Instant instant) { */ private boolean isKeyMatchedBetweenTimeRange(final LocalDateTime lastModifiedTime, final LocalDateTime startDateTime, - final LocalDateTime endDateTime) { - if (Objects.nonNull(schedulingOptions)) { + final LocalDateTime endDateTime, + final boolean isFirstScan) { + if (!isFirstScan && schedulingOptions != null) { return true; } else if (Objects.isNull(startDateTime) && Objects.isNull(endDateTime)) { return true; diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/configuration/S3ScanScanOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/configuration/S3ScanScanOptions.java index 9e59e47d8c..c0a5d50711 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/configuration/S3ScanScanOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/configuration/S3ScanScanOptions.java @@ -55,9 +55,14 @@ public boolean hasValidTimeAndRangeOptions() { return (startTime != null || endTime != null) && range != null; } - @AssertTrue(message = "start_time, end_time, and range are not valid options when using scheduling with s3 scan") + @AssertTrue(message = "end_time is not a valid option when using scheduling with s3 scan. One of start_time or range must be used for scheduled scan.") public boolean hasValidTimeOptionsWithScheduling() { - return !Objects.nonNull(schedulingOptions) || Stream.of(startTime, endTime, range).noneMatch(Objects::nonNull); + + if (schedulingOptions != null && ((startTime != null && range != null) || endTime != null)) { + return false; + } + + return true; } public Duration getRange() { diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplierTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplierTest.java index 0545a49459..867bcb1dc5 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplierTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/S3ScanPartitionCreationSupplierTest.java @@ -34,6 +34,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.TimeUnit; import java.util.function.Function; import java.util.stream.Collectors; @@ -44,6 +45,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.BDDMockito.given; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -289,6 +291,67 @@ void getNextPartition_supplier_with_scheduling_options_returns_expected_Partitio verify(listObjectsResponse, times(4)).contents(); } + + @Test + void scheduled_scan_filters_on_start_time_and_end_time_for_the_first_scan_and_does_not_filter_on_subsequent_scans() { + schedulingOptions = mock(S3ScanSchedulingOptions.class); + given(schedulingOptions.getCount()).willReturn(2); + + final String firstScanBucket = "bucket-one"; + final String notFirstScanBucket = "bucket-two"; + + final Map globalStateMap = new HashMap<>(); + globalStateMap.put(firstScanBucket, null); + globalStateMap.put(notFirstScanBucket, "2024-09-07T20:43:34.384822Z"); + globalStateMap.put(SCAN_COUNT, 0); + + final LocalDateTime startTime = LocalDateTime.ofInstant(Instant.ofEpochMilli(1725907846000L), ZoneId.systemDefault()); + final LocalDateTime endTime = LocalDateTime.ofInstant(Instant.ofEpochMilli(1725907849100L), ZoneId.systemDefault()); + + final ScanOptions firstBucketScanOptions = mock(ScanOptions.class); + final S3ScanBucketOption firstBucketScanBucketOption = mock(S3ScanBucketOption.class); + given(firstBucketScanOptions.getBucketOption()).willReturn(firstBucketScanBucketOption); + given(firstBucketScanBucketOption.getName()).willReturn(firstScanBucket); + given(firstBucketScanOptions.getUseStartDateTime()).willReturn(startTime); + given(firstBucketScanOptions.getUseEndDateTime()).willReturn(endTime); + scanOptionsList.add(firstBucketScanOptions); + + final ScanOptions notFirstScanOptions = mock(ScanOptions.class); + final S3ScanBucketOption notFirstScanBucketOption = mock(S3ScanBucketOption.class); + given(notFirstScanOptions.getBucketOption()).willReturn(notFirstScanBucketOption); + given(notFirstScanBucketOption.getName()).willReturn(notFirstScanBucket); + given(notFirstScanOptions.getUseStartDateTime()).willReturn(startTime); + given(notFirstScanOptions.getUseEndDateTime()).willReturn(endTime); + scanOptionsList.add(notFirstScanOptions); + + final ListObjectsV2Response listObjectsResponse = mock(ListObjectsV2Response.class); + final List s3ObjectsList = new ArrayList<>(); + + final Instant objectNotBetweenStartAndEndTime = Instant.ofEpochMilli(1725907846000L).minus(500L, TimeUnit.SECONDS.toChronoUnit()); + final S3Object validObject = mock(S3Object.class); + given(validObject.key()).willReturn("valid"); + given(validObject.lastModified()).willReturn(objectNotBetweenStartAndEndTime); + s3ObjectsList.add(validObject); + + final List expectedPartitionIdentifiers = new ArrayList<>(); + expectedPartitionIdentifiers.add(PartitionIdentifier.builder().withPartitionKey(notFirstScanBucket + "|" + validObject.key()).build()); + + given(listObjectsResponse.contents()) + .willReturn(s3ObjectsList); + + given(s3Client.listObjectsV2(any(ListObjectsV2Request.class))).willReturn(listObjectsResponse); + + final Function, List> partitionCreationSupplier = createObjectUnderTest(); + + final List firstScanPartitions = partitionCreationSupplier.apply(globalStateMap); + assertThat(firstScanPartitions.size(), equalTo(expectedPartitionIdentifiers.size())); + assertThat(firstScanPartitions.stream().map(PartitionIdentifier::getPartitionKey).collect(Collectors.toList()), + containsInAnyOrder(expectedPartitionIdentifiers.stream().map(PartitionIdentifier::getPartitionKey).map(Matchers::equalTo).collect(Collectors.toList()))); + + final List secondScanPartitions = partitionCreationSupplier.apply(globalStateMap); + assertThat(secondScanPartitions.isEmpty(), equalTo(true)); + } + @Test void getNextPartition_with_folder_partitioning_enabled_returns_the_expected_partition_identifiers() { folderPartitioningOptions = mock(FolderPartitioningOptions.class); diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/configuration/S3ScanScanOptionsTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/configuration/S3ScanScanOptionsTest.java index 33e9245b8b..5aeb9ca5cd 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/configuration/S3ScanScanOptionsTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/configuration/S3ScanScanOptionsTest.java @@ -45,4 +45,60 @@ public void s3scan_options_test_with_scan_yaml_configuration_test() throws JsonP assertThat(s3ScanScanOptions.getBuckets().get(0).getS3ScanBucketOption().getS3ScanFilter().getS3ScanExcludeSuffixOptions().get(0), equalTo(".jpeg")); } + + @Test + public void s3scan_options_with_scheduled_scan_does_not_allow_end_time() throws JsonProcessingException { + final String scanYaml = " start_time: 2023-01-21T18:00:00\n" + + " end_time: 2023-04-21T18:00:00\n" + + " scheduling: \n" + + " count: 1\n" + + " buckets:\n" + + " - bucket:\n" + + " name: test-s3-source-test-output\n" + + " filter:\n" + + " include_prefix:\n" + + " - bucket2\n" + + " exclude_suffix:\n" + + " - .jpeg"; + 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.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)); + 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().getS3ScanFilter().getS3ScanExcludeSuffixOptions().get(0), + equalTo(".jpeg")); + + assertThat(s3ScanScanOptions.hasValidTimeOptionsWithScheduling(), equalTo(false)); + } + + @Test + public void s3scan_options_with_scheduled_scan_allows_start_time() throws JsonProcessingException { + final String scanYaml = " start_time: 2023-01-21T18:00:00\n" + + " scheduling: \n" + + " count: 1\n" + + " buckets:\n" + + " - bucket:\n" + + " name: test-s3-source-test-output\n" + + " filter:\n" + + " include_prefix:\n" + + " - bucket2\n" + + " exclude_suffix:\n" + + " - .jpeg"; + final S3ScanScanOptions s3ScanScanOptions = objectMapper.readValue(scanYaml, S3ScanScanOptions.class); + assertThat(s3ScanScanOptions.getStartTime(),equalTo(LocalDateTime.parse("2023-01-21T18:00:00"))); + 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)); + 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().getS3ScanFilter().getS3ScanExcludeSuffixOptions().get(0), + equalTo(".jpeg")); + + assertThat(s3ScanScanOptions.hasValidTimeOptionsWithScheduling(), equalTo(true)); + } } From e8e92713cc9d08a46a98eccff0599ce5e9852946 Mon Sep 17 00:00:00 2001 From: qhung Date: Tue, 10 Sep 2024 22:06:24 +0200 Subject: [PATCH 154/159] adding registry to image value in docker-compose.yaml (#2980) Podman requires the registry url in order to pull out the image Signed-off-by: qhung <11665161+quanghungb@users.noreply.github.com> --- examples/log-ingestion/docker-compose.yaml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/examples/log-ingestion/docker-compose.yaml b/examples/log-ingestion/docker-compose.yaml index c7864d50dd..7c021d2343 100644 --- a/examples/log-ingestion/docker-compose.yaml +++ b/examples/log-ingestion/docker-compose.yaml @@ -2,7 +2,7 @@ version: '3' services: fluent-bit: container_name: fluent-bit - image: fluent/fluent-bit + image: docker.io/fluent/fluent-bit volumes: - ./fluent-bit.conf:/fluent-bit/etc/fluent-bit.conf - ./test.log:/var/log/test.log @@ -10,7 +10,7 @@ services: - opensearch-net opensearch: container_name: opensearch - image: opensearchproject/opensearch:latest + image: docker.io/opensearchproject/opensearch:latest environment: - discovery.type=single-node - bootstrap.memory_lock=true # along with the memlock settings below, disables swapping @@ -29,7 +29,7 @@ services: networks: - opensearch-net dashboards: - image: opensearchproject/opensearch-dashboards:latest + image: docker.io/opensearchproject/opensearch-dashboards:latest container_name: opensearch-dashboards ports: - 5601:5601 @@ -43,4 +43,4 @@ services: - opensearch-net networks: - opensearch-net: \ No newline at end of file + opensearch-net: From 400713b650608dc2d58750953403a3f5814062c2 Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 11 Sep 2024 15:48:12 -0500 Subject: [PATCH 155/159] Updating the WhiteSource/Mend configuration to match the version found in opensearch-system-templates. (#4933) https://github.com/opensearch-project/opensearch-system-templates/blob/e3b4fc6/.whitesource Signed-off-by: David Venable --- .whitesource | 22 ++++++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/.whitesource b/.whitesource index db4b0fec82..d465069a27 100644 --- a/.whitesource +++ b/.whitesource @@ -5,11 +5,29 @@ "projectToken": "", "baseBranches": [] }, + "scanSettingsSAST": { + "enableScan": false, + "scanPullRequests": false, + "incrementalScan": true, + "baseBranches": [], + "snippetSize": 10 + }, "checkRunSettings": { "vulnerableCheckRunConclusionLevel": "failure", - "displayMode": "diff" + "displayMode": "diff", + "useMendCheckNames": true + }, + "checkRunSettingsSAST": { + "checkRunConclusionLevel": "failure", + "severityThreshold": "high" }, "issueSettings": { - "minSeverityLevel": "LOW" + "minSeverityLevel": "LOW", + "issueType": "DEPENDENCY" + }, + "remediateSettings": { + "workflowRules": { + "enabled": true + } } } \ No newline at end of file From 4e50f3bf427208728618f976d5eb2ac8fb908693 Mon Sep 17 00:00:00 2001 From: Ling Hengqian Date: Thu, 12 Sep 2024 10:21:59 +0800 Subject: [PATCH 156/159] Fixes Jaeger Hotrod demo failure on OpenSearch 2.16.0 (#4921) Signed-off-by: linghengqian --- examples/jaeger-hotrod/README.md | 5 +-- examples/jaeger-hotrod/docker-compose.yml | 32 ++++++++++---------- examples/jaeger-hotrod/pipelines.yaml | 37 +++++++++++++++++++++++ 3 files changed, 56 insertions(+), 18 deletions(-) create mode 100644 examples/jaeger-hotrod/pipelines.yaml diff --git a/examples/jaeger-hotrod/README.md b/examples/jaeger-hotrod/README.md index 87b72b3337..7950bddd3c 100644 --- a/examples/jaeger-hotrod/README.md +++ b/examples/jaeger-hotrod/README.md @@ -7,7 +7,7 @@ This demo will use the revered [Jaeger HotROD](https://github.com/jaegertracing/ #### Demo ``` -docker-compose up -d --build +docker compose up -d ``` The above command will start the Jaeger HotROD sample, Jaeger Agent, OpenTelemetry Collector, Data Prepper, OpenSearch and OpenSearch Dashboards. Wait for few minutes for all the containers to come up, the DataPrepper container will restart until OpenSearch becomes available. @@ -15,5 +15,6 @@ The above command will start the Jaeger HotROD sample, Jaeger Agent, OpenTelemet After the Docker image is running, do the following. * Open the HotROD app at [http://localhost:8080](http://localhost:8080). Press the buttons in the UI to simulate requests. -* Load the OpenSearch Dashboards trace analytics dashboard at [http://localhost:5601/app/observability-dashboards#/trace_analytics/home](http://localhost:5601/app/observability-dashboards#/trace_analytics/home). If that link does not work, you may still be using On OpenSearch 1.1.0 or below. You will need to use [http://localhost:5601/app/trace-analytics-dashboards#/](http://localhost:5601/app/trace-analytics-dashboards#/) instead. You can view traces and the service map here. +* Log in to the OpenSearch Dashboards Web UI at [http://localhost:5601](http://localhost:5601) using the username `admin` and the password `yourStrongPassword123!`. +* Load the OpenSearch Dashboards trace analytics dashboard at [http://localhost:5601/app/observability-traces#/services](http://localhost:5601/app/observability-traces#/services). If that link does not work, you may still be using On OpenSearch 1.1.0 or below. You will need to use [http://localhost:5601/app/trace-analytics-dashboards#/](http://localhost:5601/app/trace-analytics-dashboards#/) instead. You can view traces and the service map here. diff --git a/examples/jaeger-hotrod/docker-compose.yml b/examples/jaeger-hotrod/docker-compose.yml index 879d9ec16d..5e15860dc7 100644 --- a/examples/jaeger-hotrod/docker-compose.yml +++ b/examples/jaeger-hotrod/docker-compose.yml @@ -1,28 +1,24 @@ -version: "3.7" services: data-prepper: restart: unless-stopped - container_name: data-prepper image: opensearchproject/data-prepper:2 volumes: - - ../trace_analytics_no_ssl_2x.yml:/usr/share/data-prepper/pipelines/pipelines.yaml + - ./pipelines.yaml:/usr/share/data-prepper/pipelines/pipelines.yaml - ../data-prepper-config.yaml:/usr/share/data-prepper/config/data-prepper-config.yaml - - ../demo/root-ca.pem:/usr/share/data-prepper/root-ca.pem + - opensearch-config:/usr/share/opensearch-test/:ro ports: - "21890:21890" networks: - my_network depends_on: - - "opensearch" + - opensearch otel-collector: - container_name: otel-collector image: otel/opentelemetry-collector:0.64.1 command: [ "--config=/etc/otel-collector-config.yml" ] working_dir: "/project" volumes: - ${PWD}/:/project - ./otel-collector-config.yml:/etc/otel-collector-config.yml - - ../demo/demo-data-prepper.crt:/etc/demo-data-prepper.crt ports: - "14250:14250" depends_on: @@ -30,7 +26,6 @@ services: networks: - my_network jaeger-agent: - container_name: jaeger-agent image: jaegertracing/jaeger-agent:1.51.0 command: [ "--reporter.grpc.host-port=otel-collector:14250" ] ports: @@ -40,6 +35,8 @@ services: - "5778:5778/tcp" networks: - my_network + depends_on: + - otel-collector jaeger-hot-rod: image: jaegertracing/example-hotrod:1.41.0 command: [ "all" ] @@ -54,11 +51,14 @@ services: - my_network opensearch: container_name: node-0.example.com - image: opensearchproject/opensearch:2.9.0 + image: opensearchproject/opensearch:2.16.0 environment: - discovery.type=single-node - bootstrap.memory_lock=true # along with the memlock settings below, disables swapping - "OPENSEARCH_JAVA_OPTS=-Xms512m -Xmx512m" # minimum and maximum Java heap size, recommend setting both to 50% of system RAM + - OPENSEARCH_INITIAL_ADMIN_PASSWORD=yourStrongPassword123! + volumes: + - opensearch-config:/usr/share/opensearch/config/ ulimits: memlock: soft: -1 @@ -67,17 +67,15 @@ services: soft: 65536 # maximum number of open files for the OpenSearch user, set to at least 65536 on modern systems hard: 65536 ports: - - 9200:9200 - - 9600:9600 # required for Performance Analyzer + - "9200:9200" + - "9600:9600" # required for Performance Analyzer networks: - my_network dashboards: - image: opensearchproject/opensearch-dashboards:2.9.0 + image: opensearchproject/opensearch-dashboards:2.16.0 container_name: opensearch-dashboards ports: - - 5601:5601 - expose: - - "5601" + - "5601:5601" environment: OPENSEARCH_HOSTS: '["https://node-0.example.com:9200"]' depends_on: @@ -85,4 +83,6 @@ services: networks: - my_network networks: - my_network: \ No newline at end of file + my_network: +volumes: + opensearch-config: diff --git a/examples/jaeger-hotrod/pipelines.yaml b/examples/jaeger-hotrod/pipelines.yaml new file mode 100644 index 0000000000..ef9ec67dba --- /dev/null +++ b/examples/jaeger-hotrod/pipelines.yaml @@ -0,0 +1,37 @@ +entry-pipeline: + delay: "100" + source: + otel_trace_source: + ssl: false + sink: + - pipeline: + name: "raw-pipeline" + - pipeline: + name: "service-map-pipeline" +raw-pipeline: + source: + pipeline: + name: "entry-pipeline" + processor: + - otel_trace_raw: + sink: + - opensearch: + hosts: [ "https://node-0.example.com:9200" ] + cert: "/usr/share/opensearch-test/root-ca.pem" + username: "admin" + password: "yourStrongPassword123!" + index_type: trace-analytics-raw +service-map-pipeline: + delay: "100" + source: + pipeline: + name: "entry-pipeline" + processor: + - service_map_stateful: + sink: + - opensearch: + hosts: ["https://node-0.example.com:9200"] + cert: "/usr/share/opensearch-test/root-ca.pem" + username: "admin" + password: "yourStrongPassword123!" + index_type: trace-analytics-service-map From ed5ca4b9535c9be33862e819cf072e4847bb88c8 Mon Sep 17 00:00:00 2001 From: Dinu John <86094133+dinujoh@users.noreply.github.com> Date: Thu, 12 Sep 2024 12:10:10 -0500 Subject: [PATCH 157/159] Move CsvMapper and Schema creation to constructor (#4941) Move CsvMapper and Schema creation to constructor Signed-off-by: Dinu John <86094133+dinujoh@users.noreply.github.com> --- .../dataprepper/plugins/processor/csv/CsvProcessor.java | 8 +++++--- .../plugins/processor/csv/CsvProcessorTest.java | 2 ++ 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java index 3e8780a417..405b165ac5 100644 --- a/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java +++ b/data-prepper-plugins/csv-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessor.java @@ -46,6 +46,9 @@ public class CsvProcessor extends AbstractProcessor, Record private final ExpressionEvaluator expressionEvaluator; + private final CsvMapper mapper; + private final CsvSchema schema; + @DataPrepperPluginConstructor public CsvProcessor(final PluginMetrics pluginMetrics, final CsvProcessorConfig config, @@ -61,13 +64,12 @@ public CsvProcessor(final PluginMetrics pluginMetrics, String.format("csv_when value of %s is not a valid expression statement. " + "See https://opensearch.org/docs/latest/data-prepper/pipelines/expression-syntax/ for valid expression syntax.", config.getCsvWhen())); } + this.mapper = createCsvMapper(); + this.schema = createCsvSchema(); } @Override public Collection> doExecute(final Collection> records) { - final CsvMapper mapper = createCsvMapper(); - final CsvSchema schema = createCsvSchema(); - for (final Record record : records) { final Event event = record.getData(); diff --git a/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java b/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java index cacfedb609..0e36d8ebea 100644 --- a/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java +++ b/data-prepper-plugins/csv-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/csv/CsvProcessorTest.java @@ -117,6 +117,7 @@ void test_when_messageIsEmpty_then_notParsed() { @Test void test_when_delimiterIsTab_then_parsedCorrectly() { when(processorConfig.getDelimiter()).thenReturn("\t"); + csvProcessor = createObjectUnderTest(); Record eventUnderTest = createMessageEvent("1\t2\t3"); final List> editedEvents = (List>) csvProcessor.doExecute(Collections.singletonList(eventUnderTest)); @@ -293,6 +294,7 @@ void test_when_twoEventsAndNoHeaderSourceOnOne_then_ChooseUserColumns() { @Test void test_when_differentQuoteCharacter_then_parsesCorrectly() { when(processorConfig.getQuoteCharacter()).thenReturn("\'"); + csvProcessor = createObjectUnderTest(); final Record eventUnderTest = createMessageEvent("'1','2','3'"); final List> editedEvents = (List>) csvProcessor.doExecute(Collections.singletonList(eventUnderTest)); From 627ae9ffdb2de5dda84566e7b6a4f130f5005210 Mon Sep 17 00:00:00 2001 From: Srikanth Govindarajan Date: Thu, 12 Sep 2024 14:31:16 -0700 Subject: [PATCH 158/159] Release events for Noop Sink (#4944) Signed-off-by: Srikanth Govindarajan --- .../dataprepper/plugins/sink/NoopSink.java | 11 ++- .../plugins/sink/NoopSinkTest.java | 76 +++++++++++++++++++ 2 files changed, 86 insertions(+), 1 deletion(-) create mode 100644 data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/NoopSinkTest.java diff --git a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/NoopSink.java b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/NoopSink.java index f184bd0f40..0ce0194b31 100644 --- a/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/NoopSink.java +++ b/data-prepper-plugins/common/src/main/java/org/opensearch/dataprepper/plugins/sink/NoopSink.java @@ -1,16 +1,25 @@ package org.opensearch.dataprepper.plugins.sink; import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.sink.Sink; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Collection; @DataPrepperPlugin(name = "noop", pluginType = Sink.class) public class NoopSink implements Sink> { + private static final Logger LOG = LoggerFactory.getLogger(NoopSink.class); + @Override public void output(Collection> records) { - // empty by design. + LOG.info("Releasing events for NOOP sink"); + for (Record record : records) { + Event event = (Event)record.getData(); + event.getEventHandle().release(true); + } } @Override diff --git a/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/NoopSinkTest.java b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/NoopSinkTest.java new file mode 100644 index 0000000000..e4ffeeba3d --- /dev/null +++ b/data-prepper-plugins/common/src/test/java/org/opensearch/dataprepper/plugins/sink/NoopSinkTest.java @@ -0,0 +1,76 @@ +package org.opensearch.dataprepper.plugins.sink; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import org.mockito.MockitoAnnotations; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventHandle; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; + +import java.util.List; + +public class NoopSinkTest { + + private NoopSink noopSink; + + @Mock + private Record mockRecord; + + @Mock + private Event mockEvent; + + @Mock + private EventHandle mockEventHandle; + + @Mock + private Logger logger; + + @BeforeEach + public void setup() { + MockitoAnnotations.openMocks(this); + noopSink = new NoopSink(); + } + + @Test + public void testOutput_releasesEventHandles() { + // Arrange + List> records = List.of(mockRecord); + when(mockRecord.getData()).thenReturn(mockEvent); + when(mockEvent.getEventHandle()).thenReturn(mockEventHandle); + + // Act + noopSink.output(records); + + // Assert + verify(mockEventHandle, times(1)).release(true); + } + + @Test + public void testOutput_multipleRecords_releasesAllEventHandles() { + // Arrange + Record mockRecord2 = mock(Record.class); + Event mockEvent2 = mock(Event.class); + EventHandle mockEventHandle2 = mock(EventHandle.class); + + List> records = List.of(mockRecord, mockRecord2); + + when(mockRecord.getData()).thenReturn(mockEvent); + when(mockEvent.getEventHandle()).thenReturn(mockEventHandle); + + when(mockRecord2.getData()).thenReturn(mockEvent2); + when(mockEvent2.getEventHandle()).thenReturn(mockEventHandle2); + + // Act + noopSink.output(records); + + // Assert + verify(mockEventHandle, times(1)).release(true); + verify(mockEventHandle2, times(1)).release(true); + } +} From 72063d6a4a19650141536009ae043599864c73b5 Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 13 Sep 2024 15:30:13 -0500 Subject: [PATCH 159/159] Decrease the sleep when pausing the Kafka consumer to 1 second when the circuit breaker is in use. (#4947) Signed-off-by: David Venable --- .../dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java index 9e6a979d8e..8a64c42e3b 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java @@ -357,7 +357,7 @@ public void run() { LOG.debug("Pause and skip consuming from Kafka topic due to an external condition: {}", pauseConsumePredicate); paused = true; consumer.pause(consumer.assignment()); - Thread.sleep(10000); + Thread.sleep(1000); continue; } else if(paused) { LOG.debug("Resume consuming from Kafka topic.");