From d3a9099b451261ccb87bdf385d2084f95ccf5712 Mon Sep 17 00:00:00 2001 From: kkondaka <41027584+kkondaka@users.noreply.github.com> Date: Wed, 2 Aug 2023 10:43:37 -0700 Subject: [PATCH 1/7] Fix consumer synchronization. Fix consumer to use user-specified groupId (#3100) * Fix consumer synchronization. Fix consumer to use user-specified groupId Signed-off-by: Krishna Kondaka * Fix check style error Signed-off-by: Krishna Kondaka * Fixed to retry if consume records encounters an exception Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka Co-authored-by: Krishna Kondaka --- .../kafka/source/KafkaSourceJsonTypeIT.java | 4 +- .../kafka/configuration/TopicConfig.java | 64 +++++++++-------- .../consumer/KafkaSourceCustomConsumer.java | 68 ++++++++++--------- .../plugins/kafka/source/KafkaSource.java | 7 +- .../util/KafkaSourceSecurityConfigurer.java | 11 +-- .../kafka/configuration/TopicConfigTest.java | 30 ++++---- .../test/resources/sample-pipelines-1.yaml | 6 +- .../src/test/resources/sample-pipelines.yaml | 1 + 8 files changed, 105 insertions(+), 86 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java b/data-prepper-plugins/kafka-plugins/src/integrationTest/java/org/opensearch/dataprepper/plugins/kafka/source/KafkaSourceJsonTypeIT.java index 6e7f612534..584ee24eb5 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 @@ -83,6 +83,7 @@ public class KafkaSourceJsonTypeIT { private String bootstrapServers; private String testKey; private String testTopic; + private String testGroup; public KafkaSource createObjectUnderTest() { return new KafkaSource(sourceConfig, pluginMetrics, acknowledgementSetManager, pipelineDescription); @@ -112,7 +113,7 @@ public void setup() { } catch (Exception e){} testKey = RandomStringUtils.randomAlphabetic(5); - final String testGroup = "TestGroup_"+RandomStringUtils.randomAlphabetic(6); + testGroup = "TestGroup_"+RandomStringUtils.randomAlphabetic(6); testTopic = "TestJsonTopic_"+RandomStringUtils.randomAlphabetic(5); jsonTopic = mock(TopicConfig.class); when(jsonTopic.getName()).thenReturn(testTopic); @@ -337,6 +338,7 @@ public void TestJsonRecordsWithKafkaKeyModeAsMetadata() throws Exception { Thread.sleep(1000); } kafkaSource.start(buffer); + assertThat(kafkaSource.getConsumer().groupMetadata().groupId(), equalTo(testGroup)); produceJsonRecords(bootstrapServers, topicName, numRecords); int numRetries = 0; while (numRetries++ < 10 && (receivedRecords.size() != numRecords)) { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java index 2f3ab61fcb..caf4f1236c 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfig.java @@ -18,23 +18,24 @@ * pipelines.yaml */ public class TopicConfig { - private static final String AUTO_COMMIT = "false"; - private static final Duration DEFAULT_COMMIT_INTERVAL = Duration.ofSeconds(5); - private static final Duration DEFAULT_SESSION_TIMEOUT = Duration.ofSeconds(45); - private static final int MAX_RETRY_ATTEMPT = Integer.MAX_VALUE; + static final boolean DEFAULT_AUTO_COMMIT = false; + static final Duration DEFAULT_COMMIT_INTERVAL = Duration.ofSeconds(5); + static final Duration DEFAULT_SESSION_TIMEOUT = Duration.ofSeconds(45); + static final int DEFAULT_MAX_RETRY_ATTEMPT = Integer.MAX_VALUE; static final String DEFAULT_AUTO_OFFSET_RESET = "latest"; - static final Duration THREAD_WAITING_TIME = Duration.ofSeconds(5); - private static final Duration MAX_RECORD_FETCH_TIME = Duration.ofSeconds(4); - private static final Duration BUFFER_DEFAULT_TIMEOUT = Duration.ofSeconds(5); - private static final Duration MAX_RETRY_DELAY = Duration.ofSeconds(1); - private static final Integer FETCH_MAX_BYTES = 52428800; - private static final Integer FETCH_MAX_WAIT = 500; - private static final Integer FETCH_MIN_BYTES = 1; - private static final Duration RETRY_BACKOFF = Duration.ofSeconds(100); - private static final Duration MAX_POLL_INTERVAL = Duration.ofSeconds(300000); - private static final Integer CONSUMER_MAX_POLL_RECORDS = 500; + static final Duration DEFAULT_THREAD_WAITING_TIME = Duration.ofSeconds(5); + static final Duration DEFAULT_MAX_RECORD_FETCH_TIME = Duration.ofSeconds(4); + static final Duration DEFAULT_BUFFER_TIMEOUT = Duration.ofSeconds(5); + static final Duration DEFAULT_MAX_RETRY_DELAY = Duration.ofSeconds(1); + static final Integer DEFAULT_FETCH_MAX_BYTES = 52428800; + static final Integer DEFAULT_FETCH_MAX_WAIT = 500; + static final Integer DEFAULT_FETCH_MIN_BYTES = 1; + static final Duration DEFAULT_RETRY_BACKOFF = Duration.ofSeconds(10); + static final Duration DEFAULT_RECONNECT_BACKOFF = Duration.ofSeconds(10); + static final Duration DEFAULT_MAX_POLL_INTERVAL = Duration.ofSeconds(300000); + static final Integer DEFAULT_CONSUMER_MAX_POLL_RECORDS = 500; static final Integer DEFAULT_NUM_OF_WORKERS = 2; - static final Duration HEART_BEAT_INTERVAL_DURATION = Duration.ofSeconds(5); + static final Duration DEFAULT_HEART_BEAT_INTERVAL_DURATION = Duration.ofSeconds(5); @JsonProperty("name") @NotNull @@ -54,18 +55,18 @@ public class TopicConfig { @JsonProperty("max_retry_attempts") @Valid @Size(min = 1, max = Integer.MAX_VALUE, message = " Max retry attempts should lies between 1 and Integer.MAX_VALUE") - private Integer maxRetryAttempts = MAX_RETRY_ATTEMPT; + private Integer maxRetryAttempts = DEFAULT_MAX_RETRY_ATTEMPT; @JsonProperty("max_retry_delay") @Valid @Size(min = 1) - private Duration maxRetryDelay = MAX_RETRY_DELAY; + private Duration maxRetryDelay = DEFAULT_MAX_RETRY_DELAY; @JsonProperty("serde_format") private MessageFormat serdeFormat= MessageFormat.PLAINTEXT; @JsonProperty("auto_commit") - private Boolean autoCommit = false; + private Boolean autoCommit = DEFAULT_AUTO_COMMIT; @JsonProperty("commit_interval") @Valid @@ -86,47 +87,50 @@ public class TopicConfig { private String groupName; @JsonProperty("thread_waiting_time") - private Duration threadWaitingTime = THREAD_WAITING_TIME; + private Duration threadWaitingTime = DEFAULT_THREAD_WAITING_TIME; @JsonProperty("max_record_fetch_time") - private Duration maxRecordFetchTime = MAX_RECORD_FETCH_TIME; + private Duration maxRecordFetchTime = DEFAULT_MAX_RECORD_FETCH_TIME; @JsonProperty("buffer_default_timeout") @Valid @Size(min = 1) - private Duration bufferDefaultTimeout = BUFFER_DEFAULT_TIMEOUT; + private Duration bufferDefaultTimeout = DEFAULT_BUFFER_TIMEOUT; @JsonProperty("fetch_max_bytes") @Valid @Size(min = 1, max = 52428800) - private Integer fetchMaxBytes = FETCH_MAX_BYTES; + private Integer fetchMaxBytes = DEFAULT_FETCH_MAX_BYTES; @JsonProperty("fetch_max_wait") @Valid @Size(min = 1) - private Integer fetchMaxWait = FETCH_MAX_WAIT; + private Integer fetchMaxWait = DEFAULT_FETCH_MAX_WAIT; @JsonProperty("fetch_min_bytes") @Size(min = 1) @Valid - private Integer fetchMinBytes = FETCH_MIN_BYTES; + private Integer fetchMinBytes = DEFAULT_FETCH_MIN_BYTES; @JsonProperty("key_mode") private KafkaKeyMode kafkaKeyMode = KafkaKeyMode.INCLUDE_AS_FIELD; @JsonProperty("retry_backoff") - private Duration retryBackoff = RETRY_BACKOFF; + private Duration retryBackoff = DEFAULT_RETRY_BACKOFF; + + @JsonProperty("reconnect_backoff") + private Duration reconnectBackoff = DEFAULT_RECONNECT_BACKOFF; @JsonProperty("max_poll_interval") - private Duration maxPollInterval = MAX_POLL_INTERVAL; + private Duration maxPollInterval = DEFAULT_MAX_POLL_INTERVAL; @JsonProperty("consumer_max_poll_records") - private Integer consumerMaxPollRecords = CONSUMER_MAX_POLL_RECORDS; + private Integer consumerMaxPollRecords = DEFAULT_CONSUMER_MAX_POLL_RECORDS; @JsonProperty("heart_beat_interval") @Valid @Size(min = 1) - private Duration heartBeatInterval= HEART_BEAT_INTERVAL_DURATION; + private Duration heartBeatInterval= DEFAULT_HEART_BEAT_INTERVAL_DURATION; public String getGroupId() { return groupId; @@ -220,6 +224,10 @@ public Duration getRetryBackoff() { return retryBackoff; } + public Duration getReconnectBackoff() { + return reconnectBackoff; + } + public void setRetryBackoff(Duration retryBackoff) { this.retryBackoff = retryBackoff; } diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java index 417518836b..e1a071ffcb 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaSourceCustomConsumer.java @@ -15,6 +15,7 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.CommitFailedException; import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.RecordDeserializationException; import org.apache.kafka.common.TopicPartition; import org.apache.avro.generic.GenericRecord; import org.opensearch.dataprepper.model.log.JacksonLog; @@ -40,6 +41,8 @@ import java.util.Collection; import java.util.List; import java.util.Objects; +import java.util.Set; +import java.util.HashSet; import java.util.concurrent.atomic.AtomicBoolean; import org.opensearch.dataprepper.plugins.kafka.util.MessageFormat; import com.amazonaws.services.schemaregistry.serializers.json.JsonDataWithSchema; @@ -68,6 +71,7 @@ public class KafkaSourceCustomConsumer implements Runnable, ConsumerRebalanceLis private static final ObjectMapper objectMapper = new ObjectMapper(); private final JsonFactory jsonFactory = new JsonFactory(); private Map offsetsToCommit; + private Set partitionsToReset; private final AcknowledgementSetManager acknowledgementSetManager; private final Map partitionCommitTrackerMap; private final Counter positiveAcknowledgementSetCounter; @@ -95,6 +99,7 @@ public KafkaSourceCustomConsumer(final KafkaConsumer consumer, this.acknowledgementSetManager = acknowledgementSetManager; this.pluginMetrics = pluginMetrics; this.partitionCommitTrackerMap = new HashMap<>(); + this.partitionsToReset = new HashSet<>(); this.schema = MessageFormat.getByMessageFormatByName(schemaType); Duration bufferTimeout = Duration.ofSeconds(1); this.bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE, bufferTimeout); @@ -121,29 +126,21 @@ private AcknowledgementSet createAcknowledgementSet(Map { - try { - synchronized(consumer) { - OffsetAndMetadata committedOffsetAndMetadata = consumer.committed(partition); - consumer.seek(partition, committedOffsetAndMetadata); - } - } catch (Exception e) { - LOG.error("Failed to seek to last committed offset upon negative acknowledgement "+partition, e); + synchronized(partitionsToReset) { + partitionsToReset.add(partition); } }); } @@ -157,10 +154,7 @@ private AcknowledgementSet createAcknowledgementSet(Map void consumeRecords() throws Exception { try { - ConsumerRecords records = null; - synchronized(consumer) { - records = consumer.poll(topicConfig.getThreadWaitingTime().toMillis()/2); - } + ConsumerRecords records = consumer.poll(topicConfig.getThreadWaitingTime().toMillis()/2); if (Objects.nonNull(records) && !records.isEmpty() && records.count() > 0) { Map> offsets = new HashMap<>(); AcknowledgementSet acknowledgementSet = null; @@ -176,12 +170,27 @@ public void consumeRecords() throws Exception { } } } catch (AuthenticationException e) { - LOG.warn("Authentication Error while doing poll(). Will retry after 10 seconds", e); + LOG.warn("Access Denied while doing poll(). Will retry after 10 seconds", e); Thread.sleep(10000); + } catch (RecordDeserializationException e) { + LOG.warn("Serialization error - topic {} partition {} offset {}, seeking past the error record", + e.topicPartition().topic(), e.topicPartition().partition(), e.offset()); + consumer.seek(e.topicPartition(), e.offset()+1); } } - private void commitOffsets() { + private void resetOrCommitOffsets() { + synchronized(partitionsToReset) { + partitionsToReset.forEach(partition -> { + try { + final OffsetAndMetadata offsetAndMetadata = consumer.committed(partition); + consumer.seek(partition, offsetAndMetadata); + } catch (Exception e) { + LOG.error("Failed to seek to last committed offset upon negative acknowledgement {}", partition, e); + } + }); + partitionsToReset.clear(); + } if (topicConfig.getAutoCommit()) { return; } @@ -194,13 +203,11 @@ private void commitOffsets() { return; } try { - synchronized(consumer) { - consumer.commitSync(); - } + consumer.commitSync(); offsetsToCommit.clear(); lastCommitTime = currentTimeMillis; } catch (CommitFailedException e) { - LOG.error("Failed to commit offsets in topic "+topicName, e); + LOG.error("Failed to commit offsets in topic {}", topicName, e); } } } @@ -211,14 +218,14 @@ Map getOffsetsToCommit() { @Override public void run() { - try { - consumer.subscribe(Arrays.asList(topicName)); - while (!shutdownInProgress.get()) { + consumer.subscribe(Arrays.asList(topicName)); + while (!shutdownInProgress.get()) { + try { + resetOrCommitOffsets(); consumeRecords(); - commitOffsets(); + } catch (Exception exp) { + LOG.error("Error while reading the records from the topic...", exp); } - } catch (Exception exp) { - LOG.error("Error while reading the records from the topic...", exp); } } @@ -306,9 +313,8 @@ public void shutdownConsumer(){ @Override public void onPartitionsAssigned(Collection partitions) { for (TopicPartition topicPartition : partitions) { - synchronized(consumer) { - Long committedOffset = consumer.committed(topicPartition).offset(); - consumer.seek(topicPartition, committedOffset); + synchronized(partitionsToReset) { + partitionsToReset.add(topicPartition); } } } 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 45f5a87446..b39c1f9ffa 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 @@ -95,6 +95,7 @@ public class KafkaSource implements Source> { private final Counter kafkaWorkerThreadProcessingErrors; private final PluginMetrics pluginMetrics; private KafkaSourceCustomConsumer consumer; + private KafkaConsumer kafkaConsumer; private String pipelineName; private String consumerGroupID; private String schemaType = MessageFormat.PLAINTEXT.toString(); @@ -125,7 +126,6 @@ public void start(Buffer> buffer) { int numWorkers = topic.getWorkers(); executorService = Executors.newFixedThreadPool(numWorkers); IntStream.range(0, numWorkers + 1).forEach(index -> { - KafkaConsumer kafkaConsumer; switch (schema) { case JSON: kafkaConsumer = new KafkaConsumer(consumerProperties); @@ -185,6 +185,9 @@ private long calculateLongestThreadWaitingTime() { orElse(1L); } + KafkaConsumer getConsumer() { + return kafkaConsumer; + } private Properties getConsumerProperties(final TopicConfig topicConfig) { Properties properties = new Properties(); @@ -361,6 +364,8 @@ private void setPropertiesForSchemaType(Properties properties, TopicConfig topic private void setConsumerTopicProperties(Properties properties, TopicConfig topicConfig) { properties.put(ConsumerConfig.GROUP_ID_CONFIG, consumerGroupID); + 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, diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java index fd57e8bfce..245056e06e 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaSourceSecurityConfigurer.java @@ -16,10 +16,7 @@ import software.amazon.awssdk.services.kafka.KafkaClient; import software.amazon.awssdk.services.kafka.model.GetBootstrapBrokersRequest; import software.amazon.awssdk.services.kafka.model.GetBootstrapBrokersResponse; -import software.amazon.awssdk.services.kafka.model.InternalServerErrorException; -import software.amazon.awssdk.services.kafka.model.ConflictException; -import software.amazon.awssdk.services.kafka.model.ForbiddenException; -import software.amazon.awssdk.services.kafka.model.UnauthorizedException; +import software.amazon.awssdk.services.kafka.model.KafkaException; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.services.sts.model.StsException; import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; @@ -214,17 +211,15 @@ public static String getBootStrapServersForMsk(final AwsIamAuthConfig awsIamAuth retryable = false; try { result = kafkaClient.getBootstrapBrokers(request); - } catch (InternalServerErrorException | ConflictException | ForbiddenException | UnauthorizedException | StsException e) { + } catch (KafkaException | StsException e) { LOG.debug("Failed to get bootstrap server information from MSK. Retrying...", e); - - retryable = true; try { Thread.sleep(10000); } catch (InterruptedException exp) {} } catch (Exception e) { throw new RuntimeException("Failed to get bootstrap server information from MSK.", e); } - } while (retryable && numRetries++ < MAX_KAFKA_CLIENT_RETRIES); + } while (numRetries++ < MAX_KAFKA_CLIENT_RETRIES); if (Objects.isNull(result)) { throw new RuntimeException("Failed to get bootstrap server information from MSK after trying multiple times with retryable exceptions."); } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java index db77e5cfeb..753d452095 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/configuration/TopicConfigTest.java @@ -67,21 +67,23 @@ void test_topicsConfig_not_null() { @Tag(YAML_FILE_WITH_MISSING_CONSUMER_CONFIG) void testConfigValues_default() { assertEquals("my-topic-2", topicConfig.getName()); - assertEquals(false, topicConfig.getAutoCommit()); - assertEquals(Duration.ofSeconds(5), topicConfig.getCommitInterval()); - assertEquals(45000, topicConfig.getSessionTimeOut().toMillis()); + assertEquals("my-test-group", topicConfig.getGroupId()); + assertEquals(TopicConfig.DEFAULT_AUTO_COMMIT, topicConfig.getAutoCommit()); + assertEquals(TopicConfig.DEFAULT_COMMIT_INTERVAL, topicConfig.getCommitInterval()); + assertEquals(TopicConfig.DEFAULT_SESSION_TIMEOUT, topicConfig.getSessionTimeOut()); assertEquals(TopicConfig.DEFAULT_AUTO_OFFSET_RESET, topicConfig.getAutoOffsetReset()); - assertEquals(TopicConfig.THREAD_WAITING_TIME, topicConfig.getThreadWaitingTime()); - assertEquals(Duration.ofSeconds(4), topicConfig.getMaxRecordFetchTime()); - assertEquals(Duration.ofSeconds(5), topicConfig.getBufferDefaultTimeout()); - assertEquals(52428800L, topicConfig.getFetchMaxBytes().longValue()); - assertEquals(500L, topicConfig.getFetchMaxWait().longValue()); - assertEquals(1L, topicConfig.getFetchMinBytes().longValue()); - assertEquals(Duration.ofSeconds(100), topicConfig.getRetryBackoff()); - assertEquals(Duration.ofSeconds(300000), topicConfig.getMaxPollInterval()); - assertEquals(500L, topicConfig.getConsumerMaxPollRecords().longValue()); - assertEquals(TopicConfig.DEFAULT_NUM_OF_WORKERS, topicConfig.getWorkers().intValue()); - assertEquals(TopicConfig.HEART_BEAT_INTERVAL_DURATION, topicConfig.getHeartBeatInterval()); + assertEquals(TopicConfig.DEFAULT_THREAD_WAITING_TIME, topicConfig.getThreadWaitingTime()); + assertEquals(TopicConfig.DEFAULT_MAX_RECORD_FETCH_TIME, topicConfig.getMaxRecordFetchTime()); + assertEquals(TopicConfig.DEFAULT_BUFFER_TIMEOUT, topicConfig.getBufferDefaultTimeout()); + assertEquals(TopicConfig.DEFAULT_FETCH_MAX_BYTES, topicConfig.getFetchMaxBytes()); + assertEquals(TopicConfig.DEFAULT_FETCH_MAX_WAIT, topicConfig.getFetchMaxWait()); + assertEquals(TopicConfig.DEFAULT_FETCH_MIN_BYTES, topicConfig.getFetchMinBytes()); + assertEquals(TopicConfig.DEFAULT_RETRY_BACKOFF, topicConfig.getRetryBackoff()); + assertEquals(TopicConfig.DEFAULT_RECONNECT_BACKOFF, topicConfig.getReconnectBackoff()); + assertEquals(TopicConfig.DEFAULT_MAX_POLL_INTERVAL, topicConfig.getMaxPollInterval()); + assertEquals(TopicConfig.DEFAULT_CONSUMER_MAX_POLL_RECORDS, topicConfig.getConsumerMaxPollRecords()); + assertEquals(TopicConfig.DEFAULT_NUM_OF_WORKERS, topicConfig.getWorkers()); + assertEquals(TopicConfig.DEFAULT_HEART_BEAT_INTERVAL_DURATION, topicConfig.getHeartBeatInterval()); } @Test diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml index c495902811..67f655e167 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines-1.yaml @@ -6,11 +6,11 @@ log-pipeline: topics: - name: my-topic-2 group_name: kafka-consumer-group-2 - group_id: DPKafkaProj-2 + group_id: my-test-group - name: my-topic-1 - group_id: DPKafkaProj-1 + group_id: my-test-group schema: registry_url: http://localhost:8081/ version: 1 sink: - - stdout: \ No newline at end of file + - stdout: diff --git a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml index fe3407471b..a76c38565e 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml +++ b/data-prepper-plugins/kafka-plugins/src/test/resources/sample-pipelines.yaml @@ -7,6 +7,7 @@ log-pipeline: encryption: plaintext topics: - name: my-topic-1 + group_id: my-test-group workers: 5 auto_commit: false commit_interval: PT5S From 7c7062e194d1dfd82b776ed1c8eece222e25428d Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 2 Aug 2023 13:59:04 -0500 Subject: [PATCH 2/7] Updated GitHub Actions to use "Data Prepper" in the job titles for consistency and aligning with the project name. (#3104) Signed-off-by: David Venable --- .../data-prepper-log-analytics-basic-grok-e2e-tests.yml | 2 +- .../data-prepper-peer-forwarder-local-node-e2e-tests.yml | 2 +- .../data-prepper-peer-forwarder-static-e2e-tests.yml | 2 +- ...per-trace-analytics-raw-span-compatibility-e2e-tests.yml | 2 +- .../data-prepper-trace-analytics-raw-span-e2e-tests.yml | 2 +- ...er-trace-analytics-raw-span-peer-forwarder-e2e-tests.yml | 2 +- .../data-prepper-trace-analytics-service-map-e2e-tests.yml | 2 +- .github/workflows/gradle.yml | 2 +- .../opensearch-sink-opendistro-integration-tests.yml | 2 +- .../opensearch-sink-opensearch-integration-tests.yml | 2 +- .github/workflows/performance-test-compile.yml | 2 +- .github/workflows/release.yml | 6 +++--- .github/workflows/staging-resources-cdk-check.yml | 2 +- .github/workflows/third-party-generate.yml | 2 +- 14 files changed, 16 insertions(+), 16 deletions(-) diff --git a/.github/workflows/data-prepper-log-analytics-basic-grok-e2e-tests.yml b/.github/workflows/data-prepper-log-analytics-basic-grok-e2e-tests.yml index ac4ec06a24..71405b8ee4 100644 --- a/.github/workflows/data-prepper-log-analytics-basic-grok-e2e-tests.yml +++ b/.github/workflows/data-prepper-log-analytics-basic-grok-e2e-tests.yml @@ -24,7 +24,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: 11 - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Run basic grok end-to-end tests with Gradle run: ./gradlew -PendToEndJavaVersion=${{ matrix.java }} :e2e-test:log:${{ matrix.test }} diff --git a/.github/workflows/data-prepper-peer-forwarder-local-node-e2e-tests.yml b/.github/workflows/data-prepper-peer-forwarder-local-node-e2e-tests.yml index 90952d05fe..a0f7ece568 100644 --- a/.github/workflows/data-prepper-peer-forwarder-local-node-e2e-tests.yml +++ b/.github/workflows/data-prepper-peer-forwarder-local-node-e2e-tests.yml @@ -23,7 +23,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: 11 - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Run raw-span latest release compatibility end-to-end tests with Gradle run: ./gradlew -PendToEndJavaVersion=${{ matrix.java }} :e2e-test:peerforwarder:localAggregateEndToEndTest diff --git a/.github/workflows/data-prepper-peer-forwarder-static-e2e-tests.yml b/.github/workflows/data-prepper-peer-forwarder-static-e2e-tests.yml index 2984febd2d..06623cba3a 100644 --- a/.github/workflows/data-prepper-peer-forwarder-static-e2e-tests.yml +++ b/.github/workflows/data-prepper-peer-forwarder-static-e2e-tests.yml @@ -24,7 +24,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: 11 - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Run raw-span latest release compatibility end-to-end tests with Gradle run: ./gradlew -PendToEndJavaVersion=${{ matrix.java }} :e2e-test:peerforwarder:${{ matrix.test }} diff --git a/.github/workflows/data-prepper-trace-analytics-raw-span-compatibility-e2e-tests.yml b/.github/workflows/data-prepper-trace-analytics-raw-span-compatibility-e2e-tests.yml index cb8501b3d3..4a2f8bd345 100644 --- a/.github/workflows/data-prepper-trace-analytics-raw-span-compatibility-e2e-tests.yml +++ b/.github/workflows/data-prepper-trace-analytics-raw-span-compatibility-e2e-tests.yml @@ -23,7 +23,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: 11 - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Run raw-span latest release compatibility end-to-end tests with Gradle run: ./gradlew -PendToEndJavaVersion=${{ matrix.java }} :e2e-test:trace:rawSpanLatestReleaseCompatibilityEndToEndTest diff --git a/.github/workflows/data-prepper-trace-analytics-raw-span-e2e-tests.yml b/.github/workflows/data-prepper-trace-analytics-raw-span-e2e-tests.yml index e51e2ab669..45e063c660 100644 --- a/.github/workflows/data-prepper-trace-analytics-raw-span-e2e-tests.yml +++ b/.github/workflows/data-prepper-trace-analytics-raw-span-e2e-tests.yml @@ -24,7 +24,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: 11 - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Run raw-span end-to-end tests with Gradle run: ./gradlew -PopenTelemetryVersion=${{ matrix.otelVersion }} -PendToEndJavaVersion=${{ matrix.java }} :e2e-test:trace:rawSpanEndToEndTest diff --git a/.github/workflows/data-prepper-trace-analytics-raw-span-peer-forwarder-e2e-tests.yml b/.github/workflows/data-prepper-trace-analytics-raw-span-peer-forwarder-e2e-tests.yml index 660628bb62..0dce35be61 100644 --- a/.github/workflows/data-prepper-trace-analytics-raw-span-peer-forwarder-e2e-tests.yml +++ b/.github/workflows/data-prepper-trace-analytics-raw-span-peer-forwarder-e2e-tests.yml @@ -23,7 +23,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: 11 - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Run raw-span latest release compatibility end-to-end tests with Gradle run: ./gradlew -PendToEndJavaVersion=${{ matrix.java }} :e2e-test:trace:rawSpanPeerForwarderEndToEndTest diff --git a/.github/workflows/data-prepper-trace-analytics-service-map-e2e-tests.yml b/.github/workflows/data-prepper-trace-analytics-service-map-e2e-tests.yml index 931e4946d6..14e2618546 100644 --- a/.github/workflows/data-prepper-trace-analytics-service-map-e2e-tests.yml +++ b/.github/workflows/data-prepper-trace-analytics-service-map-e2e-tests.yml @@ -23,7 +23,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: 11 - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Run service-map end-to-end tests with Gradle run: ./gradlew -PendToEndJavaVersion=${{ matrix.java }} :e2e-test:trace:serviceMapPeerForwarderEndToEndTest \ No newline at end of file diff --git a/.github/workflows/gradle.yml b/.github/workflows/gradle.yml index fc84bda7ab..df3d8c1a4c 100644 --- a/.github/workflows/gradle.yml +++ b/.github/workflows/gradle.yml @@ -22,7 +22,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: ${{ matrix.java }} - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Build with Gradle run: ./gradlew build diff --git a/.github/workflows/opensearch-sink-opendistro-integration-tests.yml b/.github/workflows/opensearch-sink-opendistro-integration-tests.yml index 45c477e12f..210f6b4539 100644 --- a/.github/workflows/opensearch-sink-opendistro-integration-tests.yml +++ b/.github/workflows/opensearch-sink-opendistro-integration-tests.yml @@ -30,7 +30,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: ${{ matrix.java }} - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Run Open Distro docker run: | diff --git a/.github/workflows/opensearch-sink-opensearch-integration-tests.yml b/.github/workflows/opensearch-sink-opensearch-integration-tests.yml index 5e34350303..50a8f00a3b 100644 --- a/.github/workflows/opensearch-sink-opensearch-integration-tests.yml +++ b/.github/workflows/opensearch-sink-opensearch-integration-tests.yml @@ -30,7 +30,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: ${{ matrix.java }} - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Run OpenSearch docker run: | diff --git a/.github/workflows/performance-test-compile.yml b/.github/workflows/performance-test-compile.yml index 3bc656e7ad..6c8ea1b9e4 100644 --- a/.github/workflows/performance-test-compile.yml +++ b/.github/workflows/performance-test-compile.yml @@ -28,7 +28,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: ${{ matrix.java }} - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Build performance tests with Gradle run: ./gradlew :performance-test:compileGatlingJava diff --git a/.github/workflows/release.yml b/.github/workflows/release.yml index 8310c911d8..c1c65fda49 100644 --- a/.github/workflows/release.yml +++ b/.github/workflows/release.yml @@ -16,7 +16,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: 11 - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Get Version run: grep '^version=' gradle.properties >> $GITHUB_ENV @@ -62,7 +62,7 @@ jobs: timeout-minutes: 30 steps: - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Get Version run: grep '^version=' gradle.properties >> $GITHUB_ENV @@ -88,7 +88,7 @@ jobs: timeout-minutes: 8 steps: - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Get Version run: grep '^version=' gradle.properties >> $GITHUB_ENV diff --git a/.github/workflows/staging-resources-cdk-check.yml b/.github/workflows/staging-resources-cdk-check.yml index 61d3938eca..61c2230eb3 100644 --- a/.github/workflows/staging-resources-cdk-check.yml +++ b/.github/workflows/staging-resources-cdk-check.yml @@ -26,7 +26,7 @@ jobs: with: node-version: '16' - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Install NPM Dependencies diff --git a/.github/workflows/third-party-generate.yml b/.github/workflows/third-party-generate.yml index 9c3d7ff828..98aba6e505 100644 --- a/.github/workflows/third-party-generate.yml +++ b/.github/workflows/third-party-generate.yml @@ -15,7 +15,7 @@ jobs: uses: actions/setup-java@v1 with: java-version: 11 - - name: Checkout Data-Prepper + - name: Checkout Data Prepper uses: actions/checkout@v2 - name: Generate Third Party Report From ce2d4dd0ed9979810f14774c12fcad685f5b9fa8 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> Date: Wed, 2 Aug 2023 14:43:33 -0700 Subject: [PATCH 3/7] GitHub-Issue#2778: Refactored Config to include Duration and ByteCount types (#3099) * Converted data types in the configuration to Data-Prepper types Signed-off-by: Marcos Gonzalez Mayedo * Added Duration to backOffTime Signed-off-by: Marcos Gonzalez Mayedo * Attempting to fix unused imports Signed-off-by: Marcos Gonzalez Mayedo --------- Signed-off-by: Marcos Gonzalez Mayedo Co-authored-by: Marcos Gonzalez Mayedo --- .../cloudwatch-logs/build.gradle | 1 + .../cloudwatch_logs/CloudWatchLogsSink.java | 2 +- .../config/ThresholdConfig.java | 48 +++++++++++-------- .../utils/CloudWatchLogsLimits.java | 22 ++++----- .../client/CloudWatchLogsServiceTest.java | 6 +-- .../config/ThresholdConfigTest.java | 47 +++++++++--------- .../utils/CloudWatchLogsLimitsTest.java | 44 +++++++++-------- 7 files changed, 91 insertions(+), 79 deletions(-) diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle index 0a3c815c0b..769b86b131 100644 --- a/data-prepper-plugins/cloudwatch-logs/build.gradle +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -18,6 +18,7 @@ dependencies { implementation 'software.amazon.awssdk:cloudwatchlogs' implementation 'org.apache.commons:commons-lang3:3.12.0' 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 testImplementation 'org.junit.jupiter:junit-jupiter' diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/CloudWatchLogsSink.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/CloudWatchLogsSink.java index 0127a4fadb..f17d79c7af 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/CloudWatchLogsSink.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/CloudWatchLogsSink.java @@ -49,7 +49,7 @@ public CloudWatchLogsSink(final PluginSetting pluginSetting, CloudWatchLogsMetrics cloudWatchLogsMetrics = new CloudWatchLogsMetrics(pluginMetrics); CloudWatchLogsLimits cloudWatchLogsLimits = new CloudWatchLogsLimits(thresholdConfig.getBatchSize(), thresholdConfig.getMaxEventSizeBytes(), - thresholdConfig.getMaxRequestSize(),thresholdConfig.getLogSendInterval()); + thresholdConfig.getMaxRequestSizeBytes(),thresholdConfig.getLogSendInterval()); CloudWatchLogsClient cloudWatchLogsClient = CloudWatchLogsClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/config/ThresholdConfig.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/config/ThresholdConfig.java index 8b974cb81b..53932cb6a4 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/config/ThresholdConfig.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/config/ThresholdConfig.java @@ -7,6 +7,12 @@ import com.fasterxml.jackson.annotation.JsonProperty; 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 java.time.temporal.ChronoUnit; /** * The threshold config holds the different configurations for @@ -15,12 +21,11 @@ */ public class ThresholdConfig { public static final int DEFAULT_BATCH_SIZE = 25; - public static final int DEFAULT_EVENT_SIZE = 256; - public static final int DEFAULT_SIZE_OF_REQUEST = 1048576; + public static final String DEFAULT_EVENT_SIZE = "256kb"; + public static final String DEFAULT_SIZE_OF_REQUEST = "1mb"; public static final int DEFAULT_RETRY_COUNT = 5; - public static final int DEFAULT_LOG_SEND_INTERVAL_TIME = 60; - public static final int DEFAULT_BACKOFF_TIME = 500; - public static final int BYTE_TO_KB_FACTOR = 1024; + public static final long DEFAULT_LOG_SEND_INTERVAL_TIME = 60; + public static final long DEFAULT_BACKOFF_TIME = 500; @JsonProperty("batch_size") @Size(min = 1, max = 10000, message = "batch_size amount should be between 1 to 10000") @@ -28,45 +33,46 @@ public class ThresholdConfig { @JsonProperty("max_event_size") @Size(min = 1, max = 256, message = "max_event_size amount should be between 1 to 256 kilobytes") - private int maxEventSize = DEFAULT_EVENT_SIZE; + private String maxEventSize = DEFAULT_EVENT_SIZE; @JsonProperty("max_request_size") - @Size(min = 1, max = 1048576, message = "max_batch_request_size amount should be between 1 and 1048576 bytes") - private int maxRequestSize = DEFAULT_SIZE_OF_REQUEST; + private String maxRequestSize = DEFAULT_SIZE_OF_REQUEST; @JsonProperty("retry_count") @Size(min = 1, max = 15, message = "retry_count amount should be between 1 and 15") private int retryCount = DEFAULT_RETRY_COUNT; @JsonProperty("log_send_interval") - @Size(min = 5, max = 300, message = "log_send_interval amount should be between 5 and 300 seconds") - private int logSendInterval = DEFAULT_LOG_SEND_INTERVAL_TIME; + @DurationMin(seconds = 60) + @DurationMax(seconds = 3600) + private Duration logSendInterval = Duration.ofSeconds(DEFAULT_LOG_SEND_INTERVAL_TIME); @JsonProperty("back_off_time") - @Size(min = 500, max = 1000, message = "back_off_time amount should be between 500 and 1000 milliseconds") - private int backOffTime = DEFAULT_BACKOFF_TIME; + @DurationMin(millis = 500) + @DurationMax(millis = 1000) + private Duration backOffTime = Duration.ofMillis(DEFAULT_BACKOFF_TIME); public int getBatchSize() { return batchSize; } - public int getMaxEventSizeBytes() { - return maxEventSize * BYTE_TO_KB_FACTOR; + public long getMaxEventSizeBytes() { + return ByteCount.parse(maxEventSize).getBytes(); } - public int getMaxRequestSize() { - return maxRequestSize; + public long getMaxRequestSizeBytes() { + return ByteCount.parse(maxRequestSize).getBytes(); } public int getRetryCount() { return retryCount; } - public int getLogSendInterval() { - return logSendInterval; + public long getLogSendInterval() { + return logSendInterval.getSeconds(); } - public int getBackOffTime() { - return backOffTime; + public long getBackOffTime() { + return (backOffTime.get(ChronoUnit.NANOS) / 1000000) + (backOffTime.getSeconds() * 1000); } -} +} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/utils/CloudWatchLogsLimits.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/utils/CloudWatchLogsLimits.java index 073daba7fd..5fa00afaa6 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/utils/CloudWatchLogsLimits.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/utils/CloudWatchLogsLimits.java @@ -5,7 +5,7 @@ package org.opensearch.dataprepper.plugins.sink.cloudwatch_logs.utils; /** - * ThresholdCheck receives parameters for which to reference the + * CloudWatchLogsLimits receives parameters for which to reference the * limits of a buffer and CloudWatchLogsClient before making a * PutLogEvent request to AWS. */ @@ -13,11 +13,11 @@ public class CloudWatchLogsLimits { // Size of overhead for each log event message. See https://docs.aws.amazon.com/AmazonCloudWatchLogs/latest/APIReference/API_PutLogEvents.html public static final int APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE = 26; private final int maxBatchSize; - private final int maxEventSizeBytes; - private final int maxRequestSizeBytes; + private final long maxEventSizeBytes; + private final long maxRequestSizeBytes; private final long logSendInterval; - public CloudWatchLogsLimits(final int maxBatchSize, final int maxEventSizeBytes, final int maxRequestSizeBytes, final int logSendInterval) { + public CloudWatchLogsLimits(final int maxBatchSize, final long maxEventSizeBytes, final long maxRequestSizeBytes, final long logSendInterval) { this.maxBatchSize = maxBatchSize; this.maxEventSizeBytes = maxEventSizeBytes; this.maxRequestSizeBytes = maxRequestSizeBytes; @@ -31,8 +31,8 @@ public CloudWatchLogsLimits(final int maxBatchSize, final int maxEventSizeBytes, * @param batchSize size of batch in events. * @return boolean true if we exceed the threshold events or false otherwise. */ - public boolean isGreaterThanLimitReached(final long currentTime, final int currentRequestSize, final int batchSize) { - int bufferSizeWithOverhead = (currentRequestSize + ((batchSize) * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + public boolean isGreaterThanLimitReached(final long currentTime, final long currentRequestSize, final int batchSize) { + long bufferSizeWithOverhead = (currentRequestSize + ((long) (batchSize) * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); return (isGreaterThanBatchSize(batchSize) || isGreaterEqualToLogSendInterval(currentTime) || isGreaterThanMaxRequestSize(bufferSizeWithOverhead)); } @@ -43,8 +43,8 @@ public boolean isGreaterThanLimitReached(final long currentTime, final int curre * @param batchSize size of batch in events. * @return boolean true if we equal the threshold events or false otherwise. */ - public boolean isEqualToLimitReached(final int currentRequestSize, final int batchSize) { - int bufferSizeWithOverhead = (currentRequestSize + ((batchSize) * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + public boolean isEqualToLimitReached(final long currentRequestSize, final int batchSize) { + long bufferSizeWithOverhead = (currentRequestSize + ((long) (batchSize) * APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); return (isEqualBatchSize(batchSize) || isEqualMaxRequestSize(bufferSizeWithOverhead)); } @@ -63,7 +63,7 @@ private boolean isGreaterEqualToLogSendInterval(final long currentTimeSeconds) { * @param eventSize int denoting size of event. * @return boolean true if greater than MaxEventSize, false otherwise. */ - public boolean isGreaterThanMaxEventSize(final int eventSize) { + public boolean isGreaterThanMaxEventSize(final long eventSize) { return (eventSize + APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE) > maxEventSizeBytes; } @@ -72,7 +72,7 @@ public boolean isGreaterThanMaxEventSize(final int eventSize) { * @param currentRequestSize int denoting size of request(Sum of PutLogEvent messages). * @return boolean true if greater than Max request size, smaller otherwise. */ - private boolean isGreaterThanMaxRequestSize(final int currentRequestSize) { + private boolean isGreaterThanMaxRequestSize(final long currentRequestSize) { return currentRequestSize > maxRequestSizeBytes; } @@ -91,7 +91,7 @@ private boolean isGreaterThanBatchSize(final int batchSize) { * @param currentRequestSize int denoting size of request(Sum of PutLogEvent messages). * @return boolean true if equal Max request size, smaller otherwise. */ - private boolean isEqualMaxRequestSize(final int currentRequestSize) { + private boolean isEqualMaxRequestSize(final long currentRequestSize) { return currentRequestSize == maxRequestSizeBytes; } diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/client/CloudWatchLogsServiceTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/client/CloudWatchLogsServiceTest.java index f984a44d89..e8e416d53e 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/client/CloudWatchLogsServiceTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/client/CloudWatchLogsServiceTest.java @@ -48,7 +48,7 @@ void setUp() { thresholdConfig = new ThresholdConfig(); cloudWatchLogsLimits = new CloudWatchLogsLimits(thresholdConfig.getBatchSize(), thresholdConfig.getMaxEventSizeBytes(), - thresholdConfig.getMaxRequestSize(), thresholdConfig.getLogSendInterval()); + thresholdConfig.getMaxRequestSizeBytes(), thresholdConfig.getLogSendInterval()); mockClient = mock(CloudWatchLogsClient.class); mockMetrics = mock(CloudWatchLogsMetrics.class); @@ -85,7 +85,7 @@ Collection> getSampleRecordsCollection() { Collection> getSampleRecordsOfLargerSize() { final ArrayList> returnCollection = new ArrayList<>(); for (int i = 0; i < thresholdConfig.getBatchSize() * 2; i++) { - JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("a".repeat((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST/24))); + JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("a".repeat((int) (thresholdConfig.getMaxRequestSizeBytes()/24))); final EventHandle mockEventHandle = mock(EventHandle.class); mockJacksonEvent.setEventHandle(mockEventHandle); returnCollection.add(new Record<>(mockJacksonEvent)); @@ -97,7 +97,7 @@ Collection> getSampleRecordsOfLargerSize() { Collection> getSampleRecordsOfLimitSize() { final ArrayList> returnCollection = new ArrayList<>(); for (int i = 0; i < thresholdConfig.getBatchSize(); i++) { - JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage".repeat(ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.BYTE_TO_KB_FACTOR)); + JacksonEvent mockJacksonEvent = (JacksonEvent) JacksonEvent.fromMessage("testMessage".repeat((int) thresholdConfig.getMaxEventSizeBytes())); final EventHandle mockEventHandle = mock(EventHandle.class); mockJacksonEvent.setEventHandle(mockEventHandle); returnCollection.add(new Record<>(mockJacksonEvent)); diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/config/ThresholdConfigTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/config/ThresholdConfigTest.java index a636e68c33..506c9011ba 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/config/ThresholdConfigTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/config/ThresholdConfigTest.java @@ -10,7 +10,10 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; +import java.time.Duration; import java.util.Map; import static org.hamcrest.MatcherAssert.assertThat; @@ -31,8 +34,8 @@ void GIVEN_new_threshold_config_SHOULD_return_valid_default_values() { assertThat(thresholdConfig.getBackOffTime(), equalTo(ThresholdConfig.DEFAULT_BACKOFF_TIME)); assertThat(thresholdConfig.getRetryCount(), equalTo(ThresholdConfig.DEFAULT_RETRY_COUNT)); assertThat(thresholdConfig.getBatchSize(), equalTo(ThresholdConfig.DEFAULT_BATCH_SIZE)); - assertThat(thresholdConfig.getMaxEventSizeBytes(), equalTo(ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.BYTE_TO_KB_FACTOR)); - assertThat(thresholdConfig.getMaxRequestSize(), equalTo(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST)); + assertThat(thresholdConfig.getMaxEventSizeBytes(), equalTo(ByteCount.parse(ThresholdConfig.DEFAULT_EVENT_SIZE).getBytes())); + assertThat(thresholdConfig.getMaxRequestSizeBytes(), equalTo(ByteCount.parse(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST).getBytes())); assertThat(thresholdConfig.getLogSendInterval(), equalTo(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME)); } @@ -45,19 +48,19 @@ void GIVEN_deserialized_threshold_config_SHOULD_return_valid_batch_size(final in } @ParameterizedTest - @ValueSource(ints = {1, 10, 256}) - void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_event_size(final int max_event_size) { - final Map jsonMap = Map.of("max_event_size", max_event_size); - final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getMaxEventSizeBytes(), equalTo(max_event_size * ThresholdConfig.BYTE_TO_KB_FACTOR)); + @ValueSource(strings = {"1kb", "10kb", "256kb"}) + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_event_size(final String max_event_size) throws NoSuchFieldException, IllegalAccessException { + ThresholdConfig sampleThresholdConfig = new ThresholdConfig(); + ReflectivelySetField.setField(sampleThresholdConfig.getClass(), sampleThresholdConfig, "maxEventSize", max_event_size); + assertThat(sampleThresholdConfig.getMaxEventSizeBytes(), equalTo(ByteCount.parse(max_event_size).getBytes())); } @ParameterizedTest - @ValueSource(ints = {1, 100, 1048576}) - void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_request_size(final int max_batch_request_size) { - final Map jsonMap = Map.of("max_request_size", max_batch_request_size); - final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getMaxRequestSize(), equalTo(max_batch_request_size)); + @ValueSource(strings = {"1b", "100b", "1048576b"}) + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_request_size(final String max_batch_request_size) throws NoSuchFieldException, IllegalAccessException { + ThresholdConfig sampleThresholdConfig = new ThresholdConfig(); + ReflectivelySetField.setField(sampleThresholdConfig.getClass(), sampleThresholdConfig, "maxRequestSize", max_batch_request_size); + assertThat(sampleThresholdConfig.getMaxRequestSizeBytes(), equalTo(ByteCount.parse(max_batch_request_size).getBytes())); } @ParameterizedTest @@ -70,17 +73,17 @@ void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_retry_count(fin @ParameterizedTest @ValueSource(ints = {5, 10, 300}) - void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_log_send_interval(final int log_send_interval) { - final Map jsonMap = Map.of("log_send_interval", log_send_interval); - final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getLogSendInterval(), equalTo(log_send_interval)); + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_max_log_send_interval(final int log_send_interval) throws NoSuchFieldException, IllegalAccessException { + ThresholdConfig sampleThresholdConfig = new ThresholdConfig(); + ReflectivelySetField.setField(sampleThresholdConfig.getClass(), sampleThresholdConfig, "logSendInterval", Duration.ofSeconds(log_send_interval)); + assertThat(sampleThresholdConfig.getLogSendInterval(), equalTo(Duration.ofSeconds(log_send_interval).getSeconds())) ; } @ParameterizedTest - @ValueSource(ints = {0, 100, 5000}) - void GIVEN_deserialized_threshold_config_SHOULD_return_valid_back_off_time(final int back_off_time) { - final Map jsonMap = Map.of("back_off_time", back_off_time); - final ThresholdConfig thresholdConfigTest = objectMapper.convertValue(jsonMap, ThresholdConfig.class); - assertThat(thresholdConfigTest.getBackOffTime(), equalTo(back_off_time)); + @ValueSource(longs = {0, 500, 1000}) + void GIVEN_deserialized_threshold_config_SHOULD_return_valid_back_off_time(final long back_off_time) throws NoSuchFieldException, IllegalAccessException { + ThresholdConfig sampleThresholdConfig = new ThresholdConfig(); + ReflectivelySetField.setField(sampleThresholdConfig.getClass(), sampleThresholdConfig, "backOffTime", Duration.ofMillis(back_off_time)); + assertThat(sampleThresholdConfig.getBackOffTime(), equalTo(back_off_time)); } -} +} \ No newline at end of file diff --git a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/utils/CloudWatchLogsLimitsTest.java b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/utils/CloudWatchLogsLimitsTest.java index bae3b69bea..5370fd09fb 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/utils/CloudWatchLogsLimitsTest.java +++ b/data-prepper-plugins/cloudwatch-logs/src/test/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/utils/CloudWatchLogsLimitsTest.java @@ -14,130 +14,132 @@ class CloudWatchLogsLimitsTest { private static CloudWatchLogsLimits cloudWatchLogsLimits; + private static ThresholdConfig thresholdConfig; @BeforeAll static void setUp() { - cloudWatchLogsLimits = new CloudWatchLogsLimits(ThresholdConfig.DEFAULT_BATCH_SIZE, ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.BYTE_TO_KB_FACTOR, - ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); + thresholdConfig = new ThresholdConfig(); + cloudWatchLogsLimits = new CloudWatchLogsLimits(ThresholdConfig.DEFAULT_BATCH_SIZE, thresholdConfig.getMaxEventSizeBytes(), + thresholdConfig.getMaxRequestSizeBytes(), ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME); } @Test void GIVEN_greater_than_limit_method_WHEN_event_size_greater_than_max_event_size_THEN_return_true() { - boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(((ThresholdConfig.DEFAULT_EVENT_SIZE + 1) * ThresholdConfig.BYTE_TO_KB_FACTOR) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize((thresholdConfig.getMaxEventSizeBytes() + 1) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); assertTrue(isEventGreater); } @Test void GIVEN_greater_than_limit_method_WHEN_event_size_less_than_max_event_size_THEN_return_false() { - boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(((ThresholdConfig.DEFAULT_EVENT_SIZE - 1) * ThresholdConfig.BYTE_TO_KB_FACTOR) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize(((thresholdConfig.getMaxEventSizeBytes()) - 1) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); assertFalse(isEventGreater); } @Test void GIVEN_greater_than_limit_method_WHEN_event_size_equal_to_max_event_size_THEN_return_false() { - boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize((ThresholdConfig.DEFAULT_EVENT_SIZE * ThresholdConfig.BYTE_TO_KB_FACTOR) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + boolean isEventGreater = cloudWatchLogsLimits.isGreaterThanMaxEventSize((thresholdConfig.getMaxEventSizeBytes()) - CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); assertFalse(isEventGreater); } @Test void GIVEN_greater_than_limit_method_WHEN_log_send_interval_equal_to_max_log_send_interval_THEN_return_true() { - boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); + boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME, thresholdConfig.getMaxRequestSizeBytes(),ThresholdConfig.DEFAULT_BATCH_SIZE); assertTrue(thresholdMetTime); } @Test void SGIVEN_greater_than_limit_method_WHEN_log_send_interval_greater_than_max_log_send_interval_THEN_return_true() { - boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME + 1, ThresholdConfig.DEFAULT_SIZE_OF_REQUEST,ThresholdConfig.DEFAULT_BATCH_SIZE); + boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME + 1, thresholdConfig.getMaxRequestSizeBytes(),ThresholdConfig.DEFAULT_BATCH_SIZE); assertTrue(thresholdMetTime); } @Test void GIVEN_greater_than_limit_method_WHEN_log_send_interval_less_than_max_log_send_interval_THEN_return_false() { - int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + long validRequestSize = thresholdConfig.getMaxRequestSizeBytes() - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); boolean thresholdMetTime = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, validRequestSize ,ThresholdConfig.DEFAULT_BATCH_SIZE); assertFalse(thresholdMetTime); } @Test void GIVEN_greater_than_limit_method_WHEN_request_size_greater_than_max_request_size_THEN_return_true() { - int requestSizeWithoutOverhead = (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST + 1) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + long requestSizeWithoutOverhead = (thresholdConfig.getMaxRequestSizeBytes() + 1) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, requestSizeWithoutOverhead, ThresholdConfig.DEFAULT_BATCH_SIZE); assertTrue(thresholdMetRequestSize); } @Test void GIVEN_greater_than_limit_method_WHEN_request_size_equal_to_max_request_size_THEN_return_false() { - int requestSizeWithoutOverhead = (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + long requestSizeWithoutOverhead = (thresholdConfig.getMaxRequestSizeBytes()) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, requestSizeWithoutOverhead, ThresholdConfig.DEFAULT_BATCH_SIZE); assertFalse(thresholdMetRequestSize); } @Test void GIVEN_greater_than_limit_method_WHEN_request_size_less_than_max_request_size_THEN_return_false() { - int requestSizeWithoutOverhead = (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + long requestSizeWithoutOverhead = (thresholdConfig.getMaxRequestSizeBytes() - 1) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); boolean thresholdMetRequestSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME - 1, requestSizeWithoutOverhead, ThresholdConfig.DEFAULT_BATCH_SIZE); assertFalse(thresholdMetRequestSize); } @Test void GIVEN_greater_than_limit_method_WHEN_batch_size_greater_than_max_batch_size_THEN_return_true() { - int requestSizeWithoutOverhead = (ThresholdConfig.DEFAULT_SIZE_OF_REQUEST) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + long requestSizeWithoutOverhead = (thresholdConfig.getMaxRequestSizeBytes()) - ThresholdConfig.DEFAULT_BATCH_SIZE * (CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, requestSizeWithoutOverhead, ThresholdConfig.DEFAULT_BATCH_SIZE + 1); assertTrue(thresholdMetBatchSize); } @Test void GIVEN_greater_than_limit_method_WHEN_batch_size_equal_to_max_batch_size_THEN_return_false() { - int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + long validRequestSize = thresholdConfig.getMaxRequestSizeBytes() - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE); assertFalse(thresholdMetBatchSize); } @Test void GIVEN_greater_than_limit_method_WHEN_batch_size_less_than_max_batch_size_THEN_return_false() { - int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + long validRequestSize = thresholdConfig.getMaxRequestSizeBytes()- ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); boolean thresholdMetBatchSize = cloudWatchLogsLimits.isGreaterThanLimitReached(ThresholdConfig.DEFAULT_LOG_SEND_INTERVAL_TIME -1, validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); assertFalse(thresholdMetBatchSize); } @Test void GIVEN_equal_to_limit_method_WHEN_request_size_equal_to_max_batch_size_THEN_return_true() { - int validRequestSize = ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); + long validRequestSize = thresholdConfig.getMaxRequestSizeBytes() - ((ThresholdConfig.DEFAULT_BATCH_SIZE) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE); boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE); assertTrue(thresholdMetRequestSize); } @Test void GIVEN_equal_to_limit_method_WHEN_request_size_greater_than_max_batch_size_THEN_return_false() { - int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST + 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + long validRequestSize = ((thresholdConfig.getMaxRequestSizeBytes() + 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); assertFalse(thresholdMetRequestSize); } @Test void GIVEN_equal_to_limit_method_WHEN_request_size_less_than_max_batch_size_THEN_return_false() { - int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + long validRequestSize = ((thresholdConfig.getMaxRequestSizeBytes() - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); boolean thresholdMetRequestSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); assertFalse(thresholdMetRequestSize); } @Test void GIVEN_equal_to_limit_method_WHEN_batch_size_equal_to_max_batch_size_THEN_return_true() { - int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); - boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(ThresholdConfig.DEFAULT_SIZE_OF_REQUEST, ThresholdConfig.DEFAULT_BATCH_SIZE); + long validRequestSize = ((thresholdConfig.getMaxRequestSizeBytes() - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(thresholdConfig.getMaxRequestSizeBytes(), ThresholdConfig.DEFAULT_BATCH_SIZE); assertTrue(thresholdMetBatchSize); } @Test void GIVEN_equal_to_limit_method_WHEN_batch_size_greater_than_max_batch_size_THEN_return_false() { - int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + long validRequestSize = ((thresholdConfig.getMaxRequestSizeBytes() - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE + 1); assertFalse(thresholdMetBatchSize); } @Test void GIVEN_equal_to_limit_method_WHEN_batch_size_less_than_max_batch_size_THEN_return_false() { - int validRequestSize = ((ThresholdConfig.DEFAULT_SIZE_OF_REQUEST - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); + long validRequestSize = ((thresholdConfig.getMaxRequestSizeBytes() - 1) - ((ThresholdConfig.DEFAULT_BATCH_SIZE - 1) * CloudWatchLogsLimits.APPROXIMATE_LOG_EVENT_OVERHEAD_SIZE)); boolean thresholdMetBatchSize = cloudWatchLogsLimits.isEqualToLimitReached(validRequestSize, ThresholdConfig.DEFAULT_BATCH_SIZE - 1); assertFalse(thresholdMetBatchSize); } From cbc8890cc3627cdcefa8df3c68e1bd60374ea994 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 2 Aug 2023 17:13:11 -0500 Subject: [PATCH 4/7] Bump com.opencsv:opencsv from 5.7.1 to 5.8 (#3097) Bumps com.opencsv:opencsv from 5.7.1 to 5.8. --- updated-dependencies: - dependency-name: com.opencsv:opencsv 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/csv-processor/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/data-prepper-plugins/csv-processor/build.gradle b/data-prepper-plugins/csv-processor/build.gradle index 48f2e6ab69..9a72135a14 100644 --- a/data-prepper-plugins/csv-processor/build.gradle +++ b/data-prepper-plugins/csv-processor/build.gradle @@ -18,7 +18,7 @@ dependencies { testImplementation project(':data-prepper-plugins:log-generator-source') testImplementation project(':data-prepper-test-common') testImplementation project(':data-prepper-plugins:common') - implementation 'com.opencsv:opencsv:5.7.1' + implementation 'com.opencsv:opencsv:5.8' } test { From c753ba6b0ed5659ab6a21bd6c1299ebd9c08ad66 Mon Sep 17 00:00:00 2001 From: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> Date: Wed, 2 Aug 2023 15:19:06 -0700 Subject: [PATCH 5/7] GitHub-Issue#2778: Added README for CloudWatch Logs Sink (#3101) * Adding README Signed-off-by: Marcos Gonzalez Mayedo * Added README to sink Signed-off-by: Marcos Gonzalez Mayedo * Added PluginFunctionality to README Signed-off-by: Marcos Gonzalez Mayedo * Update data-prepper-plugins/cloudwatch-logs/README.md Co-authored-by: Mark Kuhn Signed-off-by: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> * Update data-prepper-plugins/cloudwatch-logs/README.md Co-authored-by: Mark Kuhn Signed-off-by: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> * Added fixes to configuration in README and example id Signed-off-by: Marcos Gonzalez Mayedo * Added extra info in plugin functionality Signed-off-by: Marcos Gonzalez Mayedo --------- Signed-off-by: Marcos Gonzalez Mayedo Signed-off-by: Marcos Gonzalez Mayedo <95880281+MaGonzalMayedo@users.noreply.github.com> Co-authored-by: Marcos Gonzalez Mayedo Co-authored-by: Mark Kuhn --- .../cloudwatch-logs/README.md | 83 +++++++++++++++++++ 1 file changed, 83 insertions(+) create mode 100644 data-prepper-plugins/cloudwatch-logs/README.md diff --git a/data-prepper-plugins/cloudwatch-logs/README.md b/data-prepper-plugins/cloudwatch-logs/README.md new file mode 100644 index 0000000000..754695221e --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/README.md @@ -0,0 +1,83 @@ +# CloudWatch Logs Sink + +This Data Prepper sink allows the sending of log data to CloudWatch Logs via a CloudWatchLogsClient. + +## Usages + +The CloudWatch Logs sink should be configured as part of Data Prepper pipeline yaml file. + +## Configuration Options + +``` +pipeline: + ... + sink: + - cloudwatch_logs: + aws: + region: us-east-1 + sts_role_arn: arn:aws:iam::123456789012:role/Data-Prepper + sts_header_overrides: + custom_header: ... + custom_header2: ... + ... + sts_external_id: 123ABC + log_group: sample_group + log_stream: sample_stream + buffer_type: in_memory + threshold: + batch_size: 10000 + max_event_size: 256kb + max_request_size: 1mb + retry_count: 5 + back_off_time: 500ms +``` + +## AWS Configuration + +- `region` (Optional) : A string representing the AWS region to use for credentials. Defaults to [standard SDK behavior to determine the region](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/region-selection.html). + +- `sts_role_arn` (Optional) : A string representing AWS STS role to assume for requests to CloudWatchLogs. which will use the [standard SDK behavior for credentials](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/credentials.html). + +- `sts_header_overrides` (Optional) : A string map representing different custom headers that can be added. + +- `sts_external_id` (Optional) : A string representing the external ID to attach to AssumeRole requests. Referenced here: [how to use external ID](https://docs.aws.amazon.com/IAM/latest/UserGuide/id_roles_create_for-user_externalid.html) + +## Threshold Configuration + +- `batch_size` (Optional) : An integer value that indicates how many events we hold until we make a call to CloudWatch Logs. Defaults to 25. (Min = 1, Max = 10000) + +- `max_event_size` (Optional) : A string representing the max size in bytes of the allowed events. Defaults to "256kb". (Min = "1b", Max = "256kb") + +- `max_request_size` (Optional) : A string representing the count or size of bytes we hold until we make a call to CloudWatch Logs. Default is "1mb". (Min = "1b", Max = "1mb") + +- `retry_count` (Optional) : An integer value that indicates the number of retries we make when encountering errors sending logs to CloudWatch Logs. Defaults to 5. (Min = 1, Max = 15) + +- `log_send_interval` (Optional) : A string representing the amount of time in seconds between making requests. Defaults to "60s". (Min = "5s", Max = "300s") + +- `back_off_time` (Optional) : A string representing the amount of time in milliseconds between errored transmission re-attempts. Defaults to "500ms". (Min = "500ms", Max = "1000ms") + +## Buffer Type Configuration + +- `buffer_type` (Optional) : A string representing the type of buffer to use to hold onto events. Currently only supports `in_memory`. + +## Plugin Functionality +The cloudwatch_logs sink plugin uses credentials to establish a client to CloudWatch Logs. It currently uses the current system timestamp for publishing and implements an exponential back off strategy +for retransmission. + +The cloudwatch_logs sink plugin also adds an overhead of 26 bytes added to each event message. This is done by the AWS SDK when formatting the API call to CloudWatch Logs. This must be considered when setting custom +threshold parameters. +## Metrics + +### Counters + +* `cloudWatchLogsEventsSucceeded` - The number of log events successfully published to CloudWatch Logs. +* `cloudWatchLogsEventsFailed` - The number of log events failed while publishing to CloudWatch Logs. +* `cloudWatchLogsRequestsSucceeded` - The number of log requests successfully made to CloudWatch Logs. +* `cloudWatchLogsRequestsFailed` - The number of log requests failed to reach CloudWatch Logs. + +## Developer Guide + +This plugin is compatible with Java 11. See below + +- [CONTRIBUTING](https://github.com/opensearch-project/data-prepper/blob/main/CONTRIBUTING.md) +- [monitoring](https://github.com/opensearch-project/data-prepper/blob/main/docs/monitoring.md) From 88210059e68e318249099c8e288ac794046d579a Mon Sep 17 00:00:00 2001 From: Qi Chen Date: Thu, 3 Aug 2023 11:59:06 -0500 Subject: [PATCH 6/7] ENH: support custom index template for ES6 in opensearch sink (#3061) Signed-off-by: George Chen --- data-prepper-plugins/opensearch/README.md | 4 +- .../sink/opensearch/OpenSearchSink.java | 7 +- .../index/ComposableIndexTemplate.java | 49 ++++ .../ComposableIndexTemplateStrategy.java | 135 +--------- .../index/ComposableTemplateAPIWrapper.java | 89 +++++++ .../index/Es6IndexTemplateAPIWrapper.java | 126 +++++++++ .../opensearch/index/IndexConfiguration.java | 11 +- .../index/IndexTemplateAPIWrapper.java | 23 ++ .../index/IndexTemplateAPIWrapperFactory.java | 17 ++ .../opensearch/index/LegacyIndexTemplate.java | 48 ++++ .../OpenSearchLegacyTemplateAPIWrapper.java | 67 +++++ .../sink/opensearch/index/TemplateType.java | 10 +- .../opensearch/index/V1TemplateStrategy.java | 115 +-------- .../ComposableIndexTemplateStrategyTest.java | 233 ++++------------- .../ComposableTemplateAPIWrapperTest.java | 240 ++++++++++++++++++ .../index/Es6IndexTemplateAPIWrapperTest.java | 151 +++++++++++ .../index/IndexConfigurationTests.java | 4 +- .../IndexTemplateAPIWrapperFactoryTest.java | 44 ++++ ...penSearchLegacyTemplateAPIWrapperTest.java | 139 ++++++++++ .../opensearch/index/TemplateTypeTest.java | 9 +- .../index/V1TemplateStrategyTest.java | 129 +++------- 21 files changed, 1136 insertions(+), 514 deletions(-) create mode 100644 data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplate.java create mode 100644 data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableTemplateAPIWrapper.java create mode 100644 data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/Es6IndexTemplateAPIWrapper.java create mode 100644 data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapper.java create mode 100644 data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapperFactory.java create mode 100644 data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/LegacyIndexTemplate.java create mode 100644 data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/OpenSearchLegacyTemplateAPIWrapper.java create mode 100644 data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableTemplateAPIWrapperTest.java create mode 100644 data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/Es6IndexTemplateAPIWrapperTest.java create mode 100644 data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapperFactoryTest.java create mode 100644 data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/OpenSearchLegacyTemplateAPIWrapperTest.java diff --git a/data-prepper-plugins/opensearch/README.md b/data-prepper-plugins/opensearch/README.md index 66c75f6f57..f40f1ce0b0 100644 --- a/data-prepper-plugins/opensearch/README.md +++ b/data-prepper-plugins/opensearch/README.md @@ -93,7 +93,7 @@ Default is null. - `proxy`(optional): A String of the address of a forward HTTP proxy. The format is like ":\". Examples: "example.com:8100", "http://example.com:8100", "112.112.112.112:8100". Note: port number cannot be omitted. -- `index_type` (optional): a String from the list [`custom`, `trace-analytics-raw`, `trace-analytics-service-map`, `management_disabled`], which represents an index type. Defaults to `custom` if `serverless` is `false` in [AWS Configuration](#aws_configuration) and `distribution_version` is `null`, otherwise defaults to `management_disabled`. This index_type instructs Sink plugin what type of data it is handling. +- `index_type` (optional): a String from the list [`custom`, `trace-analytics-raw`, `trace-analytics-service-map`, `management_disabled`], which represents an index type. Defaults to `custom` if `serverless` is `false` in [AWS Configuration](#aws_configuration), otherwise defaults to `management_disabled`. This index_type instructs Sink plugin what type of data it is handling. - `enable_request_compression` (optional): A boolean that enables or disables request compression when sending requests to OpenSearch. Default is true. @@ -133,7 +133,7 @@ Default is null. * This index name can also be a plain string plus a date-time pattern as a suffix, such as `application-%{yyyy.MM.dd}`, `my-index-name-%{yyyy.MM.dd.HH}`. When OpenSearch Sink is sending data to OpenSearch, the date-time pattern will be replaced by actual UTC time. The pattern supports all the symbols that represent one hour or above and are listed in [Java DateTimeFormatter](https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html). For example, with an index pattern like `my-index-name-%{yyyy.MM.dd}`, a new index is created for each day such as `my-index-name-2022.01.25`. For another example, with an index pattern like `my-index-name-%{yyyy.MM.dd.HH}`, a new index is created for each hour such as `my-index-name-2022.01.25.13`. * This index name can also be a formatted string (with or without date-time pattern suffix), such as `my-${index}-name`. When OpenSearchSink is sending data to OpenSearch, the format portion "${index}" will be replaced by it's value in the event that is being processed. The format may also be like "${index1/index2/index3}" in which case the field "index1/index2/index3" is searched in the event and replaced by its value. - Additionally, the formatted string can include expressions to evaluate to format the index name. For example, `my-${index}-${getMetadata(\"some_metadata_key\")}-name` will inject both the `index` value from the Event, as well as the value of `some_metadata_key` from the Event metadata to construct the index name. -- `template_type`(optional): Defines what type of OpenSearch template to use. The available options are `v1` and `index-template`. The default value is `v1`, which uses the original OpenSearch templates available at the `_template` API endpoints. Select `index-template` to use composable index templates which are available at OpenSearch's `_index_template` endpoint. +- `template_type`(optional): Defines what type of OpenSearch template to use. The available options are `v1` and `index-template`. The default value is `v1`, which uses the original OpenSearch templates available at the `_template` API endpoints. Select `index-template` to use composable index templates which are available at OpenSearch's `_index_template` endpoint. Note: when `distribution_version` is `es6`, `template_type` is enforced into `v1`. - `template_file`(optional): A json file path or AWS S3 URI to be read as index template for custom data ingestion. The json file content should be the json value of `"template"` key in the json content of OpenSearch [Index templates API](https://opensearch.org/docs/latest/opensearch/index-templates/), 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 1464e011e0..ccd430c982 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 @@ -52,6 +52,8 @@ import org.opensearch.dataprepper.plugins.sink.opensearch.index.DocumentBuilder; import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexManager; import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexManagerFactory; +import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplateAPIWrapper; +import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplateAPIWrapperFactory; import org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexType; import org.opensearch.dataprepper.plugins.sink.opensearch.index.TemplateStrategy; import org.slf4j.Logger; @@ -190,7 +192,10 @@ private void doInitializeInternal() throws IOException { restHighLevelClient = openSearchSinkConfig.getConnectionConfiguration().createClient(awsCredentialsSupplier); openSearchClient = openSearchSinkConfig.getConnectionConfiguration().createOpenSearchClient(restHighLevelClient, awsCredentialsSupplier); configuredIndexAlias = openSearchSinkConfig.getIndexConfiguration().getIndexAlias(); - final TemplateStrategy templateStrategy = openSearchSinkConfig.getIndexConfiguration().getTemplateType().createTemplateStrategy(openSearchClient); + final IndexTemplateAPIWrapper indexTemplateAPIWrapper = IndexTemplateAPIWrapperFactory.getWrapper( + openSearchSinkConfig.getIndexConfiguration(), openSearchClient); + final TemplateStrategy templateStrategy = openSearchSinkConfig.getIndexConfiguration().getTemplateType() + .createTemplateStrategy(indexTemplateAPIWrapper); indexManager = indexManagerFactory.getIndexManager(indexType, openSearchClient, restHighLevelClient, openSearchSinkConfig, templateStrategy, configuredIndexAlias); final String dlqFile = openSearchSinkConfig.getRetryConfiguration().getDlqFile(); diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplate.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplate.java new file mode 100644 index 0000000000..f7a42d278b --- /dev/null +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplate.java @@ -0,0 +1,49 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class ComposableIndexTemplate implements IndexTemplate { + + private final Map indexTemplateMap; + private String name; + + public ComposableIndexTemplate(final Map indexTemplateMap) { + this.indexTemplateMap = new HashMap<>(indexTemplateMap); + } + + @Override + public void setTemplateName(final String name) { + this.name = name; + + } + + @Override + public void setIndexPatterns(final List indexPatterns) { + indexTemplateMap.put("index_patterns", indexPatterns); + } + + @Override + public void putCustomSetting(final String name, final Object value) { + + } + + @Override + public Optional getVersion() { + if(!indexTemplateMap.containsKey("version")) + return Optional.empty(); + final Number version = (Number) indexTemplateMap.get("version"); + return Optional.of(version.longValue()); + } + + public Map getIndexTemplateMap() { + return Collections.unmodifiableMap(indexTemplateMap); + } + + public String getName() { + return name; + } +} diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplateStrategy.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplateStrategy.java index 62cba5a035..0ea3848f33 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplateStrategy.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplateStrategy.java @@ -5,25 +5,10 @@ package org.opensearch.dataprepper.plugins.sink.opensearch.index; -import com.fasterxml.jackson.databind.ObjectMapper; -import jakarta.json.stream.JsonParser; -import org.opensearch.client.json.JsonpDeserializer; -import org.opensearch.client.json.JsonpMapper; -import org.opensearch.client.json.ObjectBuilderDeserializer; -import org.opensearch.client.json.ObjectDeserializer; -import org.opensearch.client.opensearch.OpenSearchClient; -import org.opensearch.client.opensearch.indices.ExistsIndexTemplateRequest; -import org.opensearch.client.opensearch.indices.GetIndexTemplateRequest; import org.opensearch.client.opensearch.indices.GetIndexTemplateResponse; -import org.opensearch.client.opensearch.indices.PutIndexTemplateRequest; import org.opensearch.client.opensearch.indices.get_index_template.IndexTemplateItem; -import org.opensearch.client.opensearch.indices.put_index_template.IndexTemplateMapping; -import org.opensearch.client.transport.endpoints.BooleanResponse; -import java.io.ByteArrayInputStream; import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; @@ -32,18 +17,24 @@ * A {@link TemplateStrategy} for the OpenSearch index template. */ class ComposableIndexTemplateStrategy implements TemplateStrategy { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private final OpenSearchClient openSearchClient; + private final IndexTemplateAPIWrapper indexTemplateAPIWrapper; - public ComposableIndexTemplateStrategy(final OpenSearchClient openSearchClient) { - this.openSearchClient = openSearchClient; + public ComposableIndexTemplateStrategy(final IndexTemplateAPIWrapper indexTemplateAPIWrapper) { + this.indexTemplateAPIWrapper = indexTemplateAPIWrapper; } @Override public Optional getExistingTemplateVersion(final String templateName) throws IOException { - return getIndexTemplate(templateName) - .map(IndexTemplateItem::indexTemplate) - .map(indexTemplate -> indexTemplate.version()); + return indexTemplateAPIWrapper.getTemplate(templateName) + .map(getIndexTemplateResponse -> { + final List indexTemplateItems = getIndexTemplateResponse.indexTemplates(); + if (indexTemplateItems.size() == 1) { + return indexTemplateItems.stream().findFirst().get().indexTemplate().version(); + } else { + throw new RuntimeException(String.format("Found zero or multiple index templates result when querying for %s", + templateName)); + } + }); } @Override @@ -53,104 +44,6 @@ public IndexTemplate createIndexTemplate(final Map templateMap) @Override public void createTemplate(final IndexTemplate indexTemplate) throws IOException { - if(!(indexTemplate instanceof ComposableIndexTemplate)) { - throw new IllegalArgumentException("Unexpected indexTemplate provided to createTemplate."); - } - - final ComposableIndexTemplate composableIndexTemplate = (ComposableIndexTemplate) indexTemplate; - - final Map templateMapping = composableIndexTemplate.indexTemplateMap; - - final String indexTemplateString = OBJECT_MAPPER.writeValueAsString(templateMapping); - - final ByteArrayInputStream byteIn = new ByteArrayInputStream( - indexTemplateString.getBytes(StandardCharsets.UTF_8)); - final JsonpMapper mapper = openSearchClient._transport().jsonpMapper(); - final JsonParser parser = mapper.jsonProvider().createParser(byteIn); - - final PutIndexTemplateRequest putIndexTemplateRequest = PutIndexTemplateRequestDeserializer.getJsonpDeserializer(composableIndexTemplate.name) - .deserialize(parser, mapper); - - openSearchClient.indices().putIndexTemplate(putIndexTemplateRequest); - - } - - private Optional getIndexTemplate(final String indexTemplateName) throws IOException { - final ExistsIndexTemplateRequest existsRequest = new ExistsIndexTemplateRequest.Builder() - .name(indexTemplateName) - .build(); - final BooleanResponse existsResponse = openSearchClient.indices().existsIndexTemplate(existsRequest); - - if (!existsResponse.value()) { - return Optional.empty(); - } - - final GetIndexTemplateRequest getRequest = new GetIndexTemplateRequest.Builder() - .name(indexTemplateName) - .build(); - final GetIndexTemplateResponse indexTemplateResponse = openSearchClient.indices().getIndexTemplate(getRequest); - - final List indexTemplateItems = indexTemplateResponse.indexTemplates(); - if (indexTemplateItems.size() == 1) { - return indexTemplateItems.stream().findFirst(); - } else { - throw new RuntimeException(String.format("Found zero or multiple index templates result when querying for %s", - indexTemplateName)); - } - } - - static class ComposableIndexTemplate implements IndexTemplate { - - private final Map indexTemplateMap; - private String name; - - private ComposableIndexTemplate(final Map indexTemplateMap) { - this.indexTemplateMap = new HashMap<>(indexTemplateMap); - } - - @Override - public void setTemplateName(final String name) { - this.name = name; - - } - - @Override - public void setIndexPatterns(final List indexPatterns) { - indexTemplateMap.put("index_patterns", indexPatterns); - } - - @Override - public void putCustomSetting(final String name, final Object value) { - - } - - @Override - public Optional getVersion() { - if(!indexTemplateMap.containsKey("version")) - return Optional.empty(); - final Number version = (Number) indexTemplateMap.get("version"); - return Optional.of(version.longValue()); - } - } - - private static class PutIndexTemplateRequestDeserializer { - private static void setupPutIndexTemplateRequestDeserializer(final ObjectDeserializer objectDeserializer) { - - objectDeserializer.add(PutIndexTemplateRequest.Builder::name, JsonpDeserializer.stringDeserializer(), "name"); - objectDeserializer.add(PutIndexTemplateRequest.Builder::indexPatterns, JsonpDeserializer.arrayDeserializer(JsonpDeserializer.stringDeserializer()), - "index_patterns"); - objectDeserializer.add(PutIndexTemplateRequest.Builder::version, JsonpDeserializer.longDeserializer(), "version"); - objectDeserializer.add(PutIndexTemplateRequest.Builder::priority, JsonpDeserializer.integerDeserializer(), "priority"); - objectDeserializer.add(PutIndexTemplateRequest.Builder::composedOf, JsonpDeserializer.arrayDeserializer(JsonpDeserializer.stringDeserializer()), - "composed_of"); - objectDeserializer.add(PutIndexTemplateRequest.Builder::template, IndexTemplateMapping._DESERIALIZER, "template"); - } - - static JsonpDeserializer getJsonpDeserializer(final String name) { - return ObjectBuilderDeserializer - .lazy( - () -> new PutIndexTemplateRequest.Builder().name(name), - PutIndexTemplateRequestDeserializer::setupPutIndexTemplateRequestDeserializer); - } + indexTemplateAPIWrapper.putTemplate(indexTemplate); } } diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableTemplateAPIWrapper.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableTemplateAPIWrapper.java new file mode 100644 index 0000000000..1b78cb9da5 --- /dev/null +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableTemplateAPIWrapper.java @@ -0,0 +1,89 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +import com.fasterxml.jackson.databind.ObjectMapper; +import jakarta.json.stream.JsonParser; +import org.opensearch.client.json.JsonpDeserializer; +import org.opensearch.client.json.JsonpMapper; +import org.opensearch.client.json.ObjectBuilderDeserializer; +import org.opensearch.client.json.ObjectDeserializer; +import org.opensearch.client.opensearch.OpenSearchClient; +import org.opensearch.client.opensearch.indices.ExistsIndexTemplateRequest; +import org.opensearch.client.opensearch.indices.GetIndexTemplateRequest; +import org.opensearch.client.opensearch.indices.GetIndexTemplateResponse; +import org.opensearch.client.opensearch.indices.PutIndexTemplateRequest; +import org.opensearch.client.opensearch.indices.put_index_template.IndexTemplateMapping; +import org.opensearch.client.transport.endpoints.BooleanResponse; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Optional; + +public class ComposableTemplateAPIWrapper implements IndexTemplateAPIWrapper { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private final OpenSearchClient openSearchClient; + + public ComposableTemplateAPIWrapper(final OpenSearchClient openSearchClient) { + this.openSearchClient = openSearchClient; + } + + @Override + public void putTemplate(final IndexTemplate indexTemplate) throws IOException { + if(!(indexTemplate instanceof ComposableIndexTemplate)) { + throw new IllegalArgumentException("Unexpected indexTemplate provided to createTemplate."); + } + + final ComposableIndexTemplate composableIndexTemplate = (ComposableIndexTemplate) indexTemplate; + final String indexTemplateString = OBJECT_MAPPER.writeValueAsString( + composableIndexTemplate.getIndexTemplateMap()); + + final ByteArrayInputStream byteIn = new ByteArrayInputStream( + indexTemplateString.getBytes(StandardCharsets.UTF_8)); + final JsonpMapper mapper = openSearchClient._transport().jsonpMapper(); + final JsonParser parser = mapper.jsonProvider().createParser(byteIn); + + final PutIndexTemplateRequest putIndexTemplateRequest = PutIndexTemplateRequestDeserializer + .getJsonpDeserializer(composableIndexTemplate.getName()) + .deserialize(parser, mapper); + + openSearchClient.indices().putIndexTemplate(putIndexTemplateRequest); + } + + @Override + public Optional getTemplate(final String indexTemplateName) throws IOException { + final ExistsIndexTemplateRequest existsRequest = new ExistsIndexTemplateRequest.Builder() + .name(indexTemplateName) + .build(); + final BooleanResponse existsResponse = openSearchClient.indices().existsIndexTemplate(existsRequest); + + if (!existsResponse.value()) { + return Optional.empty(); + } + + final GetIndexTemplateRequest getRequest = new GetIndexTemplateRequest.Builder() + .name(indexTemplateName) + .build(); + return Optional.of(openSearchClient.indices().getIndexTemplate(getRequest)); + } + + private static class PutIndexTemplateRequestDeserializer { + private static void setupPutIndexTemplateRequestDeserializer(final ObjectDeserializer objectDeserializer) { + + objectDeserializer.add(PutIndexTemplateRequest.Builder::name, JsonpDeserializer.stringDeserializer(), "name"); + objectDeserializer.add(PutIndexTemplateRequest.Builder::indexPatterns, JsonpDeserializer.arrayDeserializer(JsonpDeserializer.stringDeserializer()), + "index_patterns"); + objectDeserializer.add(PutIndexTemplateRequest.Builder::version, JsonpDeserializer.longDeserializer(), "version"); + objectDeserializer.add(PutIndexTemplateRequest.Builder::priority, JsonpDeserializer.integerDeserializer(), "priority"); + objectDeserializer.add(PutIndexTemplateRequest.Builder::composedOf, JsonpDeserializer.arrayDeserializer(JsonpDeserializer.stringDeserializer()), + "composed_of"); + objectDeserializer.add(PutIndexTemplateRequest.Builder::template, IndexTemplateMapping._DESERIALIZER, "template"); + } + + static JsonpDeserializer getJsonpDeserializer(final String name) { + return ObjectBuilderDeserializer + .lazy( + () -> new PutIndexTemplateRequest.Builder().name(name), + PutIndexTemplateRequestDeserializer::setupPutIndexTemplateRequestDeserializer); + } + } +} diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/Es6IndexTemplateAPIWrapper.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/Es6IndexTemplateAPIWrapper.java new file mode 100644 index 0000000000..36d223c62c --- /dev/null +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/Es6IndexTemplateAPIWrapper.java @@ -0,0 +1,126 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +import com.fasterxml.jackson.databind.ObjectMapper; +import jakarta.json.stream.JsonParser; +import org.opensearch.client.json.JsonpMapper; +import org.opensearch.client.opensearch.OpenSearchClient; +import org.opensearch.client.opensearch._types.ErrorResponse; +import org.opensearch.client.opensearch.indices.ExistsTemplateRequest; +import org.opensearch.client.opensearch.indices.GetTemplateRequest; +import org.opensearch.client.opensearch.indices.GetTemplateResponse; +import org.opensearch.client.opensearch.indices.OpenSearchIndicesClient; +import org.opensearch.client.opensearch.indices.PutTemplateRequest; +import org.opensearch.client.opensearch.indices.PutTemplateResponse; +import org.opensearch.client.transport.JsonEndpoint; +import org.opensearch.client.transport.endpoints.BooleanResponse; +import org.opensearch.client.transport.endpoints.SimpleEndpoint; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +public class Es6IndexTemplateAPIWrapper implements IndexTemplateAPIWrapper { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private final OpenSearchClient openSearchClient; + + public Es6IndexTemplateAPIWrapper(final OpenSearchClient openSearchClient) { + this.openSearchClient = openSearchClient; + } + + @Override + public void putTemplate(final IndexTemplate indexTemplate) throws IOException { + if(!(indexTemplate instanceof LegacyIndexTemplate)) { + throw new IllegalArgumentException("Unexpected indexTemplate provided to createTemplate."); + } + + final Map templateMapping = ((LegacyIndexTemplate) indexTemplate).getTemplateMap(); + final String indexTemplateString = OBJECT_MAPPER.writeValueAsString(templateMapping); + + // Parse byte array to Map + final ByteArrayInputStream byteIn = new ByteArrayInputStream( + indexTemplateString.getBytes(StandardCharsets.UTF_8)); + final JsonpMapper mapper = openSearchClient._transport().jsonpMapper(); + final JsonParser parser = mapper.jsonProvider().createParser(byteIn); + + final PutTemplateRequest putTemplateRequest = PutTemplateRequestDeserializer.getJsonpDeserializer() + .deserialize(parser, mapper); + + final OpenSearchIndicesClient openSearchIndicesClient = openSearchClient.indices(); + final JsonEndpoint endpoint = es6PutTemplateEndpoint(putTemplateRequest); + openSearchIndicesClient._transport().performRequest(putTemplateRequest, endpoint, openSearchIndicesClient._transportOptions()); + } + + @Override + public Optional getTemplate(final String templateName) throws IOException { + final ExistsTemplateRequest existsTemplateRequest = new ExistsTemplateRequest.Builder() + .name(templateName) + .build(); + final BooleanResponse booleanResponse = openSearchClient.indices().existsTemplate( + existsTemplateRequest); + if (!booleanResponse.value()) { + return Optional.empty(); + } + + final GetTemplateRequest getTemplateRequest = new GetTemplateRequest.Builder() + .name(templateName) + .build(); + return Optional.of(openSearchClient.indices().getTemplate(getTemplateRequest)); + } + + private JsonEndpoint es6PutTemplateEndpoint( + final PutTemplateRequest putTemplateRequest) { + return new SimpleEndpoint<>( + + // Request method + request -> { + return "PUT"; + + }, + + // Request path + request -> { + final int _name = 1 << 0; + + int propsSet = 0; + + propsSet |= _name; + + if (propsSet == (_name)) { + StringBuilder buf = new StringBuilder(); + buf.append("/_template"); + buf.append("/"); + SimpleEndpoint.pathEncode(request.name(), buf); + buf.append("?include_type_name=false"); + return buf.toString(); + } + throw SimpleEndpoint.noPathTemplateFound("path"); + + }, + + // Request parameters + request -> { + Map params = new HashMap<>(); + if (request.masterTimeout() != null) { + params.put("master_timeout", request.masterTimeout()._toJsonString()); + } + if (request.clusterManagerTimeout() != null) { + params.put("cluster_manager_timeout", request.clusterManagerTimeout()._toJsonString()); + } + if (request.flatSettings() != null) { + params.put("flat_settings", String.valueOf(request.flatSettings())); + } + if (request.create() != null) { + params.put("create", String.valueOf(request.create())); + } + if (request.timeout() != null) { + params.put("timeout", request.timeout()._toJsonString()); + } + return params; + + }, SimpleEndpoint.emptyMap(), true, PutTemplateResponse._DESERIALIZER); + } +} 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 0cf6221302..ccc2e1c951 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,7 +57,7 @@ public class IndexConfiguration { public static final String DOCUMENT_ROOT_KEY = "document_root_key"; private IndexType indexType; - private final TemplateType templateType; + private TemplateType templateType; private final String indexAlias; private final Map indexTemplate; private final String documentIdField; @@ -90,7 +90,7 @@ private IndexConfiguration(final Builder builder) { this.s3AwsExternalId = builder.s3AwsStsExternalId; this.s3Client = builder.s3Client; - this.templateType = builder.templateType != null ? builder.templateType : TemplateType.V1; + determineTemplateType(builder); this.indexTemplate = readIndexTemplate(builder.templateFile, indexType, templateType); if (builder.numReplicas > 0) { @@ -130,13 +130,18 @@ private IndexConfiguration(final Builder builder) { this.documentRootKey = builder.documentRootKey; } + private void determineTemplateType(Builder builder) { + this.templateType = DistributionVersion.ES6.equals(builder.distributionVersion) ? TemplateType.V1 : + (builder.templateType != null ? builder.templateType : TemplateType.V1); + } + private void determineIndexType(Builder builder) { if(builder.indexType != null) { Optional mappedIndexType = IndexType.getByValue(builder.indexType); indexType = mappedIndexType.orElseThrow( () -> new IllegalArgumentException("Value of the parameter, index_type, must be from the list: " + IndexType.getIndexTypeValues())); - } else if (builder.serverless || DistributionVersion.ES6.equals(builder.distributionVersion)) { + } else if (builder.serverless) { this.indexType = IndexType.MANAGEMENT_DISABLED; } else { this.indexType = IndexType.CUSTOM; diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapper.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapper.java new file mode 100644 index 0000000000..8a267d15b3 --- /dev/null +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapper.java @@ -0,0 +1,23 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +import java.io.IOException; +import java.util.Optional; + +/** + * Client wrapper for calling index template APIs. + */ +public interface IndexTemplateAPIWrapper { + /** + * Create or update the index template + * + * @param indexTemplate The {@link IndexTemplate} to create or update. + */ + void putTemplate(IndexTemplate indexTemplate) throws IOException; + + /** + * Retrieve the existing index template + * + * @param name The index template name to retrieve by. + */ + Optional getTemplate(String name) throws IOException; +} diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapperFactory.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapperFactory.java new file mode 100644 index 0000000000..0d3333c788 --- /dev/null +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapperFactory.java @@ -0,0 +1,17 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +import org.opensearch.client.opensearch.OpenSearchClient; +import org.opensearch.dataprepper.plugins.sink.opensearch.DistributionVersion; + +public class IndexTemplateAPIWrapperFactory { + public static IndexTemplateAPIWrapper getWrapper(final IndexConfiguration indexConfiguration, + final OpenSearchClient openSearchClient) { + if (DistributionVersion.ES6.equals(indexConfiguration.getDistributionVersion())) { + return new Es6IndexTemplateAPIWrapper(openSearchClient); + } else if (TemplateType.V1.equals(indexConfiguration.getTemplateType())) { + return new OpenSearchLegacyTemplateAPIWrapper(openSearchClient); + } else { + return new ComposableTemplateAPIWrapper(openSearchClient); + } + } +} diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/LegacyIndexTemplate.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/LegacyIndexTemplate.java new file mode 100644 index 0000000000..1d6e6be00e --- /dev/null +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/LegacyIndexTemplate.java @@ -0,0 +1,48 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class LegacyIndexTemplate implements IndexTemplate { + + public static final String SETTINGS_KEY = "settings"; + private final Map templateMap; + + public LegacyIndexTemplate(final Map templateMap) { + this.templateMap = new HashMap<>(templateMap); + if(this.templateMap.containsKey(SETTINGS_KEY)) { + final HashMap copiedSettings = new HashMap<>((Map) this.templateMap.get(SETTINGS_KEY)); + this.templateMap.put(SETTINGS_KEY, copiedSettings); + } + } + + @Override + public void setTemplateName(final String name) { + templateMap.put("name", name); + } + + @Override + public void setIndexPatterns(final List indexPatterns) { + templateMap.put("index_patterns", indexPatterns); + } + + @Override + public void putCustomSetting(final String name, final Object value) { + Map settings = (Map) this.templateMap.computeIfAbsent(SETTINGS_KEY, x -> new HashMap<>()); + settings.put(name, value); + } + + @Override + public Optional getVersion() { + if(!templateMap.containsKey("version")) + return Optional.empty(); + final Number version = (Number) templateMap.get("version"); + return Optional.of(version.longValue()); + } + + Map getTemplateMap() { + return this.templateMap; + } +} diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/OpenSearchLegacyTemplateAPIWrapper.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/OpenSearchLegacyTemplateAPIWrapper.java new file mode 100644 index 0000000000..5912d72f21 --- /dev/null +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/OpenSearchLegacyTemplateAPIWrapper.java @@ -0,0 +1,67 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +import com.fasterxml.jackson.databind.ObjectMapper; +import jakarta.json.stream.JsonParser; +import org.opensearch.client.json.JsonpMapper; +import org.opensearch.client.opensearch.OpenSearchClient; +import org.opensearch.client.opensearch.indices.ExistsTemplateRequest; +import org.opensearch.client.opensearch.indices.GetTemplateRequest; +import org.opensearch.client.opensearch.indices.GetTemplateResponse; +import org.opensearch.client.opensearch.indices.OpenSearchIndicesClient; +import org.opensearch.client.opensearch.indices.PutTemplateRequest; +import org.opensearch.client.transport.endpoints.BooleanResponse; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.Optional; + +public class OpenSearchLegacyTemplateAPIWrapper implements IndexTemplateAPIWrapper { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private final OpenSearchClient openSearchClient; + + public OpenSearchLegacyTemplateAPIWrapper(final OpenSearchClient openSearchClient) { + this.openSearchClient = openSearchClient; + } + + @Override + public void putTemplate(IndexTemplate indexTemplate) throws IOException { + if(!(indexTemplate instanceof LegacyIndexTemplate)) { + throw new IllegalArgumentException("Unexpected indexTemplate provided to createTemplate."); + } + + final Map templateMapping = ((LegacyIndexTemplate) indexTemplate).getTemplateMap(); + final String indexTemplateString = OBJECT_MAPPER.writeValueAsString(templateMapping); + + // Parse byte array to Map + final ByteArrayInputStream byteIn = new ByteArrayInputStream( + indexTemplateString.getBytes(StandardCharsets.UTF_8)); + final JsonpMapper mapper = openSearchClient._transport().jsonpMapper(); + final JsonParser parser = mapper.jsonProvider().createParser(byteIn); + + final PutTemplateRequest putTemplateRequest = PutTemplateRequestDeserializer.getJsonpDeserializer() + .deserialize(parser, mapper); + + final OpenSearchIndicesClient openSearchIndicesClient = openSearchClient.indices(); + openSearchIndicesClient.putTemplate(putTemplateRequest); + } + + @Override + public Optional getTemplate(final String templateName) throws IOException { + final ExistsTemplateRequest existsTemplateRequest = new ExistsTemplateRequest.Builder() + .name(templateName) + .build(); + final BooleanResponse booleanResponse = openSearchClient.indices().existsTemplate( + existsTemplateRequest); + if (!booleanResponse.value()) { + return Optional.empty(); + } + + final GetTemplateRequest getTemplateRequest = new GetTemplateRequest.Builder() + .name(templateName) + .build(); + return Optional.of(openSearchClient.indices().getTemplate(getTemplateRequest)); + } +} diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/TemplateType.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/TemplateType.java index b00e1d5ad7..0c48d8ef41 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/TemplateType.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/TemplateType.java @@ -5,8 +5,6 @@ package org.opensearch.dataprepper.plugins.sink.opensearch.index; -import org.opensearch.client.opensearch.OpenSearchClient; - import java.util.Arrays; import java.util.Map; import java.util.function.Function; @@ -33,9 +31,9 @@ public enum TemplateType { )); private final String name; - private final Function factoryFunction; + private final Function factoryFunction; - TemplateType(final String name, final Function factoryFunction) { + TemplateType(final String name, final Function factoryFunction) { this.name = name; this.factoryFunction = factoryFunction; } @@ -44,8 +42,8 @@ public static TemplateType fromTypeName(final String name) { return TYPE_NAME_MAP.get(name); } - public TemplateStrategy createTemplateStrategy(final OpenSearchClient openSearchClient) { - return factoryFunction.apply(openSearchClient); + public TemplateStrategy createTemplateStrategy(final IndexTemplateAPIWrapper indexTemplateAPIWrapper) { + return factoryFunction.apply(indexTemplateAPIWrapper); } String getTypeName() { diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/V1TemplateStrategy.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/V1TemplateStrategy.java index 46f103b163..aa780d9479 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/V1TemplateStrategy.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/V1TemplateStrategy.java @@ -5,37 +5,30 @@ package org.opensearch.dataprepper.plugins.sink.opensearch.index; -import com.fasterxml.jackson.databind.ObjectMapper; -import jakarta.json.stream.JsonParser; -import org.opensearch.client.json.JsonpMapper; -import org.opensearch.client.opensearch.OpenSearchClient; -import org.opensearch.client.opensearch.indices.ExistsTemplateRequest; -import org.opensearch.client.opensearch.indices.GetTemplateRequest; import org.opensearch.client.opensearch.indices.GetTemplateResponse; -import org.opensearch.client.opensearch.indices.PutTemplateRequest; -import org.opensearch.client.opensearch.indices.TemplateMapping; -import org.opensearch.client.transport.endpoints.BooleanResponse; -import java.io.ByteArrayInputStream; import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.Optional; class V1TemplateStrategy implements TemplateStrategy { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private final OpenSearchClient openSearchClient; + private final IndexTemplateAPIWrapper indexTemplateAPIWrapper; - public V1TemplateStrategy(final OpenSearchClient openSearchClient) { - this.openSearchClient = openSearchClient; + public V1TemplateStrategy(final IndexTemplateAPIWrapper indexTemplateAPIWrapper) { + this.indexTemplateAPIWrapper = indexTemplateAPIWrapper; } @Override public Optional getExistingTemplateVersion(final String templateName) throws IOException { - return getTemplateMapping(templateName) - .map(TemplateMapping::version); + return indexTemplateAPIWrapper.getTemplate(templateName) + .map(response -> { + if (response.result().size() == 1) { + return response.result().values().stream().findFirst().get().version(); + } else { + throw new RuntimeException(String.format("Found zero or multiple index templates result when querying for %s", + templateName)); + } + }); } @Override @@ -45,88 +38,6 @@ public IndexTemplate createIndexTemplate(final Map templateMap) @Override public void createTemplate(final IndexTemplate indexTemplate) throws IOException { - if(!(indexTemplate instanceof LegacyIndexTemplate)) { - throw new IllegalArgumentException("Unexpected indexTemplate provided to createTemplate."); - } - - final Map templateMapping = ((LegacyIndexTemplate) indexTemplate).getTemplateMap(); - - final String indexTemplateString = OBJECT_MAPPER.writeValueAsString(templateMapping); - - // Parse byte array to Map - final ByteArrayInputStream byteIn = new ByteArrayInputStream( - indexTemplateString.getBytes(StandardCharsets.UTF_8)); - final JsonpMapper mapper = openSearchClient._transport().jsonpMapper(); - final JsonParser parser = mapper.jsonProvider().createParser(byteIn); - - final PutTemplateRequest putTemplateRequest = PutTemplateRequestDeserializer.getJsonpDeserializer() - .deserialize(parser, mapper); - - openSearchClient.indices().putTemplate(putTemplateRequest); - } - - private Optional getTemplateMapping(final String templateName) throws IOException { - final ExistsTemplateRequest existsTemplateRequest = new ExistsTemplateRequest.Builder() - .name(templateName) - .build(); - final BooleanResponse booleanResponse = openSearchClient.indices().existsTemplate( - existsTemplateRequest); - if (!booleanResponse.value()) { - return Optional.empty(); - } - - final GetTemplateRequest getTemplateRequest = new GetTemplateRequest.Builder() - .name(templateName) - .build(); - final GetTemplateResponse response = openSearchClient.indices().getTemplate(getTemplateRequest); - - if (response.result().size() == 1) { - return response.result().values().stream().findFirst(); - } else { - throw new RuntimeException(String.format("Found zero or multiple index templates result when querying for %s", - templateName)); - } - } - - static class LegacyIndexTemplate implements IndexTemplate { - - public static final String SETTINGS_KEY = "settings"; - private final Map templateMap; - - private LegacyIndexTemplate(final Map templateMap) { - this.templateMap = new HashMap<>(templateMap); - if(this.templateMap.containsKey(SETTINGS_KEY)) { - final HashMap copiedSettings = new HashMap<>((Map) this.templateMap.get(SETTINGS_KEY)); - this.templateMap.put(SETTINGS_KEY, copiedSettings); - } - } - - @Override - public void setTemplateName(final String name) { - templateMap.put("name", name); - } - - @Override - public void setIndexPatterns(final List indexPatterns) { - templateMap.put("index_patterns", indexPatterns); - } - - @Override - public void putCustomSetting(final String name, final Object value) { - Map settings = (Map) this.templateMap.computeIfAbsent(SETTINGS_KEY, x -> new HashMap<>()); - settings.put(name, value); - } - - @Override - public Optional getVersion() { - if(!templateMap.containsKey("version")) - return Optional.empty(); - final Number version = (Number) templateMap.get("version"); - return Optional.of(version.longValue()); - } - - Map getTemplateMap() { - return this.templateMap; - } + indexTemplateAPIWrapper.putTemplate(indexTemplate); } } diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplateStrategyTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplateStrategyTest.java index 100ed8baa1..f9e0154007 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplateStrategyTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableIndexTemplateStrategyTest.java @@ -11,22 +11,13 @@ 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.junit.jupiter.MockitoExtension; -import org.opensearch.client.opensearch.OpenSearchClient; -import org.opensearch.client.opensearch.indices.ExistsIndexTemplateRequest; -import org.opensearch.client.opensearch.indices.GetIndexTemplateRequest; import org.opensearch.client.opensearch.indices.GetIndexTemplateResponse; -import org.opensearch.client.opensearch.indices.OpenSearchIndicesClient; -import org.opensearch.client.opensearch.indices.PutIndexTemplateRequest; +import org.opensearch.client.opensearch.indices.get_index_template.IndexTemplate; import org.opensearch.client.opensearch.indices.get_index_template.IndexTemplateItem; -import org.opensearch.client.transport.OpenSearchTransport; -import org.opensearch.client.transport.endpoints.BooleanResponse; -import org.opensearch.dataprepper.plugins.sink.opensearch.bulk.PreSerializedJsonpMapper; import java.io.IOException; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -35,90 +26,65 @@ 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.CoreMatchers.nullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.lenient; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) class ComposableIndexTemplateStrategyTest { @Mock - private OpenSearchClient openSearchClient; - + private IndexTemplateAPIWrapper indexTemplateAPIWrapper; + @Mock + private GetIndexTemplateResponse getIndexTemplateResponse; + @Mock + private IndexTemplateItem indexTemplateItem; @Mock - private OpenSearchIndicesClient openSearchIndicesClient; + private IndexTemplate indexTemplate; private Random random; private String indexTemplateName; @BeforeEach void setUp() { random = new Random(); - lenient().when(openSearchClient.indices()).thenReturn(openSearchIndicesClient); indexTemplateName = UUID.randomUUID().toString(); } - private ComposableIndexTemplateStrategy createObjectUnderTest() { - return new ComposableIndexTemplateStrategy(openSearchClient); + return new ComposableIndexTemplateStrategy(indexTemplateAPIWrapper); } @Test - void getExistingTemplateVersion_should_calls_existIndexTemplate_with_indexTemplateName() throws IOException { - final BooleanResponse booleanResponse = mock(BooleanResponse.class); - when(booleanResponse.value()).thenReturn(false); - when(openSearchIndicesClient.existsIndexTemplate(any(ExistsIndexTemplateRequest.class))) - .thenReturn(booleanResponse); - + void getExistingTemplateVersion_should_calls_getTemplate_with_templateName() throws IOException { + when(indexTemplateAPIWrapper.getTemplate(any())).thenReturn(Optional.of(getIndexTemplateResponse)); + when(getIndexTemplateResponse.indexTemplates()).thenReturn(List.of(indexTemplateItem)); + when(indexTemplateItem.indexTemplate()).thenReturn(indexTemplate); createObjectUnderTest().getExistingTemplateVersion(indexTemplateName); - final ArgumentCaptor existsTemplateRequestArgumentCaptor = ArgumentCaptor.forClass(ExistsIndexTemplateRequest.class); - verify(openSearchIndicesClient).existsIndexTemplate(existsTemplateRequestArgumentCaptor.capture()); - - final ExistsIndexTemplateRequest actualRequest = existsTemplateRequestArgumentCaptor.getValue(); - assertThat(actualRequest.name(), equalTo(indexTemplateName)); - - verifyNoMoreInteractions(openSearchIndicesClient); + verify(indexTemplateAPIWrapper).getTemplate(eq(indexTemplateName)); + verify(indexTemplate).version(); } @Test void getExistingTemplateVersion_should_return_empty_if_no_template_exists() throws IOException { - final BooleanResponse booleanResponse = mock(BooleanResponse.class); - when(booleanResponse.value()).thenReturn(false); - when(openSearchIndicesClient.existsIndexTemplate(any(ExistsIndexTemplateRequest.class))) - .thenReturn(booleanResponse); - - final Optional optionalVersion = createObjectUnderTest().getExistingTemplateVersion(indexTemplateName); - - assertThat(optionalVersion, notNullValue()); - assertThat(optionalVersion.isPresent(), equalTo(false)); + final Optional version = createObjectUnderTest().getExistingTemplateVersion(indexTemplateName); + assertThat(version.isEmpty(), is(true)); } @Nested class WithExistingIndexTemplate { - @BeforeEach - void setUp() throws IOException { - final BooleanResponse booleanResponse = mock(BooleanResponse.class); - when(booleanResponse.value()).thenReturn(true); - when(openSearchIndicesClient.existsIndexTemplate(any(ExistsIndexTemplateRequest.class))) - .thenReturn(booleanResponse); - } - @Test void getExistingTemplateVersion_should_return_empty_if_index_template_exists_without_version() throws IOException { - final GetIndexTemplateResponse getIndexTemplateResponse = mock(GetIndexTemplateResponse.class); - final IndexTemplateItem indexTemplateItem = mock(IndexTemplateItem.class); - org.opensearch.client.opensearch.indices.get_index_template.IndexTemplate indexTemplate = mock(org.opensearch.client.opensearch.indices.get_index_template.IndexTemplate.class); - when(indexTemplate.version()).thenReturn(null); + when(indexTemplateAPIWrapper.getTemplate(any())).thenReturn(Optional.of(getIndexTemplateResponse)); + when(getIndexTemplateResponse.indexTemplates()).thenReturn(List.of(indexTemplateItem)); when(indexTemplateItem.indexTemplate()).thenReturn(indexTemplate); - when(getIndexTemplateResponse.indexTemplates()).thenReturn(Collections.singletonList(indexTemplateItem)); - when(openSearchIndicesClient.getIndexTemplate(any(GetIndexTemplateRequest.class))) - .thenReturn(getIndexTemplateResponse); + when(indexTemplate.version()).thenReturn(null); final Optional optionalVersion = createObjectUnderTest().getExistingTemplateVersion(indexTemplateName); @@ -129,14 +95,10 @@ void getExistingTemplateVersion_should_return_empty_if_index_template_exists_wit @Test void getExistingTemplateVersion_should_return_template_version_if_template_exists() throws IOException { final Long version = (long) (random.nextInt(10_000) + 100); - final GetIndexTemplateResponse getIndexTemplateResponse = mock(GetIndexTemplateResponse.class); - final IndexTemplateItem indexTemplateItem = mock(IndexTemplateItem.class); - org.opensearch.client.opensearch.indices.get_index_template.IndexTemplate indexTemplate = mock(org.opensearch.client.opensearch.indices.get_index_template.IndexTemplate.class); - when(indexTemplate.version()).thenReturn(version); + when(indexTemplateAPIWrapper.getTemplate(any())).thenReturn(Optional.of(getIndexTemplateResponse)); + when(getIndexTemplateResponse.indexTemplates()).thenReturn(List.of(indexTemplateItem)); when(indexTemplateItem.indexTemplate()).thenReturn(indexTemplate); - when(getIndexTemplateResponse.indexTemplates()).thenReturn(Collections.singletonList(indexTemplateItem)); - when(openSearchIndicesClient.getIndexTemplate(any(GetIndexTemplateRequest.class))) - .thenReturn(getIndexTemplateResponse); + when(indexTemplate.version()).thenReturn(version); final Optional optionalVersion = createObjectUnderTest().getExistingTemplateVersion(indexTemplateName); @@ -147,133 +109,37 @@ void getExistingTemplateVersion_should_return_template_version_if_template_exist @ParameterizedTest @ValueSource(ints = {0, 2}) - void getExistingTemplateVersion_should_throw_if_get_template_returns_unexpected_number_of_templates(final int numberOfTemplatesReturned) throws IOException { - final GetIndexTemplateResponse getIndexTemplateResponse = mock(GetIndexTemplateResponse.class); - final List templateResult = mock(List.class); - when(templateResult.size()).thenReturn(numberOfTemplatesReturned); - when(getIndexTemplateResponse.indexTemplates()).thenReturn(templateResult); - when(openSearchIndicesClient.getIndexTemplate(any(GetIndexTemplateRequest.class))) - .thenReturn(getIndexTemplateResponse); - + void getExistingTemplateVersion_should_throw_if_get_template_returns_unexpected_number_of_templates( + final int numberOfTemplatesReturned) throws IOException { + when(indexTemplateAPIWrapper.getTemplate(any())).thenReturn(Optional.of(getIndexTemplateResponse)); + final List indexTemplateItems = mock(List.class); + when(indexTemplateItems.size()).thenReturn(numberOfTemplatesReturned); + when(getIndexTemplateResponse.indexTemplates()).thenReturn(indexTemplateItems); - final ComposableIndexTemplateStrategy objectUnderTest = createObjectUnderTest(); - assertThrows(RuntimeException.class, () -> objectUnderTest.getExistingTemplateVersion(indexTemplateName)); + assertThrows(RuntimeException.class, + () -> createObjectUnderTest().getExistingTemplateVersion(indexTemplateName)); - verify(openSearchIndicesClient).getIndexTemplate(any(GetIndexTemplateRequest.class)); + verify(indexTemplateAPIWrapper).getTemplate(eq(indexTemplateName)); } } @Test - void createTemplate_throws_if_template_is_not_ComposableIndexTemplate() { - final IndexTemplate indexTemplate = mock(IndexTemplate.class); + void createTemplate_throws_if_putTemplate_throws() throws IOException { + doThrow(IOException.class).when(indexTemplateAPIWrapper).putTemplate(any()); + final org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplate indexTemplate = mock( + org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplate.class); final ComposableIndexTemplateStrategy objectUnderTest = createObjectUnderTest(); - assertThrows(IllegalArgumentException.class, () -> objectUnderTest.createTemplate(indexTemplate)); + assertThrows(IOException.class, () -> objectUnderTest.createTemplate(indexTemplate)); } - @Nested - class IndexTemplateWithCreateTemplateTests { - private ArgumentCaptor putIndexTemplateRequestArgumentCaptor; - private List indexPatterns; - - @BeforeEach - void setUp() { - final OpenSearchTransport openSearchTransport = mock(OpenSearchTransport.class); - when(openSearchClient._transport()).thenReturn(openSearchTransport); - when(openSearchTransport.jsonpMapper()).thenReturn(new PreSerializedJsonpMapper()); - - putIndexTemplateRequestArgumentCaptor = ArgumentCaptor.forClass(PutIndexTemplateRequest.class); - - indexPatterns = Collections.singletonList(UUID.randomUUID().toString()); - } - - @Test - void createTemplate_with_setTemplateName_performs_putIndexTemplate_request() throws IOException { - final ComposableIndexTemplateStrategy objectUnderTest = createObjectUnderTest(); - - final IndexTemplate indexTemplate = objectUnderTest.createIndexTemplate(new HashMap<>()); - indexTemplate.setTemplateName(indexTemplateName); - objectUnderTest.createTemplate(indexTemplate); - - verify(openSearchIndicesClient).putIndexTemplate(putIndexTemplateRequestArgumentCaptor.capture()); - - final PutIndexTemplateRequest actualPutRequest = putIndexTemplateRequestArgumentCaptor.getValue(); - - assertThat(actualPutRequest.name(), equalTo(indexTemplateName)); - - assertThat(actualPutRequest.version(), nullValue()); - assertThat(actualPutRequest.indexPatterns(), notNullValue()); - assertThat(actualPutRequest.indexPatterns(), equalTo(Collections.emptyList())); - assertThat(actualPutRequest.template(), nullValue()); - assertThat(actualPutRequest.priority(), nullValue()); - assertThat(actualPutRequest.composedOf(), notNullValue()); - assertThat(actualPutRequest.composedOf(), equalTo(Collections.emptyList())); - } - - @Test - void createTemplate_with_setIndexPatterns_performs_putIndexTemplate_request() throws IOException { - final ComposableIndexTemplateStrategy objectUnderTest = createObjectUnderTest(); - - final List indexPatterns = Collections.singletonList(UUID.randomUUID().toString()); - - final IndexTemplate indexTemplate = objectUnderTest.createIndexTemplate(new HashMap<>()); - indexTemplate.setTemplateName(indexTemplateName); - indexTemplate.setIndexPatterns(indexPatterns); - objectUnderTest.createTemplate(indexTemplate); - - verify(openSearchIndicesClient).putIndexTemplate(putIndexTemplateRequestArgumentCaptor.capture()); - - final PutIndexTemplateRequest actualPutRequest = putIndexTemplateRequestArgumentCaptor.getValue(); - - assertThat(actualPutRequest.name(), equalTo(indexTemplateName)); - assertThat(actualPutRequest.indexPatterns(), equalTo(indexPatterns)); - - assertThat(actualPutRequest.version(), nullValue()); - assertThat(actualPutRequest.template(), nullValue()); - assertThat(actualPutRequest.priority(), nullValue()); - assertThat(actualPutRequest.composedOf(), notNullValue()); - assertThat(actualPutRequest.composedOf(), equalTo(Collections.emptyList())); - } - - @Test - void createTemplate_with_defined_template_values_performs_putIndexTemplate_request() throws IOException { - final ComposableIndexTemplateStrategy objectUnderTest = createObjectUnderTest(); + @Test + void createTemplate_performs_putTemplate_request() throws IOException { + final ComposableIndexTemplateStrategy objectUnderTest = createObjectUnderTest(); + final org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplate indexTemplate = mock(org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplate.class); - final Long version = (long) (random.nextInt(10_000) + 100); - final int priority = random.nextInt(1000) + 100; - final String numberOfShards = Integer.toString(random.nextInt(1000) + 100); - final List composedOf = Collections.singletonList(UUID.randomUUID().toString()); - - final IndexTemplate indexTemplate = objectUnderTest.createIndexTemplate( - Map.of("version", version, - "priority", priority, - "composed_of", composedOf, - "template", Map.of( - "settings", Map.of( - "index", Map.of("number_of_shards", numberOfShards)), - "mappings", Map.of("date_detection", true) - ) - )); - indexTemplate.setTemplateName(indexTemplateName); - indexTemplate.setIndexPatterns(indexPatterns); - objectUnderTest.createTemplate(indexTemplate); - - verify(openSearchIndicesClient).putIndexTemplate(putIndexTemplateRequestArgumentCaptor.capture()); - - final PutIndexTemplateRequest actualPutRequest = putIndexTemplateRequestArgumentCaptor.getValue(); - - assertThat(actualPutRequest.name(), equalTo(indexTemplateName)); - assertThat(actualPutRequest.indexPatterns(), equalTo(indexPatterns)); - assertThat(actualPutRequest.version(), equalTo(version)); - assertThat(actualPutRequest.priority(), equalTo(priority)); - assertThat(actualPutRequest.composedOf(), equalTo(composedOf)); - assertThat(actualPutRequest.template(), notNullValue()); - assertThat(actualPutRequest.template().mappings(), notNullValue()); - assertThat(actualPutRequest.template().mappings().dateDetection(), equalTo(true)); - assertThat(actualPutRequest.template().settings(), notNullValue()); - assertThat(actualPutRequest.template().settings().index(), notNullValue()); - assertThat(actualPutRequest.template().settings().index().numberOfShards(), equalTo(numberOfShards)); - } + objectUnderTest.createTemplate(indexTemplate); + verify(indexTemplateAPIWrapper).putTemplate(indexTemplate); } @Nested @@ -287,7 +153,8 @@ void setUp() { @Test void getVersion_returns_empty_if_no_version() { - final IndexTemplate indexTemplate = createObjectUnderTest().createIndexTemplate(providedTemplateMap); + final org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplate indexTemplate = + createObjectUnderTest().createIndexTemplate(providedTemplateMap); final Optional optionalVersion = indexTemplate.getVersion(); assertThat(optionalVersion, notNullValue()); @@ -299,7 +166,8 @@ void getVersion_returns_version_from_root_map() { final Long version = (long) (random.nextInt(10_000) + 100); providedTemplateMap.put("version", version); - final IndexTemplate indexTemplate = createObjectUnderTest().createIndexTemplate(providedTemplateMap); + final org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplate indexTemplate = + createObjectUnderTest().createIndexTemplate(providedTemplateMap); final Optional optionalVersion = indexTemplate.getVersion(); assertThat(optionalVersion, notNullValue()); @@ -312,7 +180,8 @@ void getVersion_returns_version_from_root_map_when_provided_as_int() { final Integer version = random.nextInt(10_000) + 100; providedTemplateMap.put("version", version); - final IndexTemplate indexTemplate = createObjectUnderTest().createIndexTemplate(providedTemplateMap); + final org.opensearch.dataprepper.plugins.sink.opensearch.index.IndexTemplate indexTemplate = + createObjectUnderTest().createIndexTemplate(providedTemplateMap); final Optional optionalVersion = indexTemplate.getVersion(); assertThat(optionalVersion, notNullValue()); diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableTemplateAPIWrapperTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableTemplateAPIWrapperTest.java new file mode 100644 index 0000000000..d6040a017d --- /dev/null +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/ComposableTemplateAPIWrapperTest.java @@ -0,0 +1,240 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +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.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.client.opensearch.OpenSearchClient; +import org.opensearch.client.opensearch.indices.ExistsIndexTemplateRequest; +import org.opensearch.client.opensearch.indices.GetIndexTemplateRequest; +import org.opensearch.client.opensearch.indices.GetIndexTemplateResponse; +import org.opensearch.client.opensearch.indices.OpenSearchIndicesClient; +import org.opensearch.client.opensearch.indices.PutIndexTemplateRequest; +import org.opensearch.client.transport.OpenSearchTransport; +import org.opensearch.client.transport.endpoints.BooleanResponse; +import org.opensearch.dataprepper.plugins.sink.opensearch.bulk.PreSerializedJsonpMapper; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +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; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ComposableTemplateAPIWrapperTest { + @Mock + private OpenSearchClient openSearchClient; + @Mock + private OpenSearchIndicesClient openSearchIndicesClient; + @Mock + private OpenSearchTransport openSearchTransport; + @Mock + private GetIndexTemplateResponse getIndexTemplateResponse; + @Mock + private BooleanResponse booleanResponse; + @Captor + private ArgumentCaptor putIndexTemplateRequestArgumentCaptor; + @Captor + private ArgumentCaptor existsIndexTemplateRequestArgumentCaptor; + + private String indexTemplateName; + private ComposableTemplateAPIWrapper objectUnderTest; + private Random random; + + @BeforeEach + void setUp() { + random = new Random(); + objectUnderTest = new ComposableTemplateAPIWrapper(openSearchClient); + lenient().when(openSearchClient.indices()).thenReturn(openSearchIndicesClient); + indexTemplateName = UUID.randomUUID().toString(); + } + + @Test + void putTemplate_throws_if_template_is_not_ComposableIndexTemplate() { + final IndexTemplate indexTemplate = mock(IndexTemplate.class); + assertThrows(IllegalArgumentException.class, () -> objectUnderTest.putTemplate(indexTemplate)); + } + + @Test + void putTemplate_performs_putIndexTemplate_request() throws IOException { + when(openSearchClient._transport()).thenReturn(openSearchTransport); + when(openSearchTransport.jsonpMapper()).thenReturn(new PreSerializedJsonpMapper()); + + final List indexPatterns = Collections.singletonList(UUID.randomUUID().toString()); + final IndexTemplate indexTemplate = new ComposableIndexTemplate(new HashMap<>()); + indexTemplate.setTemplateName(indexTemplateName); + indexTemplate.setIndexPatterns(indexPatterns); + objectUnderTest.putTemplate(indexTemplate); + + verify(openSearchIndicesClient).putIndexTemplate(putIndexTemplateRequestArgumentCaptor.capture()); + + final PutIndexTemplateRequest actualPutRequest = putIndexTemplateRequestArgumentCaptor.getValue(); + + assertThat(actualPutRequest.name(), equalTo(indexTemplateName)); + assertThat(actualPutRequest.indexPatterns(), equalTo(indexPatterns)); + } + + @Test + void getExistingTemplate_should_calls_existIndexTemplate_with_templateName() throws IOException { + when(booleanResponse.value()).thenReturn(false); + when(openSearchIndicesClient.existsIndexTemplate(any(ExistsIndexTemplateRequest.class))) + .thenReturn(booleanResponse); + + objectUnderTest.getTemplate(indexTemplateName); + + verify(openSearchIndicesClient).existsIndexTemplate(existsIndexTemplateRequestArgumentCaptor.capture()); + + final ExistsIndexTemplateRequest actualRequest = existsIndexTemplateRequestArgumentCaptor.getValue(); + assertThat(actualRequest.name(), notNullValue()); + assertThat(actualRequest.name(), equalTo(indexTemplateName)); + } + + @Test + void getExistingTemplate_should_return_empty_if_no_index_template_exists() throws IOException { + when(booleanResponse.value()).thenReturn(false); + when(openSearchIndicesClient.existsIndexTemplate(any(ExistsIndexTemplateRequest.class))) + .thenReturn(booleanResponse); + + final Optional optionalGetIndexTemplateResponse = objectUnderTest.getTemplate( + indexTemplateName); + + assertThat(optionalGetIndexTemplateResponse, notNullValue()); + assertThat(optionalGetIndexTemplateResponse.isPresent(), equalTo(false)); + } + + @Test + void getExistingTemplate_should_return_template_if_template_exists() throws IOException { + when(booleanResponse.value()).thenReturn(true); + when(openSearchIndicesClient.existsIndexTemplate(any(ExistsIndexTemplateRequest.class))) + .thenReturn(booleanResponse); + when(openSearchIndicesClient.getIndexTemplate(any(GetIndexTemplateRequest.class))) + .thenReturn(getIndexTemplateResponse); + + final Optional optionalGetIndexTemplateResponse = objectUnderTest.getTemplate( + indexTemplateName); + + assertThat(optionalGetIndexTemplateResponse, notNullValue()); + assertThat(optionalGetIndexTemplateResponse.isPresent(), equalTo(true)); + assertThat(optionalGetIndexTemplateResponse.get(), equalTo(getIndexTemplateResponse)); + } + + + @Nested + class IndexTemplateWithCreateTemplateTests { + private ArgumentCaptor putIndexTemplateRequestArgumentCaptor; + private List indexPatterns; + + @BeforeEach + void setUp() { + final OpenSearchTransport openSearchTransport = mock(OpenSearchTransport.class); + when(openSearchClient._transport()).thenReturn(openSearchTransport); + when(openSearchTransport.jsonpMapper()).thenReturn(new PreSerializedJsonpMapper()); + + putIndexTemplateRequestArgumentCaptor = ArgumentCaptor.forClass(PutIndexTemplateRequest.class); + + indexPatterns = Collections.singletonList(UUID.randomUUID().toString()); + } + + @Test + void putTemplate_with_setTemplateName_performs_putIndexTemplate_request() throws IOException { + final IndexTemplate indexTemplate = new ComposableIndexTemplate(new HashMap<>()); + indexTemplate.setTemplateName(indexTemplateName); + objectUnderTest.putTemplate(indexTemplate); + + verify(openSearchIndicesClient).putIndexTemplate(putIndexTemplateRequestArgumentCaptor.capture()); + + final PutIndexTemplateRequest actualPutRequest = putIndexTemplateRequestArgumentCaptor.getValue(); + + assertThat(actualPutRequest.name(), equalTo(indexTemplateName)); + + assertThat(actualPutRequest.version(), nullValue()); + assertThat(actualPutRequest.indexPatterns(), notNullValue()); + assertThat(actualPutRequest.indexPatterns(), equalTo(Collections.emptyList())); + assertThat(actualPutRequest.template(), nullValue()); + assertThat(actualPutRequest.priority(), nullValue()); + assertThat(actualPutRequest.composedOf(), notNullValue()); + assertThat(actualPutRequest.composedOf(), equalTo(Collections.emptyList())); + } + + @Test + void putTemplate_with_setIndexPatterns_performs_putIndexTemplate_request() throws IOException { + final List indexPatterns = Collections.singletonList(UUID.randomUUID().toString()); + + final IndexConfiguration indexConfiguration = mock(IndexConfiguration.class); + final IndexTemplate indexTemplate = new ComposableIndexTemplate(new HashMap<>()); + indexTemplate.setTemplateName(indexTemplateName); + indexTemplate.setIndexPatterns(indexPatterns); + objectUnderTest.putTemplate(indexTemplate); + + verify(openSearchIndicesClient).putIndexTemplate(putIndexTemplateRequestArgumentCaptor.capture()); + + final PutIndexTemplateRequest actualPutRequest = putIndexTemplateRequestArgumentCaptor.getValue(); + + assertThat(actualPutRequest.name(), equalTo(indexTemplateName)); + assertThat(actualPutRequest.indexPatterns(), equalTo(indexPatterns)); + + assertThat(actualPutRequest.version(), nullValue()); + assertThat(actualPutRequest.template(), nullValue()); + assertThat(actualPutRequest.priority(), nullValue()); + assertThat(actualPutRequest.composedOf(), notNullValue()); + assertThat(actualPutRequest.composedOf(), equalTo(Collections.emptyList())); + } + + @Test + void putTemplate_with_defined_template_values_performs_putIndexTemplate_request() throws IOException { + final Long version = (long) (random.nextInt(10_000) + 100); + final int priority = random.nextInt(1000) + 100; + final String numberOfShards = Integer.toString(random.nextInt(1000) + 100); + final List composedOf = Collections.singletonList(UUID.randomUUID().toString()); + + final IndexConfiguration indexConfiguration = mock(IndexConfiguration.class); + final IndexTemplate indexTemplate = new ComposableIndexTemplate( + Map.of("version", version, + "priority", priority, + "composed_of", composedOf, + "template", Map.of( + "settings", Map.of( + "index", Map.of("number_of_shards", numberOfShards)), + "mappings", Map.of("date_detection", true) + ) + )); + indexTemplate.setTemplateName(indexTemplateName); + indexTemplate.setIndexPatterns(indexPatterns); + objectUnderTest.putTemplate(indexTemplate); + + verify(openSearchIndicesClient).putIndexTemplate(putIndexTemplateRequestArgumentCaptor.capture()); + + final PutIndexTemplateRequest actualPutRequest = putIndexTemplateRequestArgumentCaptor.getValue(); + + assertThat(actualPutRequest.name(), equalTo(indexTemplateName)); + assertThat(actualPutRequest.indexPatterns(), equalTo(indexPatterns)); + assertThat(actualPutRequest.version(), equalTo(version)); + assertThat(actualPutRequest.priority(), equalTo(priority)); + assertThat(actualPutRequest.composedOf(), equalTo(composedOf)); + assertThat(actualPutRequest.template(), notNullValue()); + assertThat(actualPutRequest.template().mappings(), notNullValue()); + assertThat(actualPutRequest.template().mappings().dateDetection(), equalTo(true)); + assertThat(actualPutRequest.template().settings(), notNullValue()); + assertThat(actualPutRequest.template().settings().index(), notNullValue()); + assertThat(actualPutRequest.template().settings().index().numberOfShards(), equalTo(numberOfShards)); + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/Es6IndexTemplateAPIWrapperTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/Es6IndexTemplateAPIWrapperTest.java new file mode 100644 index 0000000000..ec3f7b2362 --- /dev/null +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/Es6IndexTemplateAPIWrapperTest.java @@ -0,0 +1,151 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +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.Captor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.client.opensearch.OpenSearchClient; +import org.opensearch.client.opensearch._types.ErrorResponse; +import org.opensearch.client.opensearch.indices.ExistsTemplateRequest; +import org.opensearch.client.opensearch.indices.GetTemplateRequest; +import org.opensearch.client.opensearch.indices.GetTemplateResponse; +import org.opensearch.client.opensearch.indices.OpenSearchIndicesClient; +import org.opensearch.client.opensearch.indices.PutTemplateRequest; +import org.opensearch.client.opensearch.indices.PutTemplateResponse; +import org.opensearch.client.transport.JsonEndpoint; +import org.opensearch.client.transport.OpenSearchTransport; +import org.opensearch.client.transport.TransportOptions; +import org.opensearch.client.transport.endpoints.BooleanResponse; +import org.opensearch.dataprepper.plugins.sink.opensearch.bulk.PreSerializedJsonpMapper; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertAll; +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.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class Es6IndexTemplateAPIWrapperTest { + @Mock + private OpenSearchClient openSearchClient; + @Mock + private OpenSearchIndicesClient openSearchIndicesClient; + @Mock + private OpenSearchTransport openSearchTransport; + @Mock + private GetTemplateResponse getTemplateResponse; + @Mock + private BooleanResponse booleanResponse; + @Mock + private TransportOptions transportOptions; + @Captor + private ArgumentCaptor putTemplateRequestArgumentCaptor; + @Captor + private ArgumentCaptor> jsonEndpointArgumentCaptor; + @Captor + private ArgumentCaptor existsTemplateRequestArgumentCaptor; + + private String templateName; + private Es6IndexTemplateAPIWrapper objectUnderTest; + + @BeforeEach + void setUp() { + objectUnderTest = new Es6IndexTemplateAPIWrapper(openSearchClient); + lenient().when(openSearchClient.indices()).thenReturn(openSearchIndicesClient); + templateName = UUID.randomUUID().toString(); + } + + @Test + void putTemplate_throws_if_template_is_not_LegacyIndexTemplate() { + final IndexTemplate indexTemplate = mock(IndexTemplate.class); + assertThrows(IllegalArgumentException.class, () -> objectUnderTest.putTemplate( + indexTemplate)); + } + + @Test + void putTemplate_performs_perform_request() throws IOException { + when(openSearchClient._transport()).thenReturn(openSearchTransport); + when(openSearchIndicesClient._transport()).thenReturn(openSearchTransport); + when(openSearchIndicesClient._transportOptions()).thenReturn(transportOptions); + when(openSearchTransport.jsonpMapper()).thenReturn(new PreSerializedJsonpMapper()); + + final List indexPatterns = Collections.singletonList(UUID.randomUUID().toString()); + final IndexTemplate indexTemplate = new LegacyIndexTemplate(new HashMap<>()); + indexTemplate.setTemplateName(templateName); + indexTemplate.setIndexPatterns(indexPatterns); + objectUnderTest.putTemplate(indexTemplate); + + verify(openSearchTransport).performRequest( + putTemplateRequestArgumentCaptor.capture(), jsonEndpointArgumentCaptor.capture(), eq(transportOptions)); + + final PutTemplateRequest actualPutRequest = putTemplateRequestArgumentCaptor.getValue(); + + assertThat(actualPutRequest.name(), equalTo(templateName)); + assertThat(actualPutRequest.indexPatterns(), equalTo(indexPatterns)); + final JsonEndpoint endpoint = jsonEndpointArgumentCaptor.getValue(); + assertThat(endpoint.requestUrl(actualPutRequest), equalTo( + String.format("/_template/%s?include_type_name=false", templateName))); + } + + @Test + void getExistingTemplate_should_calls_existTemplate_with_templateName() throws IOException { + when(booleanResponse.value()).thenReturn(false); + when(openSearchIndicesClient.existsTemplate(any(ExistsTemplateRequest.class))) + .thenReturn(booleanResponse); + + objectUnderTest.getTemplate(templateName); + + verify(openSearchIndicesClient).existsTemplate(existsTemplateRequestArgumentCaptor.capture()); + + final ExistsTemplateRequest actualRequest = existsTemplateRequestArgumentCaptor.getValue(); + assertThat(actualRequest.name(), notNullValue()); + assertAll( + () -> assertThat(actualRequest.name().size(), equalTo(1)), + () -> assertThat(actualRequest.name(), hasItem(templateName)) + ); + } + + @Test + void getExistingTemplate_should_return_empty_if_no_template_exists() throws IOException { + when(booleanResponse.value()).thenReturn(false); + when(openSearchIndicesClient.existsTemplate(any(ExistsTemplateRequest.class))) + .thenReturn(booleanResponse); + + final Optional optionalGetTemplateResponse = objectUnderTest.getTemplate(templateName); + + assertThat(optionalGetTemplateResponse, notNullValue()); + assertThat(optionalGetTemplateResponse.isPresent(), equalTo(false)); + } + + @Test + void getExistingTemplate_should_return_template_if_template_exists() throws IOException { + when(booleanResponse.value()).thenReturn(true); + when(openSearchIndicesClient.existsTemplate(any(ExistsTemplateRequest.class))) + .thenReturn(booleanResponse); + when(openSearchIndicesClient.getTemplate(any(GetTemplateRequest.class))) + .thenReturn(getTemplateResponse); + + final Optional optionalGetTemplateResponse = objectUnderTest.getTemplate(templateName); + + assertThat(optionalGetTemplateResponse, notNullValue()); + assertThat(optionalGetTemplateResponse.isPresent(), equalTo(true)); + assertThat(optionalGetTemplateResponse.get(), equalTo(getTemplateResponse)); + } +} \ No newline at end of file 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 e1da13867f..64810bf1f2 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 @@ -398,10 +398,12 @@ public void testReadIndexConfig_es6Override() { final Map metadata = initializeConfigMetaData( null, "foo", null, null, null, null); metadata.put(DISTRIBUTION_VERSION, "es6"); + metadata.put(TEMPLATE_TYPE, TemplateType.INDEX_TEMPLATE.getTypeName()); final PluginSetting pluginSetting = getPluginSetting(metadata); final IndexConfiguration indexConfiguration = IndexConfiguration.readIndexConfig(pluginSetting); assertEquals(indexConfiguration.getDistributionVersion(), DistributionVersion.ES6); - assertEquals(IndexType.MANAGEMENT_DISABLED, indexConfiguration.getIndexType()); + assertEquals(TemplateType.V1, indexConfiguration.getTemplateType()); + assertEquals(IndexType.CUSTOM, indexConfiguration.getIndexType()); } @Test diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapperFactoryTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapperFactoryTest.java new file mode 100644 index 0000000000..7f5f4ef4d7 --- /dev/null +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/IndexTemplateAPIWrapperFactoryTest.java @@ -0,0 +1,44 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +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.client.opensearch.OpenSearchClient; +import org.opensearch.dataprepper.plugins.sink.opensearch.DistributionVersion; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class IndexTemplateAPIWrapperFactoryTest { + @Mock + private IndexConfiguration indexConfiguration; + + @Mock + private OpenSearchClient openSearchClient; + + @Test + void testGetEs6IndexTemplateAPIWrapper() { + when(indexConfiguration.getDistributionVersion()).thenReturn(DistributionVersion.ES6); + assertThat(IndexTemplateAPIWrapperFactory.getWrapper(indexConfiguration, openSearchClient), + instanceOf(Es6IndexTemplateAPIWrapper.class)); + } + + @Test + void testGetOpenSearchLegacyTemplateAPIWrapper() { + when(indexConfiguration.getDistributionVersion()).thenReturn(DistributionVersion.DEFAULT); + when(indexConfiguration.getTemplateType()).thenReturn(TemplateType.V1); + assertThat(IndexTemplateAPIWrapperFactory.getWrapper(indexConfiguration, openSearchClient), + instanceOf(OpenSearchLegacyTemplateAPIWrapper.class)); + } + + @Test + void testGetComposableTemplateAPIWrapper() { + when(indexConfiguration.getDistributionVersion()).thenReturn(DistributionVersion.DEFAULT); + when(indexConfiguration.getTemplateType()).thenReturn(TemplateType.INDEX_TEMPLATE); + assertThat(IndexTemplateAPIWrapperFactory.getWrapper(indexConfiguration, openSearchClient), + instanceOf(ComposableTemplateAPIWrapper.class)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/OpenSearchLegacyTemplateAPIWrapperTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/OpenSearchLegacyTemplateAPIWrapperTest.java new file mode 100644 index 0000000000..0a830eb80c --- /dev/null +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/OpenSearchLegacyTemplateAPIWrapperTest.java @@ -0,0 +1,139 @@ +package org.opensearch.dataprepper.plugins.sink.opensearch.index; + +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.Captor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.client.opensearch.OpenSearchClient; +import org.opensearch.client.opensearch.indices.ExistsTemplateRequest; +import org.opensearch.client.opensearch.indices.GetTemplateRequest; +import org.opensearch.client.opensearch.indices.GetTemplateResponse; +import org.opensearch.client.opensearch.indices.OpenSearchIndicesClient; +import org.opensearch.client.opensearch.indices.PutTemplateRequest; +import org.opensearch.client.transport.OpenSearchTransport; +import org.opensearch.client.transport.endpoints.BooleanResponse; +import org.opensearch.dataprepper.plugins.sink.opensearch.bulk.PreSerializedJsonpMapper; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.hasItem; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertAll; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class OpenSearchLegacyTemplateAPIWrapperTest { + @Mock + private OpenSearchClient openSearchClient; + @Mock + private OpenSearchIndicesClient openSearchIndicesClient; + @Mock + private OpenSearchTransport openSearchTransport; + @Mock + private GetTemplateResponse getTemplateResponse; + @Mock + private BooleanResponse booleanResponse; + @Captor + private ArgumentCaptor putTemplateRequestArgumentCaptor; + @Captor + private ArgumentCaptor existsTemplateRequestArgumentCaptor; + + private String templateName; + private OpenSearchLegacyTemplateAPIWrapper objectUnderTest; + private Random random; + + @BeforeEach + void setUp() { + random = new Random(); + objectUnderTest = new OpenSearchLegacyTemplateAPIWrapper(openSearchClient); + lenient().when(openSearchClient.indices()).thenReturn(openSearchIndicesClient); + templateName = UUID.randomUUID().toString(); + } + + @Test + void putTemplate_throws_if_template_is_not_LegacyIndexTemplate() { + final IndexTemplate indexTemplate = mock(IndexTemplate.class); + assertThrows(IllegalArgumentException.class, () -> objectUnderTest.putTemplate( + indexTemplate)); + } + + @Test + void putTemplate_performs_putTemplate_request() throws IOException { + when(openSearchClient._transport()).thenReturn(openSearchTransport); + when(openSearchTransport.jsonpMapper()).thenReturn(new PreSerializedJsonpMapper()); + + final List indexPatterns = Collections.singletonList(UUID.randomUUID().toString()); + final IndexTemplate indexTemplate = new LegacyIndexTemplate(new HashMap<>()); + indexTemplate.setTemplateName(templateName); + indexTemplate.setIndexPatterns(indexPatterns); + objectUnderTest.putTemplate(indexTemplate); + + verify(openSearchIndicesClient).putTemplate(putTemplateRequestArgumentCaptor.capture()); + + final PutTemplateRequest actualPutRequest = putTemplateRequestArgumentCaptor.getValue(); + + assertThat(actualPutRequest.name(), equalTo(templateName)); + assertThat(actualPutRequest.indexPatterns(), equalTo(indexPatterns)); + } + + @Test + void getExistingTemplate_should_calls_existTemplate_with_templateName() throws IOException { + when(booleanResponse.value()).thenReturn(false); + when(openSearchIndicesClient.existsTemplate(any(ExistsTemplateRequest.class))) + .thenReturn(booleanResponse); + + objectUnderTest.getTemplate(templateName); + + verify(openSearchIndicesClient).existsTemplate(existsTemplateRequestArgumentCaptor.capture()); + + final ExistsTemplateRequest actualRequest = existsTemplateRequestArgumentCaptor.getValue(); + assertThat(actualRequest.name(), notNullValue()); + assertAll( + () -> assertThat(actualRequest.name().size(), equalTo(1)), + () -> assertThat(actualRequest.name(), hasItem(templateName)) + ); + } + + @Test + void getExistingTemplate_should_return_empty_if_no_template_exists() throws IOException { + when(booleanResponse.value()).thenReturn(false); + when(openSearchIndicesClient.existsTemplate(any(ExistsTemplateRequest.class))) + .thenReturn(booleanResponse); + + final Optional optionalGetTemplateResponse = objectUnderTest.getTemplate(templateName); + + assertThat(optionalGetTemplateResponse, notNullValue()); + assertThat(optionalGetTemplateResponse.isPresent(), equalTo(false)); + } + + @Test + void getExistingTemplate_should_return_template_if_template_exists() throws IOException { + when(booleanResponse.value()).thenReturn(true); + when(openSearchIndicesClient.existsTemplate(any(ExistsTemplateRequest.class))) + .thenReturn(booleanResponse); + when(openSearchIndicesClient.getTemplate(any(GetTemplateRequest.class))) + .thenReturn(getTemplateResponse); + + final Optional optionalGetTemplateResponse = objectUnderTest.getTemplate(templateName); + + assertThat(optionalGetTemplateResponse, notNullValue()); + assertThat(optionalGetTemplateResponse.isPresent(), equalTo(true)); + assertThat(optionalGetTemplateResponse.get(), equalTo(getTemplateResponse)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/TemplateTypeTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/TemplateTypeTest.java index bc9422ec7d..a27659e29d 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/TemplateTypeTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/TemplateTypeTest.java @@ -13,7 +13,6 @@ import org.junit.jupiter.params.provider.ArgumentsProvider; import org.junit.jupiter.params.provider.ArgumentsSource; import org.junit.jupiter.params.provider.EnumSource; -import org.opensearch.client.opensearch.OpenSearchClient; import java.util.UUID; import java.util.stream.Stream; @@ -27,24 +26,24 @@ import static org.mockito.Mockito.mock; class TemplateTypeTest { - private OpenSearchClient openSearchClient; + private IndexTemplateAPIWrapper indexTemplateAPIWrapper; @BeforeEach void setUp() { - openSearchClient = mock(OpenSearchClient.class); + indexTemplateAPIWrapper = mock(IndexTemplateAPIWrapper.class); } @ParameterizedTest @ArgumentsSource(EnumToStrategyClass.class) void createTemplateStrategy_returns_instance_of_expected_type(final TemplateType objectUnderTest, final Class expectedStrategyClass) { - assertThat(objectUnderTest.createTemplateStrategy(openSearchClient), + assertThat(objectUnderTest.createTemplateStrategy(indexTemplateAPIWrapper), instanceOf(expectedStrategyClass)); } @ParameterizedTest @EnumSource(TemplateType.class) void createTemplateStrategy_returns_for_all_enum_types(final TemplateType objectUnderTest) { - assertThat(objectUnderTest.createTemplateStrategy(openSearchClient), notNullValue()); + assertThat(objectUnderTest.createTemplateStrategy(indexTemplateAPIWrapper), notNullValue()); } @ParameterizedTest diff --git a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/V1TemplateStrategyTest.java b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/V1TemplateStrategyTest.java index 53b89cf4a7..911bd17600 100644 --- a/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/V1TemplateStrategyTest.java +++ b/data-prepper-plugins/opensearch/src/test/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/V1TemplateStrategyTest.java @@ -11,19 +11,10 @@ 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.junit.jupiter.MockitoExtension; -import org.opensearch.client.opensearch.OpenSearchClient; -import org.opensearch.client.opensearch.indices.ExistsTemplateRequest; -import org.opensearch.client.opensearch.indices.GetTemplateRequest; import org.opensearch.client.opensearch.indices.GetTemplateResponse; -import org.opensearch.client.opensearch.indices.OpenSearchIndicesClient; -import org.opensearch.client.opensearch.indices.PutTemplateRequest; import org.opensearch.client.opensearch.indices.TemplateMapping; -import org.opensearch.client.transport.OpenSearchTransport; -import org.opensearch.client.transport.endpoints.BooleanResponse; -import org.opensearch.dataprepper.plugins.sink.opensearch.bulk.PreSerializedJsonpMapper; import java.io.IOException; import java.util.Collections; @@ -35,16 +26,16 @@ import java.util.UUID; 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.not; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.hasKey; -import static org.junit.jupiter.api.Assertions.assertAll; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.lenient; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -52,75 +43,47 @@ @ExtendWith(MockitoExtension.class) class V1TemplateStrategyTest { @Mock - private OpenSearchClient openSearchClient; - + private IndexTemplateAPIWrapper indexTemplateAPIWrapper; + @Mock + private GetTemplateResponse getTemplateResponse; @Mock - private OpenSearchIndicesClient openSearchIndicesClient; + private TemplateMapping templateMapping; private Random random; private String templateName; @BeforeEach void setUp() { random = new Random(); - lenient().when(openSearchClient.indices()).thenReturn(openSearchIndicesClient); templateName = UUID.randomUUID().toString(); } private V1TemplateStrategy createObjectUnderTest() { - return new V1TemplateStrategy(openSearchClient); + return new V1TemplateStrategy(indexTemplateAPIWrapper); } @Test - void getExistingTemplateVersion_should_calls_existTemplate_with_templateName() throws IOException { - final BooleanResponse booleanResponse = mock(BooleanResponse.class); - when(booleanResponse.value()).thenReturn(false); - when(openSearchIndicesClient.existsTemplate(any(ExistsTemplateRequest.class))) - .thenReturn(booleanResponse); - + void getExistingTemplateVersion_should_calls_getTemplate_with_templateName() throws IOException { + when(indexTemplateAPIWrapper.getTemplate(any())).thenReturn(Optional.of(getTemplateResponse)); + when(getTemplateResponse.result()).thenReturn(Map.of(templateName, templateMapping)); createObjectUnderTest().getExistingTemplateVersion(templateName); - final ArgumentCaptor existsTemplateRequestArgumentCaptor = ArgumentCaptor.forClass(ExistsTemplateRequest.class); - verify(openSearchIndicesClient).existsTemplate(existsTemplateRequestArgumentCaptor.capture()); - - final ExistsTemplateRequest actualRequest = existsTemplateRequestArgumentCaptor.getValue(); - assertThat(actualRequest.name(), notNullValue()); - assertAll( - () -> assertThat(actualRequest.name().size(), equalTo(1)), - () -> assertThat(actualRequest.name(), hasItem(templateName)) - ); + verify(indexTemplateAPIWrapper).getTemplate(eq(templateName)); + verify(templateMapping).version(); } @Test void getExistingTemplateVersion_should_return_empty_if_no_template_exists() throws IOException { - final BooleanResponse booleanResponse = mock(BooleanResponse.class); - when(booleanResponse.value()).thenReturn(false); - when(openSearchIndicesClient.existsTemplate(any(ExistsTemplateRequest.class))) - .thenReturn(booleanResponse); - - final Optional optionalVersion = createObjectUnderTest().getExistingTemplateVersion(templateName); - - assertThat(optionalVersion, notNullValue()); - assertThat(optionalVersion.isPresent(), equalTo(false)); + final Optional version = createObjectUnderTest().getExistingTemplateVersion(templateName); + assertThat(version.isEmpty(), is(true)); } @Nested class WithExistingTemplate { - @BeforeEach - void setUp() throws IOException { - final BooleanResponse booleanResponse = mock(BooleanResponse.class); - when(booleanResponse.value()).thenReturn(true); - when(openSearchIndicesClient.existsTemplate(any(ExistsTemplateRequest.class))) - .thenReturn(booleanResponse); - } - @Test void getExistingTemplateVersion_should_return_empty_if_template_exists_without_version() throws IOException { - final GetTemplateResponse getTemplateResponse = mock(GetTemplateResponse.class); - final TemplateMapping templateMapping = mock(TemplateMapping.class); + when(indexTemplateAPIWrapper.getTemplate(any())).thenReturn(Optional.of(getTemplateResponse)); + when(getTemplateResponse.result()).thenReturn(Map.of(templateName, templateMapping)); when(templateMapping.version()).thenReturn(null); - when(getTemplateResponse.result()).thenReturn(Collections.singletonMap(templateName, templateMapping)); - when(openSearchIndicesClient.getTemplate(any(GetTemplateRequest.class))) - .thenReturn(getTemplateResponse); final Optional optionalVersion = createObjectUnderTest().getExistingTemplateVersion(templateName); @@ -130,13 +93,10 @@ void getExistingTemplateVersion_should_return_empty_if_template_exists_without_v @Test void getExistingTemplateVersion_should_return_template_version_if_template_exists() throws IOException { - final GetTemplateResponse getTemplateResponse = mock(GetTemplateResponse.class); - final TemplateMapping templateMapping = mock(TemplateMapping.class); final Long version = (long) (random.nextInt(10_000) + 100); + when(indexTemplateAPIWrapper.getTemplate(any())).thenReturn(Optional.of(getTemplateResponse)); + when(getTemplateResponse.result()).thenReturn(Map.of(templateName, templateMapping)); when(templateMapping.version()).thenReturn(version); - when(getTemplateResponse.result()).thenReturn(Collections.singletonMap(templateName, templateMapping)); - when(openSearchIndicesClient.getTemplate(any(GetTemplateRequest.class))) - .thenReturn(getTemplateResponse); final Optional optionalVersion = createObjectUnderTest().getExistingTemplateVersion(templateName); @@ -147,49 +107,36 @@ void getExistingTemplateVersion_should_return_template_version_if_template_exist @ParameterizedTest @ValueSource(ints = {0, 2}) - void getExistingTemplateVersion_should_throw_if_get_template_returns_unexpected_number_of_templates(final int numberOfTemplatesReturned) throws IOException { - final GetTemplateResponse getTemplateResponse = mock(GetTemplateResponse.class); + void getExistingTemplateVersion_should_throw_if_get_template_returns_unexpected_number_of_templates( + final int numberOfTemplatesReturned) throws IOException { + when(indexTemplateAPIWrapper.getTemplate(any())).thenReturn(Optional.of(getTemplateResponse)); final Map templateResult = mock(Map.class); when(templateResult.size()).thenReturn(numberOfTemplatesReturned); when(getTemplateResponse.result()).thenReturn(templateResult); - when(openSearchIndicesClient.getTemplate(any(GetTemplateRequest.class))) - .thenReturn(getTemplateResponse); - final V1TemplateStrategy objectUnderTest = createObjectUnderTest(); - assertThrows(RuntimeException.class, () -> objectUnderTest.getExistingTemplateVersion(templateName)); + assertThrows(RuntimeException.class, + () -> createObjectUnderTest().getExistingTemplateVersion(templateName)); - verify(openSearchIndicesClient).getTemplate(any(GetTemplateRequest.class)); + verify(indexTemplateAPIWrapper).getTemplate(eq(templateName)); } } @Test - void createTemplate_throws_if_template_is_not_LegacyIndexTemplate() { + void createTemplate_throws_if_putTemplate_throws() throws IOException { + doThrow(IOException.class).when(indexTemplateAPIWrapper).putTemplate(any()); final IndexTemplate indexTemplate = mock(IndexTemplate.class); final V1TemplateStrategy objectUnderTest = createObjectUnderTest(); - assertThrows(IllegalArgumentException.class, () -> objectUnderTest.createTemplate(indexTemplate)); + assertThrows(IOException.class, () -> objectUnderTest.createTemplate(indexTemplate)); } @Test void createTemplate_performs_putTemplate_request() throws IOException { - final OpenSearchTransport openSearchTransport = mock(OpenSearchTransport.class); - when(openSearchClient._transport()).thenReturn(openSearchTransport); - when(openSearchTransport.jsonpMapper()).thenReturn(new PreSerializedJsonpMapper()); final V1TemplateStrategy objectUnderTest = createObjectUnderTest(); + final IndexTemplate indexTemplate = mock(IndexTemplate.class); - final List indexPatterns = Collections.singletonList(UUID.randomUUID().toString()); - final IndexTemplate indexTemplate = objectUnderTest.createIndexTemplate(new HashMap<>()); - indexTemplate.setTemplateName(templateName); - indexTemplate.setIndexPatterns(indexPatterns); objectUnderTest.createTemplate(indexTemplate); - - final ArgumentCaptor putTemplateRequestArgumentCaptor = ArgumentCaptor.forClass(PutTemplateRequest.class); - verify(openSearchIndicesClient).putTemplate(putTemplateRequestArgumentCaptor.capture()); - - final PutTemplateRequest actualPutRequest = putTemplateRequestArgumentCaptor.getValue(); - - assertThat(actualPutRequest.name(), equalTo(templateName)); - assertThat(actualPutRequest.indexPatterns(), equalTo(indexPatterns)); + verify(indexTemplateAPIWrapper).putTemplate(indexTemplate); } @Nested @@ -205,11 +152,11 @@ void setUp() { void createIndexTemplate_setTemplateName_sets_the_name() { final IndexTemplate indexTemplate = createObjectUnderTest().createIndexTemplate(providedTemplateMap); - assertThat(indexTemplate, instanceOf(V1TemplateStrategy.LegacyIndexTemplate.class)); + assertThat(indexTemplate, instanceOf(LegacyIndexTemplate.class)); indexTemplate.setTemplateName(templateName); - final Map returnedTemplateMap = ((V1TemplateStrategy.LegacyIndexTemplate) indexTemplate).getTemplateMap(); + final Map returnedTemplateMap = ((LegacyIndexTemplate) indexTemplate).getTemplateMap(); assertThat(returnedTemplateMap, hasKey("name")); assertThat(returnedTemplateMap.get("name"), equalTo(templateName)); @@ -220,12 +167,12 @@ void createIndexTemplate_setTemplateName_sets_the_name() { void createIndexTemplate_setIndexPatterns_sets_the_indexPatterns() { final IndexTemplate indexTemplate = createObjectUnderTest().createIndexTemplate(providedTemplateMap); - assertThat(indexTemplate, instanceOf(V1TemplateStrategy.LegacyIndexTemplate.class)); + assertThat(indexTemplate, instanceOf(LegacyIndexTemplate.class)); final List indexPatterns = Collections.singletonList(UUID.randomUUID().toString()); indexTemplate.setIndexPatterns(indexPatterns); - final Map returnedTemplateMap = ((V1TemplateStrategy.LegacyIndexTemplate) indexTemplate).getTemplateMap(); + final Map returnedTemplateMap = ((LegacyIndexTemplate) indexTemplate).getTemplateMap(); assertThat(returnedTemplateMap, hasKey("index_patterns")); assertThat(returnedTemplateMap.get("index_patterns"), equalTo(indexPatterns)); @@ -240,14 +187,14 @@ void putCustomSetting_setIndexPatterns_sets_existing_settings() { providedTemplateMap.put("settings", providedSettings); final IndexTemplate indexTemplate = createObjectUnderTest().createIndexTemplate(providedTemplateMap); - assertThat(indexTemplate, instanceOf(V1TemplateStrategy.LegacyIndexTemplate.class)); + assertThat(indexTemplate, instanceOf(LegacyIndexTemplate.class)); final String customKey = UUID.randomUUID().toString(); final String customValue = UUID.randomUUID().toString(); indexTemplate.putCustomSetting(customKey, customValue); - final Map returnedTemplateMap = ((V1TemplateStrategy.LegacyIndexTemplate) indexTemplate).getTemplateMap(); + final Map returnedTemplateMap = ((LegacyIndexTemplate) indexTemplate).getTemplateMap(); assertThat(returnedTemplateMap, hasKey("settings")); assertThat(returnedTemplateMap.get("settings"), instanceOf(Map.class)); final Map settings = (Map) returnedTemplateMap.get("settings"); @@ -263,13 +210,13 @@ void putCustomSetting_setIndexPatterns_sets_existing_settings() { void putCustomSetting_setIndexPatterns_sets_new_settings() { final IndexTemplate indexTemplate = createObjectUnderTest().createIndexTemplate(providedTemplateMap); - assertThat(indexTemplate, instanceOf(V1TemplateStrategy.LegacyIndexTemplate.class)); + assertThat(indexTemplate, instanceOf(LegacyIndexTemplate.class)); final String customKey = UUID.randomUUID().toString(); final String customValue = UUID.randomUUID().toString(); indexTemplate.putCustomSetting(customKey, customValue); - final Map returnedTemplateMap = ((V1TemplateStrategy.LegacyIndexTemplate) indexTemplate).getTemplateMap(); + final Map returnedTemplateMap = ((LegacyIndexTemplate) indexTemplate).getTemplateMap(); assertThat(returnedTemplateMap, hasKey("settings")); assertThat(returnedTemplateMap.get("settings"), instanceOf(Map.class)); final Map settings = (Map) returnedTemplateMap.get("settings"); From fbfb82ebe8896bbd77f8008853474072ee62d2be Mon Sep 17 00:00:00 2001 From: mallikagogoi7 Date: Fri, 4 Aug 2023 20:40:49 +0530 Subject: [PATCH 7/7] Prometheus Sink Boiler plate code for issue #1744. (#3078) * Prometheus Sink boiler plate code for issue #1744. Signed-off-by: mallikagogoi7 * Prometheus Sink Fix for issue #1744. Signed-off-by: mallikagogoi7 * Prometheus Sink review comment rsolved for issue #1744. Signed-off-by: mallikagogoi7 --- .../prometheus-sink/build.gradle | 24 ++ .../FailedHttpResponseInterceptor.java | 26 ++ .../sink/prometheus/HttpEndPointResponse.java | 46 ++++ .../prometheus/OAuthAccessTokenManager.java | 20 ++ .../sink/prometheus/PrometheusSink.java | 82 ++++++ .../configuration/AuthTypeOptions.java | 34 +++ .../configuration/AuthenticationOptions.java | 24 ++ .../AwsAuthenticationOptions.java | 45 ++++ .../configuration/BasicAuthCredentials.java | 25 ++ .../configuration/BearerTokenOptions.java | 61 +++++ .../configuration/HTTPMethodOptions.java | 33 +++ .../PrometheusSinkConfiguration.java | 252 ++++++++++++++++++ .../configuration/ThresholdOptions.java | 60 +++++ .../BasicAuthPrometheusSinkHandler.java | 33 +++ .../BearerTokenAuthPrometheusSinkHandler.java | 44 +++ .../prometheus/handler/HttpAuthOptions.java | 111 ++++++++ .../MultiAuthPrometheusSinkHandler.java | 19 ++ .../service/PrometheusSinkAwsService.java | 19 ++ .../service/PrometheusSinkService.java | 52 ++++ settings.gradle | 1 + 20 files changed, 1011 insertions(+) create mode 100644 data-prepper-plugins/prometheus-sink/build.gradle create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/HttpEndPointResponse.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BasicAuthCredentials.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/HTTPMethodOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/ThresholdOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandler.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandler.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/HttpAuthOptions.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/MultiAuthPrometheusSinkHandler.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java create mode 100644 data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java diff --git a/data-prepper-plugins/prometheus-sink/build.gradle b/data-prepper-plugins/prometheus-sink/build.gradle new file mode 100644 index 0000000000..1ce6d4684e --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/build.gradle @@ -0,0 +1,24 @@ + +dependencies { + implementation project(':data-prepper-api') + implementation project(path: ':data-prepper-plugins:common') + implementation 'com.arpnetworking.metrics:prometheus-remote-protocol:1.0.1' + 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.apache.commons:commons-compress:1.21' + implementation project(':data-prepper-plugins:common') + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'org.hibernate.validator:hibernate-validator:7.0.5.Final' + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + implementation 'software.amazon.awssdk:auth' + implementation 'org.apache.commons:commons-lang3:3.12.0' + implementation project(':data-prepper-plugins:failures-common') + implementation 'org.apache.httpcomponents.client5:httpclient5:5.2' + testImplementation project(':data-prepper-test-common') +} + +test { + useJUnitPlatform() +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java new file mode 100644 index 0000000000..f8b43302a6 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/FailedHttpResponseInterceptor.java @@ -0,0 +1,26 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +import org.apache.hc.core5.http.EntityDetails; +import org.apache.hc.core5.http.HttpResponse; +import org.apache.hc.core5.http.HttpResponseInterceptor; +import org.apache.hc.core5.http.protocol.HttpContext; + +import java.io.IOException; + +public class FailedHttpResponseInterceptor implements HttpResponseInterceptor { + + private final String url; + + public FailedHttpResponseInterceptor(final String url){ + this.url = url; + } + + @Override + public void process(HttpResponse response, EntityDetails entity, HttpContext context) throws IOException { + //TODO: implementation + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/HttpEndPointResponse.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/HttpEndPointResponse.java new file mode 100644 index 0000000000..346ff80d38 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/HttpEndPointResponse.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +public class HttpEndPointResponse { + private String url; + private int statusCode; + private String errorMessage; + + public HttpEndPointResponse(final String url, + final int statusCode, + final String errorMessage) { + this.url = url; + this.statusCode = statusCode; + this.errorMessage = errorMessage; + } + + public HttpEndPointResponse(final String url, + final int statusCode) { + this.url = url; + this.statusCode = statusCode; + } + + public String getUrl() { + return url; + } + + public int getStatusCode() { + return statusCode; + } + + public String getErrorMessage() { + return errorMessage; + } + + @Override + public String toString() { + return "{" + + "url='" + url + '\'' + + ", statusCode=" + statusCode + + ", errorMessage='" + errorMessage + '\'' + + '}'; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java new file mode 100644 index 0000000000..0cdfb1b34d --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/OAuthAccessTokenManager.java @@ -0,0 +1,20 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.BearerTokenOptions; + +public class OAuthAccessTokenManager { + + public String getAccessToken(final BearerTokenOptions bearerTokenOptions) { + //TODO implementation + return null; + } + + public boolean isTokenExpired(final String token){ + //TODO implementation + return false; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java new file mode 100644 index 0000000000..3bca042c60 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/PrometheusSink.java @@ -0,0 +1,82 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus; + +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.PipelineDescription; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.sink.AbstractSink; +import org.opensearch.dataprepper.model.sink.Sink; + +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; +import org.opensearch.dataprepper.plugins.sink.prometheus.service.PrometheusSinkService; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; + +@DataPrepperPlugin(name = "prometheus", pluginType = Sink.class, pluginConfigurationType = PrometheusSinkConfiguration.class) +public class PrometheusSink extends AbstractSink> { + + private static final Logger LOG = LoggerFactory.getLogger(PrometheusSink.class); + + private volatile boolean sinkInitialized; + + private final PrometheusSinkService prometheusSinkService; + + @DataPrepperPluginConstructor + public PrometheusSink(final PluginSetting pluginSetting, + final PrometheusSinkConfiguration prometheusSinkConfiguration, + final PluginFactory pluginFactory, + final PipelineDescription pipelineDescription, + final AwsCredentialsSupplier awsCredentialsSupplier) { + super(pluginSetting); + this.sinkInitialized = Boolean.FALSE; + this.prometheusSinkService = new PrometheusSinkService( + prometheusSinkConfiguration); + } + + @Override + public boolean isReady() { + return sinkInitialized; + } + + @Override + public void doInitialize() { + try { + doInitializeInternal(); + } catch (InvalidPluginConfigurationException e) { + LOG.error("Invalid plugin configuration, Hence failed to initialize prometheus-sink plugin."); + this.shutdown(); + throw e; + } catch (Exception e) { + LOG.error("Failed to initialize prometheus-sink 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; + } + prometheusSinkService.output(records); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java new file mode 100644 index 0000000000..c643908ea5 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthTypeOptions.java @@ -0,0 +1,34 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +public enum AuthTypeOptions { + HTTP_BASIC("http-basic"), + BEARER_TOKEN("bearer-token"), + UNAUTHENTICATED("unauthenticated"); + + private static final Map OPTIONS_MAP = Arrays.stream(AuthTypeOptions.values()) + .collect(Collectors.toMap( + value -> value.option, + value -> value + )); + + private final String option; + + AuthTypeOptions(final String option) { + this.option = option; + } + + @JsonCreator + static AuthTypeOptions fromOptionValue(final String option) { + return OPTIONS_MAP.get(option); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java new file mode 100644 index 0000000000..e5bfa60534 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AuthenticationOptions.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class AuthenticationOptions { + + @JsonProperty("http_basic") + private BasicAuthCredentials httpBasic; + + @JsonProperty("bearer_token") + private BearerTokenOptions bearerTokenOptions; + + public BasicAuthCredentials getHttpBasic() { + return httpBasic; + } + + public BearerTokenOptions getBearerTokenOptions() { + return bearerTokenOptions; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java new file mode 100644 index 0000000000..7dd94a243c --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/AwsAuthenticationOptions.java @@ -0,0 +1,45 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +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/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BasicAuthCredentials.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BasicAuthCredentials.java new file mode 100644 index 0000000000..9abc1f80d8 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BasicAuthCredentials.java @@ -0,0 +1,25 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class BasicAuthCredentials { + + + @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/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java new file mode 100644 index 0000000000..3ff05da0b8 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/BearerTokenOptions.java @@ -0,0 +1,61 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotNull; + +public class BearerTokenOptions { + + @JsonProperty("client_id") + @NotNull(message = "client id type is mandatory for refresh token") + private String clientId; + + @JsonProperty("client_secret") + @NotNull(message = "client secret type is mandatory for refresh token") + private String clientSecret; + + @JsonProperty("token_url") + @NotNull(message = "token url type is mandatory for refresh token") + private String tokenURL; + + @JsonProperty("grant_type") + @NotNull(message = "grant type is mandatory for refresh token") + private String grantType; + + @JsonProperty("scope") + @NotNull(message = "scope is mandatory for refresh token") + private String scope; + + private String refreshToken; + + public String getScope() { + return scope; + } + + public String getGrantType() { + return grantType; + } + + public String getRefreshToken() { + return refreshToken; + } + + public String getClientId() { + return clientId; + } + + public String getClientSecret() { + return clientSecret; + } + + public String getTokenURL() { + return tokenURL; + } + + public void setRefreshToken(String refreshToken) { + this.refreshToken = refreshToken; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/HTTPMethodOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/HTTPMethodOptions.java new file mode 100644 index 0000000000..0f6c13a853 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/HTTPMethodOptions.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +public enum HTTPMethodOptions { + PUT("PUT"), + POST("POST"); + + private static final Map OPTIONS_MAP = Arrays.stream(HTTPMethodOptions.values()) + .collect(Collectors.toMap( + value -> value.option, + value -> value + )); + + private final String option; + + HTTPMethodOptions(final String option) { + this.option = option; + } + + @JsonCreator + static HTTPMethodOptions fromOptionValue(final String option) { + return OPTIONS_MAP.get(option); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java new file mode 100644 index 0000000000..63724c47ed --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/PrometheusSinkConfiguration.java @@ -0,0 +1,252 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import org.apache.commons.lang3.StringUtils; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.plugins.accumulator.BufferTypeOptions; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +public class PrometheusSinkConfiguration { + + private static final int DEFAULT_UPLOAD_RETRIES = 5; + + static final boolean DEFAULT_SSL = false; + + private static final String S3_PREFIX = "s3://"; + + static final String SSL_KEY_CERT_FILE = "sslKeyCertChainFile"; + static final String SSL_KEY_FILE = "sslKeyFile"; + static final String SSL = "ssl"; + static final String AWS_REGION = "awsRegion"; + + + public static final String STS_REGION = "sts_region"; + + public static final String STS_ROLE_ARN = "sts_role_arn"; + static final boolean DEFAULT_USE_ACM_CERT_FOR_SSL = false; + static final int DEFAULT_ACM_CERT_ISSUE_TIME_OUT_MILLIS = 120000; + public static final String SSL_IS_ENABLED = "%s is enabled"; + + public static final Duration DEFAULT_HTTP_RETRY_INTERVAL = Duration.ofSeconds(30); + + @NotNull + @JsonProperty("url") + private String url; + + @JsonProperty("codec") + private PluginModel codec; + + @JsonProperty("http_method") + private HTTPMethodOptions httpMethod = HTTPMethodOptions.POST; + + @JsonProperty("proxy") + private String proxy; + + @JsonProperty("auth_type") + private AuthTypeOptions authType = AuthTypeOptions.UNAUTHENTICATED; + + @JsonProperty("authentication") + private AuthenticationOptions authentication; + + @JsonProperty("ssl_certificate_file") + private String sslCertificateFile; + + @JsonProperty("ssl_key_file") + private String sslKeyFile; + + @JsonProperty("aws_sigv4") + private boolean awsSigv4; + + @JsonProperty("buffer_type") + private BufferTypeOptions bufferType = BufferTypeOptions.INMEMORY; + + @NotNull + @JsonProperty("threshold") + private ThresholdOptions thresholdOptions; + + @JsonProperty("max_retries") + private int maxUploadRetries = DEFAULT_UPLOAD_RETRIES; + + @JsonProperty("aws") + @Valid + private AwsAuthenticationOptions awsAuthenticationOptions; + + @JsonProperty("custom_header") + private Map> customHeaderOptions; + + @JsonProperty("dlq_file") + private String dlqFile; + + @JsonProperty("dlq") + private PluginModel dlq; + + @JsonProperty("use_acm_cert_for_ssl") + private boolean useAcmCertForSSL = DEFAULT_USE_ACM_CERT_FOR_SSL; + + @JsonProperty("acm_private_key_password") + private String acmPrivateKeyPassword; + + @JsonProperty("acm_certificate_arn") + private String acmCertificateArn; + + @JsonProperty("acm_cert_issue_time_out_millis") + private long acmCertIssueTimeOutMillis = DEFAULT_ACM_CERT_ISSUE_TIME_OUT_MILLIS; + + @JsonProperty("ssl") + private boolean ssl = DEFAULT_SSL; + + @JsonProperty("http_retry_interval") + private Duration httpRetryInterval = DEFAULT_HTTP_RETRY_INTERVAL; + + + private boolean sslCertAndKeyFileInS3; + + public String getUrl() { + return url; + } + + public boolean isSsl() { + return ssl; + } + + public Duration getHttpRetryInterval() { + return httpRetryInterval; + } + + public String getAcmPrivateKeyPassword() { + return acmPrivateKeyPassword; + } + + public boolean isSslCertAndKeyFileInS3() { + return sslCertAndKeyFileInS3; + } + + public long getAcmCertIssueTimeOutMillis() { + return acmCertIssueTimeOutMillis; + } + + public boolean useAcmCertForSSL() { + return useAcmCertForSSL; + } + + public void validateAndInitializeCertAndKeyFileInS3() { + boolean certAndKeyFileInS3 = false; + if (useAcmCertForSSL) { + validateSSLArgument(String.format(SSL_IS_ENABLED, useAcmCertForSSL), acmCertificateArn, acmCertificateArn); + validateSSLArgument(String.format(SSL_IS_ENABLED, useAcmCertForSSL), awsAuthenticationOptions.getAwsRegion().toString(), AWS_REGION); + } else if(ssl) { + validateSSLCertificateFiles(); + certAndKeyFileInS3 = isSSLCertificateLocatedInS3(); + if (certAndKeyFileInS3) { + validateSSLArgument("The certificate and key files are located in S3", awsAuthenticationOptions.getAwsRegion().toString(), AWS_REGION); + } + } + sslCertAndKeyFileInS3 = certAndKeyFileInS3; + } + private void validateSSLArgument(final String sslTypeMessage, final String argument, final String argumentName) { + if (StringUtils.isEmpty(argument)) { + throw new IllegalArgumentException(String.format("%s, %s can not be empty or null", sslTypeMessage, argumentName)); + } + } + + private void validateSSLCertificateFiles() { + validateSSLArgument(String.format(SSL_IS_ENABLED, SSL), sslCertificateFile, SSL_KEY_CERT_FILE); + validateSSLArgument(String.format(SSL_IS_ENABLED, SSL), sslKeyFile, SSL_KEY_FILE); + } + + private boolean isSSLCertificateLocatedInS3() { + return sslCertificateFile.toLowerCase().startsWith(S3_PREFIX) && + sslKeyFile.toLowerCase().startsWith(S3_PREFIX); + } + + public String getAcmCertificateArn() { + return acmCertificateArn; + } + + public PluginModel getCodec() { + return codec; + } + + public HTTPMethodOptions getHttpMethod() { + return httpMethod; + } + + public String getProxy() { + return proxy; + } + + public AuthTypeOptions getAuthType() { + return authType; + } + + public AuthenticationOptions getAuthentication() { + return authentication; + } + + public String getSslCertificateFile() { + return sslCertificateFile; + } + + public String getSslKeyFile() { + return sslKeyFile; + } + + public boolean isAwsSigv4() { + return awsSigv4; + } + + public BufferTypeOptions getBufferType() { + return bufferType; + } + + public ThresholdOptions getThresholdOptions() { + return thresholdOptions; + } + + public int getMaxUploadRetries() { + return maxUploadRetries; + } + + public Map> getCustomHeaderOptions() { + return customHeaderOptions; + } + + public AwsAuthenticationOptions getAwsAuthenticationOptions() { + return awsAuthenticationOptions; + } + + public String getDlqFile() { + return dlqFile; + } + + 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(); + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/ThresholdOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/ThresholdOptions.java new file mode 100644 index 0000000000..b9886f6ce1 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/configuration/ThresholdOptions.java @@ -0,0 +1,60 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.configuration; + +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; + +/** + * An implementation class of http index configuration Options + */ +public class ThresholdOptions { + + private static final String DEFAULT_BYTE_CAPACITY = "50mb"; + + @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/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandler.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandler.java new file mode 100644 index 0000000000..603c8b10d7 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BasicAuthPrometheusSinkHandler.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +import org.apache.hc.client5.http.io.HttpClientConnectionManager; + +/** + * * This class handles Basic Authentication + */ +public class BasicAuthPrometheusSinkHandler implements MultiAuthPrometheusSinkHandler { + + private final HttpClientConnectionManager httpClientConnectionManager; + + private final String username; + + private final String password; + + public BasicAuthPrometheusSinkHandler(final String username, + final String password, + final HttpClientConnectionManager httpClientConnectionManager){ + this.httpClientConnectionManager = httpClientConnectionManager; + this.username = username; + this.password = password; + } + + @Override + public HttpAuthOptions authenticate(final HttpAuthOptions.Builder httpAuthOptionsBuilder) { + //TODO: implementation + return null; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandler.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandler.java new file mode 100644 index 0000000000..3b768dd091 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/BearerTokenAuthPrometheusSinkHandler.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.opensearch.dataprepper.plugins.sink.prometheus.OAuthAccessTokenManager; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.BearerTokenOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * * This class handles Bearer Token Authentication + */ +public class BearerTokenAuthPrometheusSinkHandler implements MultiAuthPrometheusSinkHandler { + + private static final Logger LOG = LoggerFactory.getLogger(BearerTokenAuthPrometheusSinkHandler.class); + + private final HttpClientConnectionManager httpClientConnectionManager; + + private final BearerTokenOptions bearerTokenOptions; + + private final ObjectMapper objectMapper; + + private OAuthAccessTokenManager oAuthAccessTokenManager; + + public BearerTokenAuthPrometheusSinkHandler(final BearerTokenOptions bearerTokenOptions, + final HttpClientConnectionManager httpClientConnectionManager, + final OAuthAccessTokenManager oAuthAccessTokenManager){ + this.bearerTokenOptions = bearerTokenOptions; + this.httpClientConnectionManager = httpClientConnectionManager; + this.objectMapper = new ObjectMapper(); + this.oAuthAccessTokenManager = oAuthAccessTokenManager; + } + + @Override + public HttpAuthOptions authenticate(final HttpAuthOptions.Builder httpAuthOptionsBuilder) { + //TODO: implementation + return null; + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/HttpAuthOptions.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/HttpAuthOptions.java new file mode 100644 index 0000000000..843eeb5c90 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/HttpAuthOptions.java @@ -0,0 +1,111 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.apache.hc.client5.http.io.HttpClientConnectionManager; +import org.apache.hc.core5.http.HttpHost; +import org.apache.hc.core5.http.io.support.ClassicRequestBuilder; + + +public class HttpAuthOptions { + + private String url; + + private HttpClientBuilder httpClientBuilder; + + private ClassicRequestBuilder classicHttpRequestBuilder; + + private HttpClientConnectionManager httpClientConnectionManager; + + private int workers; + + private HttpHost proxy; + + public HttpClientBuilder getHttpClientBuilder() { + return httpClientBuilder; + } + + public ClassicRequestBuilder getClassicHttpRequestBuilder() { + return classicHttpRequestBuilder; + } + + public int getWorkers() { + return workers; + } + + public String getUrl() { + return url; + } + + public HttpHost getProxy() { + return proxy; + } + + public HttpClientConnectionManager getHttpClientConnectionManager() { + return httpClientConnectionManager; + } + + private HttpAuthOptions(Builder builder) { + this.url = builder.url; + this.httpClientBuilder = builder.httpClientBuilder; + this.classicHttpRequestBuilder = builder.classicHttpRequestBuilder; + this.httpClientConnectionManager = builder.httpClientConnectionManager; + this.workers = builder.workers; + this.proxy = builder.proxy; + } + public static class Builder { + + private String url; + private HttpClientBuilder httpClientBuilder; + private ClassicRequestBuilder classicHttpRequestBuilder; + private HttpClientConnectionManager httpClientConnectionManager; + private int workers; + private HttpHost proxy; + + public HttpAuthOptions build() { + return new HttpAuthOptions(this); + } + + public Builder setUrl(String url) { + this.url = url; + return this; + } + + public String getUrl() { + return url; + } + + public Builder setHttpClientBuilder(HttpClientBuilder httpClientBuilder) { + this.httpClientBuilder = httpClientBuilder; + return this; + } + + public Builder setClassicHttpRequestBuilder(ClassicRequestBuilder classicHttpRequestBuilder) { + this.classicHttpRequestBuilder = classicHttpRequestBuilder; + return this; + } + + public Builder setHttpClientConnectionManager(HttpClientConnectionManager httpClientConnectionManager) { + this.httpClientConnectionManager = httpClientConnectionManager; + return this; + } + + public Builder setWorkers(int workers) { + this.workers = workers; + return this; + } + + public Builder setProxy(HttpHost proxy) { + this.proxy = proxy; + return this; + } + + public ClassicRequestBuilder getClassicHttpRequestBuilder() { + return classicHttpRequestBuilder; + } + } + +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/MultiAuthPrometheusSinkHandler.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/MultiAuthPrometheusSinkHandler.java new file mode 100644 index 0000000000..fa651a08d4 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/handler/MultiAuthPrometheusSinkHandler.java @@ -0,0 +1,19 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.handler; + +/** + * An interface to handle multiple authentications + */ +public interface MultiAuthPrometheusSinkHandler { + + /** + * This method can be used to implement multiple authentication based on configuration + * @param httpAuthOptionsBuilder HttpAuthOptions.Builder + * @return HttpAuthOptions + */ + HttpAuthOptions authenticate(final HttpAuthOptions.Builder httpAuthOptionsBuilder); + +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java new file mode 100644 index 0000000000..3474968831 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkAwsService.java @@ -0,0 +1,19 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.service; + +import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; + +/** + * This service class contains logic for handling AWS related services + */ +public class PrometheusSinkAwsService { + + public static void attachSigV4(final PrometheusSinkConfiguration prometheusSinkConfiguration, final HttpClientBuilder httpClientBuilder, final AwsCredentialsSupplier awsCredentialsSupplier) { + // TODO: implementation + } +} diff --git a/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java new file mode 100644 index 0000000000..cdb047d7a6 --- /dev/null +++ b/data-prepper-plugins/prometheus-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/prometheus/service/PrometheusSinkService.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.sink.prometheus.service; + +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.sink.prometheus.configuration.PrometheusSinkConfiguration; + +import java.util.Collection; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +/** + * This service class contains logic for sending data to Prometheus Endpoints + */ +public class PrometheusSinkService { + + private final Lock reentrantLock; + + private final PrometheusSinkConfiguration prometheusSinkConfiguration; + + public PrometheusSinkService(final PrometheusSinkConfiguration prometheusSinkConfiguration){ + this.prometheusSinkConfiguration = prometheusSinkConfiguration; + this.reentrantLock = new ReentrantLock(); + } + + /** + * This method process buffer records and send to Prometheus End points based on configured codec + * @param records Collection of Event + */ + public void output(Collection> records) { + reentrantLock.lock(); + try { + records.forEach(record -> { + final Event event = record.getData(); + + //TODO: call buildRemoteWriteRequest() + //TODO: compress Remote.WriteRequest to byte[] + //TODO: write to currentBuffer + + // TODO: threshold check + // TODO: push current buffer data to prometheus endpoint + // TODO: implement retry mechanism + // TODO: push failed Data to DLQ + }); + }finally { + reentrantLock.unlock(); + } + } +} \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index 67c85c3a12..02125825fd 100644 --- a/settings.gradle +++ b/settings.gradle @@ -128,3 +128,4 @@ include 'data-prepper-plugins:buffer-common' include 'data-prepper-plugins:sqs-source' include 'data-prepper-plugins:cloudwatch-logs' include 'data-prepper-plugins:http-sink' +include 'data-prepper-plugins:prometheus-sink'