From 8152c5792fe404ce8c90351af8936c0f935443ae Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Tue, 28 Jan 2025 18:05:57 -0600 Subject: [PATCH 01/18] Add debug logs for Kafka consumer (#5369) Signed-off-by: Taylor Gray --- .../dataprepper/buffer/common/BufferAccumulator.java | 3 +++ .../plugins/kafka/consumer/KafkaCustomConsumer.java | 11 +++++++++++ 2 files changed, 14 insertions(+) diff --git a/data-prepper-plugins/buffer-common/src/main/java/org/opensearch/dataprepper/buffer/common/BufferAccumulator.java b/data-prepper-plugins/buffer-common/src/main/java/org/opensearch/dataprepper/buffer/common/BufferAccumulator.java index eeaedf4ec1..69960f7dd7 100644 --- a/data-prepper-plugins/buffer-common/src/main/java/org/opensearch/dataprepper/buffer/common/BufferAccumulator.java +++ b/data-prepper-plugins/buffer-common/src/main/java/org/opensearch/dataprepper/buffer/common/BufferAccumulator.java @@ -68,6 +68,7 @@ public void add(final T record) throws Exception { public void flush() throws Exception { try { + LOG.debug("Flushing buffer accumulator"); flushAccumulatedToBuffer(); } catch (final TimeoutException timeoutException) { flushWithBackoff(); @@ -80,11 +81,13 @@ private boolean flushWithBackoff() throws Exception{ boolean flushedSuccessfully; for (int retryCount = 0; retryCount < MAX_FLUSH_RETRIES_ON_IO_EXCEPTION; retryCount++) { + LOG.debug("Retrying buffer flush on retry count {}", retryCount); final ScheduledFuture flushBufferFuture = scheduledExecutorService.schedule(() -> { try { flushAccumulatedToBuffer(); return true; } catch (final TimeoutException e) { + LOG.debug("Timed out retrying buffer accumulator"); return false; } }, nextDelay, TimeUnit.MILLISECONDS); diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java index a84f800d8d..a07d2f5130 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumer.java @@ -204,6 +204,7 @@ ConsumerRecords doPoll() throws Exception { void consumeRecords() throws Exception { try { ConsumerRecords records = doPoll(); + LOG.debug("Consumed records with count {}", records.count()); if (Objects.nonNull(records) && !records.isEmpty() && records.count() > 0) { Map offsets = new HashMap<>(); AcknowledgementSet acknowledgementSet = null; @@ -367,6 +368,7 @@ public void run() { boolean retryingAfterException = false; while (!shutdownInProgress.get()) { + LOG.debug("Still running Kafka consumer in start of loop"); try { if (retryingAfterException) { LOG.debug("Pause consuming from Kafka topic due a previous exception."); @@ -382,12 +384,15 @@ public void run() { paused = false; consumer.resume(consumer.assignment()); } + LOG.debug("Still running Kafka consumer preparing to commit offsets and consume records"); synchronized(this) { commitOffsets(false); resetOffsets(); } consumeRecords(); + LOG.debug("Exited consume records"); topicMetrics.update(consumer); + LOG.debug("Updated consumer metrics"); retryingAfterException = false; } catch (Exception exp) { LOG.error("Error while reading the records from the topic {}. Retry after 10 seconds", topicName, exp); @@ -475,6 +480,7 @@ private void processRecord(final AcknowledgementSet acknowledgementSet, final Re } long numRetries = 0; while (true) { + LOG.debug("In while loop for processing records, paused = {}", paused); try { if (numRetries == 0) { bufferAccumulator.add(record); @@ -485,7 +491,9 @@ private void processRecord(final AcknowledgementSet acknowledgementSet, final Re } catch (Exception e) { if (!paused && numRetries++ > maxRetriesOnException) { paused = true; + LOG.debug("Preparing to call pause"); consumer.pause(consumer.assignment()); + LOG.debug("Pause was called"); } if (e instanceof SizeOverflowException) { topicMetrics.getNumberOfBufferSizeOverflows().increment(); @@ -493,8 +501,10 @@ private void processRecord(final AcknowledgementSet acknowledgementSet, final Re LOG.debug("Error while adding record to buffer, retrying ", e); } try { + LOG.debug("Sleeping due to exception"); Thread.sleep(RETRY_ON_EXCEPTION_SLEEP_MS); if (paused) { + LOG.debug("Calling doPoll()"); ConsumerRecords records = doPoll(); if (records.count() > 0) { LOG.warn("Unexpected records received while the consumer is paused. Resetting the partitions to retry from last read pointer"); @@ -509,6 +519,7 @@ private void processRecord(final AcknowledgementSet acknowledgementSet, final Re } if (paused) { + LOG.debug("Resuming consumption"); consumer.resume(consumer.assignment()); paused = false; } From 70e8c8bb1d4c910d42ad13579482c40b71d56b75 Mon Sep 17 00:00:00 2001 From: Srikanth Govindarajan Date: Tue, 28 Jan 2025 21:16:41 -0800 Subject: [PATCH 02/18] lambda processor should retry for certain class of exceptions (#5320) * lambda processor should retry for certain class of exceptions Signed-off-by: Srikanth Govindarajan * Address Comment on complete codec Signed-off-by: Srikanth Govindarajan * Add retryCondidition to lambda Client Signed-off-by: Srikanth Govindarajan * Address comments Signed-off-by: Srikanth Govindarajan * Address comments and add UT and IT Signed-off-by: Srikanth Govindarajan * Address comment on completeCodec Signed-off-by: Srikanth Govindarajan --------- Signed-off-by: Srikanth Govindarajan --- .../lambda/processor/LambdaProcessorIT.java | 152 +++++++++++++++ .../lambda/common/LambdaCommonHandler.java | 3 + .../common/accumlator/InMemoryBuffer.java | 16 +- .../common/client/LambdaClientFactory.java | 6 +- .../common/util/CountingRetryCondition.java | 23 +++ .../util/CustomLambdaRetryCondition.java | 17 ++ .../common/util/LambdaRetryStrategy.java | 104 +++++++++++ .../lambda/processor/LambdaProcessor.java | 2 +- .../client/LambdaClientFactoryTest.java | 109 +++++++++++ .../lambda/processor/LambdaProcessorTest.java | 174 +++++++++++++++++- .../lambda/utils/CountingHttpClient.java | 32 ++++ .../lambda/utils/LambdaRetryStrategyTest.java | 74 ++++++++ .../lambda-processor-with-retries.yaml | 15 ++ 13 files changed, 718 insertions(+), 9 deletions(-) create mode 100644 data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/CountingRetryCondition.java create mode 100644 data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/CustomLambdaRetryCondition.java create mode 100644 data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/LambdaRetryStrategy.java create mode 100644 data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/utils/CountingHttpClient.java create mode 100644 data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/utils/LambdaRetryStrategyTest.java create mode 100644 data-prepper-plugins/aws-lambda/src/test/resources/lambda-processor-with-retries.yaml diff --git a/data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorIT.java b/data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorIT.java index 8203819fcb..38769c630e 100644 --- a/data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorIT.java +++ b/data-prepper-plugins/aws-lambda/src/integrationTest/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorIT.java @@ -22,8 +22,11 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; + +import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; import org.mockito.junit.jupiter.MockitoSettings; import org.mockito.quality.Strictness; @@ -42,15 +45,23 @@ import org.opensearch.dataprepper.plugins.codec.json.JsonInputCodec; import org.opensearch.dataprepper.plugins.codec.json.JsonInputCodecConfig; import org.opensearch.dataprepper.plugins.lambda.common.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.lambda.common.client.LambdaClientFactory; import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.lambda.common.config.BatchOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.ClientOptions; import org.opensearch.dataprepper.plugins.lambda.common.config.InvocationType; import org.opensearch.dataprepper.plugins.lambda.common.config.ThresholdOptions; +import org.opensearch.dataprepper.plugins.lambda.common.util.CountingRetryCondition; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.http.nio.netty.NettyNioAsyncHttpClient; import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.lambda.LambdaAsyncClient; import software.amazon.awssdk.services.lambda.model.InvokeResponse; +import java.time.Duration; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -58,8 +69,10 @@ import java.util.List; import java.util.Map; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; @ExtendWith(MockitoExtension.class) @@ -373,4 +386,143 @@ private List> createRecords(int numRecords) { } return records; } + + @Test + void testRetryLogicWithThrottlingUsingMultipleThreads() throws Exception { + /* + * This test tries to create multiple parallel Lambda invocations + * while concurrency=1. The first invocation "occupies" the single concurrency slot + * The subsequent invocations should then get a 429 TooManyRequestsException, + * triggering our CountingRetryCondition. + */ + + /* Lambda handler function looks like this: + def lambda_handler(event, context): + # Simulate a slow operation so that + # if concurrency = 1, multiple parallel invocations + # will result in TooManyRequestsException for the second+ invocation. + time.sleep(10) + + # Return a simple success response + return { + "statusCode": 200, + "body": "Hello from concurrency-limited Lambda!" + } + + */ + + functionName = "lambdaExceptionSimulation"; + // Create a CountingRetryCondition + CountingRetryCondition countingRetryCondition = new CountingRetryCondition(); + + // Configure a LambdaProcessorConfig + + // We'll set invocation type to RequestResponse + InvocationType invocationType = mock(InvocationType.class); + when(invocationType.getAwsLambdaValue()).thenReturn(InvocationType.REQUEST_RESPONSE.getAwsLambdaValue()); + when(lambdaProcessorConfig.getInvocationType()).thenReturn(invocationType); + + when(lambdaProcessorConfig.getFunctionName()).thenReturn(functionName); + // If your code uses "responseEventsMatch", you can set it: + when(lambdaProcessorConfig.getResponseEventsMatch()).thenReturn(true); + + // Set up mock ClientOptions for concurrency + small retries + ClientOptions clientOptions = mock(ClientOptions.class); + when(clientOptions.getMaxConnectionRetries()).thenReturn(3); // up to 3 retries + when(clientOptions.getMaxConcurrency()).thenReturn(5); + when(clientOptions.getConnectionTimeout()).thenReturn(Duration.ofSeconds(5)); + when(clientOptions.getApiCallTimeout()).thenReturn(Duration.ofSeconds(30)); + when(lambdaProcessorConfig.getClientOptions()).thenReturn(clientOptions); + + // AWS auth + AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); + when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.of(lambdaRegion)); + when(awsAuthenticationOptions.getAwsStsRoleArn()).thenReturn(role); + when(awsAuthenticationOptions.getAwsStsExternalId()).thenReturn(null); + when(awsAuthenticationOptions.getAwsStsHeaderOverrides()).thenReturn(null); + when(lambdaProcessorConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); + + // Setup the mock for getProvider + when(awsCredentialsSupplier.getProvider(any())).thenReturn(awsCredentialsProvider); + + // Mock the factory to inject our CountingRetryCondition into the LambdaAsyncClient + try (MockedStatic mockedFactory = mockStatic(LambdaClientFactory.class)) { + + LambdaAsyncClient clientWithCountingCondition = LambdaAsyncClient.builder() + .region(Region.of(lambdaRegion)) + .credentialsProvider(awsCredentialsProvider) + .overrideConfiguration(ClientOverrideConfiguration.builder() + .retryPolicy( + RetryPolicy.builder() + .retryCondition(countingRetryCondition) + .numRetries(3) + .build() + ) + .build()) + // netty concurrency = 5 to allow parallel requests + .httpClient(NettyNioAsyncHttpClient.builder() + .maxConcurrency(5) + .build()) + .build(); + + mockedFactory.when(() -> + LambdaClientFactory.createAsyncLambdaClient( + any(AwsAuthenticationOptions.class), + any(AwsCredentialsSupplier.class), + any(ClientOptions.class))) + .thenReturn(clientWithCountingCondition); + + // 7) Instantiate the real LambdaProcessor + when(pluginSetting.getName()).thenReturn("lambda-processor"); + when(pluginSetting.getPipelineName()).thenReturn("test-pipeline"); + lambdaProcessor = new LambdaProcessor( + pluginFactory, + pluginSetting, + lambdaProcessorConfig, + awsCredentialsSupplier, + expressionEvaluator + ); + + // Create multiple parallel tasks to call doExecute(...) + // Each doExecute() invocation sends records to Lambda in an async manner. + int parallelInvocations = 5; + ExecutorService executor = Executors.newFixedThreadPool(parallelInvocations); + + List>>> futures = new ArrayList<>(); + for (int i = 0; i < parallelInvocations; i++) { + // Each subset of records calls the processor + List> records = createRecords(2); + Future>> future = executor.submit(() -> { + return lambdaProcessor.doExecute(records); + }); + futures.add(future); + } + + // Wait for all tasks to complete + executor.shutdown(); + boolean finishedInTime = executor.awaitTermination(5, TimeUnit.MINUTES); + if (!finishedInTime) { + throw new RuntimeException("Test timed out waiting for executor tasks to complete."); + } + + // Check results or handle exceptions + for (Future>> f : futures) { + try { + Collection> out = f.get(); + } catch (ExecutionException ee) { + // A 429 from AWS will be thrown as TooManyRequestsException + // If all retries failed, we might see an exception here. + } + } + + // Finally, check that we had at least one retry + // If concurrency=1 is truly enforced, at least some calls should have gotten a 429 + // -> triggered CountingRetryCondition + int retryCount = countingRetryCondition.getRetryCount(); + assertTrue( + retryCount > 0, + "Should have at least one retry due to concurrency-based throttling (429)." + ); + } + } } diff --git a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/LambdaCommonHandler.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/LambdaCommonHandler.java index 3518508b64..5ef5dac7ac 100644 --- a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/LambdaCommonHandler.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/LambdaCommonHandler.java @@ -36,6 +36,9 @@ private LambdaCommonHandler() { } public static boolean isSuccess(InvokeResponse response) { + if(response == null) { + return false; + } int statusCode = response.statusCode(); return statusCode >= 200 && statusCode < 300; } diff --git a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java index f3e2ea1f8f..662436a01f 100644 --- a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/accumlator/InMemoryBuffer.java @@ -72,6 +72,16 @@ public void addRecord(Record record) { eventCount++; } + void completeCodec() { + if (eventCount > 0) { + try { + requestCodec.complete(this.byteArrayOutputStream); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + public List> getRecords() { return records; } @@ -98,11 +108,7 @@ public InvokeRequest getRequestPayload(String functionName, String invocationTyp return null; } - try { - requestCodec.complete(this.byteArrayOutputStream); - } catch (IOException e) { - throw new RuntimeException(e); - } + completeCodec(); SdkBytes payload = getPayload(); payloadRequestSize = payload.asByteArray().length; diff --git a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/client/LambdaClientFactory.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/client/LambdaClientFactory.java index 87b7a4271b..f019111b4f 100644 --- a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/client/LambdaClientFactory.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/client/LambdaClientFactory.java @@ -5,6 +5,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.lambda.common.config.ClientOptions; +import org.opensearch.dataprepper.plugins.lambda.common.util.CustomLambdaRetryCondition; import org.opensearch.dataprepper.plugins.metricpublisher.MicrometerMetricPublisher; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; @@ -48,13 +49,14 @@ private static ClientOverrideConfiguration createOverrideConfiguration( .maxBackoffTime(clientOptions.getMaxBackoff()) .build(); - final RetryPolicy retryPolicy = RetryPolicy.builder() + final RetryPolicy customRetryPolicy = RetryPolicy.builder() + .retryCondition(new CustomLambdaRetryCondition()) .numRetries(clientOptions.getMaxConnectionRetries()) .backoffStrategy(backoffStrategy) .build(); return ClientOverrideConfiguration.builder() - .retryPolicy(retryPolicy) + .retryPolicy(customRetryPolicy) .addMetricPublisher(new MicrometerMetricPublisher(awsSdkMetrics)) .apiCallTimeout(clientOptions.getApiCallTimeout()) .build(); diff --git a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/CountingRetryCondition.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/CountingRetryCondition.java new file mode 100644 index 0000000000..afc7c756dd --- /dev/null +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/CountingRetryCondition.java @@ -0,0 +1,23 @@ +package org.opensearch.dataprepper.plugins.lambda.common.util; + +import software.amazon.awssdk.core.retry.RetryPolicyContext; + +import java.util.concurrent.atomic.AtomicInteger; + +//Used ONLY for tests +public class CountingRetryCondition extends CustomLambdaRetryCondition { + private final AtomicInteger retryCount = new AtomicInteger(0); + + @Override + public boolean shouldRetry(RetryPolicyContext context) { + boolean shouldRetry = super.shouldRetry(context); + if (shouldRetry) { + retryCount.incrementAndGet(); + } + return shouldRetry; + } + + public int getRetryCount() { + return retryCount.get(); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/CustomLambdaRetryCondition.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/CustomLambdaRetryCondition.java new file mode 100644 index 0000000000..08e3743c13 --- /dev/null +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/CustomLambdaRetryCondition.java @@ -0,0 +1,17 @@ +package org.opensearch.dataprepper.plugins.lambda.common.util; + +import software.amazon.awssdk.core.retry.conditions.RetryCondition; +import software.amazon.awssdk.core.retry.RetryPolicyContext; + +public class CustomLambdaRetryCondition implements RetryCondition { + + @Override + public boolean shouldRetry(RetryPolicyContext context) { + Throwable exception = context.exception(); + if (exception != null) { + return LambdaRetryStrategy.isRetryableException(exception); + } + + return false; + } +} diff --git a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/LambdaRetryStrategy.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/LambdaRetryStrategy.java new file mode 100644 index 0000000000..82abf37832 --- /dev/null +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/common/util/LambdaRetryStrategy.java @@ -0,0 +1,104 @@ +package org.opensearch.dataprepper.plugins.lambda.common.util; + +import software.amazon.awssdk.core.exception.SdkClientException; +import software.amazon.awssdk.services.lambda.model.InvokeResponse; +import software.amazon.awssdk.services.lambda.model.TooManyRequestsException; +import software.amazon.awssdk.services.lambda.model.ServiceException; + +import java.util.Set; + + +/** + * Similar to BulkRetryStrategy in the OpenSearch sink. + * Categorizes AWS Lambda exceptions and status codes into + * retryable and non-retryable scenarios. + */ +public final class LambdaRetryStrategy { + + private LambdaRetryStrategy() { + } + + /** + * Possibly a set of “bad request” style errors which might fall + */ + private static final Set BAD_REQUEST_ERRORS = Set.of( + 400, // Bad Request + 422, // Unprocessable Entity + 417, // Expectation Failed + 406 // Not Acceptable + ); + + /** + * Status codes which may indicate a security or policy problem, so we don't retry. + */ + private static final Set NOT_ALLOWED_ERRORS = Set.of( + 401, // Unauthorized + 403, // Forbidden + 405 // Method Not Allowed + ); + + /** + * Examples of input or payload errors that are likely not retryable + * unless the pipeline itself corrects them. + */ + private static final Set INVALID_INPUT_ERRORS = Set.of( + 413, // Payload Too Large + 414, // URI Too Long + 416 // Range Not Satisfiable + ); + + /** + * Example of a “timeout” scenario. Lambda can return 429 for "Too Many Requests" or + * 408 (if applicable) for timeouts in some contexts. + * This can be considered retryable if you want to handle the throttling scenario. + */ + private static final Set TIMEOUT_ERRORS = Set.of( + 408, // Request Timeout + 429 // Too Many Requests (often used as "throttling" for Lambda) + ); + + public static boolean isRetryableStatusCode(final int statusCode) { + return TIMEOUT_ERRORS.contains(statusCode) || (statusCode >= 500 && statusCode < 600); + } + + /* + * Note:isRetryable and isRetryableException should match + */ + public static boolean isRetryableException(final Throwable t) { + if (t instanceof TooManyRequestsException) { + // Throttling => often can retry with backoff + return true; + } + if (t instanceof ServiceException) { + // Usually indicates a 5xx => can retry + return true; + } + if (t instanceof SdkClientException) { + // Possibly network/connection error => can retry + return true; + } + return false; + } + + /** + * Determines if this is definitely NOT retryable (client error or permanent failure). + */ + public static boolean isNonRetryable(final InvokeResponse response) { + if(response == null) return false; + + int statusCode = response.statusCode(); + return BAD_REQUEST_ERRORS.contains(statusCode) + || NOT_ALLOWED_ERRORS.contains(statusCode) + || INVALID_INPUT_ERRORS.contains(statusCode); + } + + /** + * For convenience, you can create more fine-grained checks or + * direct set membership checks (e.g. isBadRequest(...), isTimeout(...)) if you want. + */ + public static boolean isTimeoutError(final InvokeResponse response) { + return TIMEOUT_ERRORS.contains(response.statusCode()); + } + +} + diff --git a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessor.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessor.java index 786939f5a1..0b9f8b6ffe 100644 --- a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessor.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessor.java @@ -267,4 +267,4 @@ public boolean isReadyForShutdown() { public void shutdown() { } -} +} \ No newline at end of file diff --git a/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/client/LambdaClientFactoryTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/client/LambdaClientFactoryTest.java index cd68d73362..9435721384 100644 --- a/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/client/LambdaClientFactoryTest.java +++ b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/common/client/LambdaClientFactoryTest.java @@ -8,21 +8,33 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.plugins.lambda.common.config.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.lambda.common.config.ClientOptions; +import org.opensearch.dataprepper.plugins.lambda.common.util.CountingRetryCondition; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicyContext; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.lambda.LambdaAsyncClient; +import software.amazon.awssdk.services.lambda.model.InvokeRequest; +import software.amazon.awssdk.services.lambda.model.InvokeResponse; +import software.amazon.awssdk.services.lambda.model.TooManyRequestsException; import java.util.HashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) class LambdaClientFactoryTest { @Mock @@ -75,4 +87,101 @@ void testCreateAsyncLambdaClientOverrideConfiguration() { assertNotNull(overrideConfig.metricPublishers()); assertFalse(overrideConfig.metricPublishers().isEmpty()); } + + @Test + void testRetryConditionIsCalledWithTooManyRequestsException() { + // Arrange + CountingRetryCondition countingRetryCondition = new CountingRetryCondition(); + + // Create mock Lambda client + LambdaAsyncClient mockClient = mock(LambdaAsyncClient.class); + + // Setup mock to return TooManyRequestsException for the first 3 calls + when(mockClient.invoke(any(InvokeRequest.class))) + .thenReturn(CompletableFuture.failedFuture(TooManyRequestsException.builder().build())) + .thenReturn(CompletableFuture.failedFuture(TooManyRequestsException.builder().build())) + .thenReturn(CompletableFuture.failedFuture(TooManyRequestsException.builder().build())); + + // Create test request + InvokeRequest request = InvokeRequest.builder() + .functionName("test-function") + .build(); + + // Simulate retries + for (int i = 0; i < 3; i++) { + try { + CompletableFuture future = mockClient.invoke(request); + RetryPolicyContext context = RetryPolicyContext.builder() + .exception(TooManyRequestsException.builder().build()) + .retriesAttempted(i) + .build(); + + // Test the retry condition + countingRetryCondition.shouldRetry(context); + + future.join(); + } catch (CompletionException e) { + assertTrue(e.getCause() instanceof TooManyRequestsException); + } + } + + // Verify retry count + assertEquals(3, countingRetryCondition.getRetryCount(), + "Retry condition should have been called exactly 3 times"); + } + + @Test + void testRetryConditionFirstFailsAndThenSucceeds() { + // Arrange + CountingRetryCondition countingRetryCondition = new CountingRetryCondition(); + + // Create mock Lambda client + LambdaAsyncClient mockClient = mock(LambdaAsyncClient.class); + + // Setup mock to return TooManyRequestsException for first 2 calls, then succeed on 3rd + when(mockClient.invoke(any(InvokeRequest.class))) + .thenReturn(CompletableFuture.failedFuture(TooManyRequestsException.builder().build())) + .thenReturn(CompletableFuture.failedFuture(TooManyRequestsException.builder().build())) + .thenReturn(CompletableFuture.completedFuture(InvokeResponse.builder() + .statusCode(200) + .build())); + + // Create test request + InvokeRequest request = InvokeRequest.builder() + .functionName("test-function") + .build(); + + // Track if we reached success + boolean successReached = false; + + // Simulate retries with eventual success + for (int i = 0; i < 3; i++) { + try { + CompletableFuture future = mockClient.invoke(request); + + if (i < 2) { + // For first two attempts, verify retry condition + RetryPolicyContext context = RetryPolicyContext.builder() + .exception(TooManyRequestsException.builder().build()) + .retriesAttempted(i) + .build(); + countingRetryCondition.shouldRetry(context); + } + + InvokeResponse response = future.join(); + if (response.statusCode() == 200) { + successReached = true; + } + } catch (CompletionException e) { + assertTrue(e.getCause() instanceof TooManyRequestsException, + "Exception should be TooManyRequestsException"); + } + } + + // Verify retry count and success + assertEquals(2, countingRetryCondition.getRetryCount(), + "Retry condition should have been called exactly 2 times"); + assertTrue(successReached, "Should have reached successful completion"); + } + } diff --git a/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorTest.java index 5c2ee0e8e6..72b3a74d32 100644 --- a/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorTest.java +++ b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorTest.java @@ -66,10 +66,11 @@ import static org.mockito.ArgumentMatchers.anyDouble; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; -import static org.mockito.Mockito.reset; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -128,6 +129,9 @@ public class LambdaProcessorTest { @Mock private Timer lambdaLatencyMetric; + @Mock + private ClientOptions mockClientOptions; + @Mock private LambdaAsyncClient lambdaAsyncClient; @@ -611,4 +615,172 @@ public void testDoExecute_for_strict_and_aggregate_mode(String configFile, assertEquals("[lambda_failure]", record.getData().getMetadata().getTags().toString()); } } + + //NOTE: This test will not pass as invoke failure is handled internally through sdk. + // The first attempt will fail and the second attempt will not even be considered for execution. +// @Test +// public void testDoExecute_retryScenario_successOnSecondAttempt() throws Exception { +// // Arrange +// final List> records = getSampleEventRecords(2); +// +// // First attempt throws TooManyRequestsException => no valid payload +// when(lambdaAsyncClient.invoke(any(InvokeRequest.class))) +// .thenReturn(CompletableFuture.failedFuture( +// TooManyRequestsException.builder() +// .message("First attempt throttled") +// .build() +// )) +// // Second attempt => success with 200 +// .thenReturn(CompletableFuture.completedFuture( +// InvokeResponse.builder() +// .statusCode(200) +// .payload(SdkBytes.fromUtf8String( +// "[{\"successKey1\":\"successValue1\"},{\"successKey2\":\"successValue2\"}]")) +// .build() +// )); +// +// // Create a config which has at least 1 maxConnectionRetries so we can retry once. +// final LambdaProcessorConfig config = createLambdaConfigurationFromYaml("lambda-processor-with-retries.yaml"); +// +// // Instantiate the processor +// final LambdaProcessor processor = new LambdaProcessor( +// pluginFactory, +// pluginSetting, +// config, +// awsCredentialsSupplier, +// expressionEvaluator +// ); +// populatePrivateFields(processor); +// +// // Act +// final Collection> resultRecords = processor.doExecute(records); +// +// // Assert +// // Because the second invocation is successful (200), +// // we expect the final records to NOT have the "lambda_failure" tag +// assertEquals(records.size(), resultRecords.size()); +// for (Record record : resultRecords) { +// assertFalse( +// record.getData().getMetadata().getTags().contains("lambda_failure"), +// "Record should NOT have a failure tag after a successful retry" +// ); +// } +// +// // We invoked the lambda client 2 times total: first attempt + one retry +// verify(lambdaAsyncClient, times(2)).invoke(any(InvokeRequest.class)); +// +// // Second attempt is success => increment success counters +// verify(numberOfRequestsSuccessCounter, times(1)).increment(); +// } + + @Test + public void testDoExecute_retryScenario_failsAfterMaxRetries() throws Exception { + // Arrange + final List> records = getSampleEventRecords(3); + + // Simulate a 500 status code (Retryable) + final InvokeResponse failedResponse = InvokeResponse.builder() + .statusCode(500) + .payload(SdkBytes.fromUtf8String("Internal server error")) + .build(); + + // Stub the lambda client to always return failedResponse + when(lambdaAsyncClient.invoke(any(InvokeRequest.class))) + .thenReturn(CompletableFuture.completedFuture(failedResponse)) + .thenReturn(CompletableFuture.completedFuture(failedResponse)) + .thenReturn(CompletableFuture.completedFuture(failedResponse)); + + // Create a config with exactly 1 maxConnectionRetries (allowing 2 total attempts) + final LambdaProcessorConfig config = createLambdaConfigurationFromYaml("lambda-processor-success-config.yaml"); + + // Instantiate the processor + final LambdaProcessor processor = new LambdaProcessor(pluginFactory, pluginSetting, config, + awsCredentialsSupplier, expressionEvaluator); + populatePrivateFields(processor); + + // Act + final Collection> resultRecords = processor.doExecute(records); + + // Assert + // All records should have the "lambda_failure" tag + assertEquals(records.size(), resultRecords.size(), "Result records count should match input records count."); + for (Record record : resultRecords) { + assertTrue(record.getData().getMetadata().getTags().contains("lambda_failure"), + "Record should have 'lambda_failure' tag after all retries fail"); + } + + // Expect 3 invocations: initial attempt + 3 retry + verify(lambdaAsyncClient, atLeastOnce()).invoke(any(InvokeRequest.class)); + // No success counters + verify(numberOfRequestsSuccessCounter, never()).increment(); + // Records failed counter should increment once with the total number of records + verify(numberOfRecordsFailedCounter, times(1)).increment(records.size()); + } + + + @Test + public void testDoExecute_nonRetryableStatusCode_noRetryAttempted() throws Exception { + // Arrange + final List> records = getSampleEventRecords(2); + + // 400 is a client error => non-retryable + final InvokeResponse badRequestResponse = InvokeResponse.builder() + .statusCode(400) + .payload(SdkBytes.fromUtf8String("Bad request")) + .build(); + + when(lambdaAsyncClient.invoke(any(InvokeRequest.class))) + .thenReturn(CompletableFuture.completedFuture(badRequestResponse)); + + final LambdaProcessorConfig config = createLambdaConfigurationFromYaml("lambda-processor-with-retries.yaml"); + + final LambdaProcessor processor = new LambdaProcessor(pluginFactory, pluginSetting, config, + awsCredentialsSupplier, expressionEvaluator); + populatePrivateFields(processor); + + // Act + final Collection> resultRecords = processor.doExecute(records); + + // Assert + assertEquals(records.size(), resultRecords.size()); + for (Record record : resultRecords) { + assertTrue(record.getData().getMetadata().getTags().contains("lambda_failure"), + "Non-retryable failure should cause 'lambda_failure' tag"); + } + // Only 1 attempt => no second invoke + verify(lambdaAsyncClient, times(1)).invoke(any(InvokeRequest.class)); + // Fail counters + verify(numberOfRecordsFailedCounter).increment(2); + } + + @Test + public void testDoExecute_nonRetryableException_thrownImmediatelyFail() throws Exception { + // Arrange + final List> records = getSampleEventRecords(2); + + // Some random exception that is not in the list of retryable exceptions + when(lambdaAsyncClient.invoke(any(InvokeRequest.class))) + .thenThrow(new IllegalArgumentException("Non-retryable exception")); + + final LambdaProcessorConfig config = createLambdaConfigurationFromYaml("lambda-processor-with-retries.yaml"); + + final LambdaProcessor processor = new LambdaProcessor(pluginFactory, pluginSetting, config, + awsCredentialsSupplier, expressionEvaluator); + populatePrivateFields(processor); + + // Act + final Collection> resultRecords = processor.doExecute(records); + + // Assert + // We expect no success => all records come back tagged + assertEquals(records.size(), resultRecords.size()); + for (Record record : resultRecords) { + assertTrue(record.getData().getMetadata().getTags().contains("lambda_failure"), + "Record should have 'lambda_failure' after a non-retryable exception"); + } + + // Attempted only once + verify(lambdaAsyncClient, times(1)).invoke(any(InvokeRequest.class)); + verify(numberOfRequestsFailedCounter, times(1)).increment(); + } } diff --git a/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/utils/CountingHttpClient.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/utils/CountingHttpClient.java new file mode 100644 index 0000000000..feddd99538 --- /dev/null +++ b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/utils/CountingHttpClient.java @@ -0,0 +1,32 @@ +package org.opensearch.dataprepper.plugins.lambda.utils; + +import software.amazon.awssdk.http.async.SdkAsyncHttpClient; +import software.amazon.awssdk.http.async.AsyncExecuteRequest; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicInteger; + +public class CountingHttpClient implements SdkAsyncHttpClient { + private final SdkAsyncHttpClient delegate; + private final AtomicInteger requestCount = new AtomicInteger(0); + + public CountingHttpClient(SdkAsyncHttpClient delegate) { + this.delegate = delegate; + } + + @Override + public CompletableFuture execute(AsyncExecuteRequest request) { + requestCount.incrementAndGet(); + return delegate.execute(request); + } + + @Override + public void close() { + delegate.close(); + } + + public int getRequestCount() { + return requestCount.get(); + } +} + diff --git a/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/utils/LambdaRetryStrategyTest.java b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/utils/LambdaRetryStrategyTest.java new file mode 100644 index 0000000000..064b24d8fc --- /dev/null +++ b/data-prepper-plugins/aws-lambda/src/test/java/org/opensearch/dataprepper/plugins/lambda/utils/LambdaRetryStrategyTest.java @@ -0,0 +1,74 @@ +package org.opensearch.dataprepper.plugins.lambda.utils; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.mockito.junit.jupiter.MockitoSettings; +import org.mockito.quality.Strictness; +import org.opensearch.dataprepper.plugins.lambda.common.accumlator.Buffer; +import org.opensearch.dataprepper.plugins.lambda.common.config.ClientOptions; +import org.opensearch.dataprepper.plugins.lambda.common.config.InvocationType; +import org.opensearch.dataprepper.plugins.lambda.common.config.LambdaCommonConfig; +import org.opensearch.dataprepper.plugins.lambda.common.util.LambdaRetryStrategy; +import org.slf4j.Logger; +import software.amazon.awssdk.services.lambda.LambdaAsyncClient; +import software.amazon.awssdk.services.lambda.model.InvokeResponse; + +import java.time.Duration; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +@MockitoSettings(strictness = Strictness.LENIENT) +public class LambdaRetryStrategyTest { + + @Mock + private LambdaAsyncClient lambdaAsyncClient; + + @Mock + private Buffer buffer; + + @Mock + private LambdaCommonConfig config; + + @Mock + private Logger logger; + + @BeforeEach + void setUp() { +// when(lambdaAsyncClient.invoke(any(InvokeRequest.class))).thenReturn(CompletableFuture.completedFuture(InvokeResponse.builder().statusCode(200).build())); + when(config.getClientOptions()).thenReturn(mock(ClientOptions.class)); + when(config.getClientOptions().getMaxConnectionRetries()).thenReturn(3); + when(config.getClientOptions().getBaseDelay()).thenReturn(Duration.ofMillis(100)); + when(config.getFunctionName()).thenReturn("testFunction"); + when(config.getInvocationType()).thenReturn(InvocationType.REQUEST_RESPONSE); + } + + @Test + void testIsRetryable() { + assertTrue(LambdaRetryStrategy.isRetryableStatusCode(429)); + assertTrue(LambdaRetryStrategy.isRetryableStatusCode(500)); + assertFalse(LambdaRetryStrategy.isRetryableStatusCode(200)); + } + + @Test + void testIsNonRetryable() { + assertTrue(LambdaRetryStrategy.isNonRetryable(InvokeResponse.builder().statusCode(400).build())); + assertTrue(LambdaRetryStrategy.isNonRetryable(InvokeResponse.builder().statusCode(403).build())); + assertFalse(LambdaRetryStrategy.isNonRetryable(InvokeResponse.builder().statusCode(500).build())); + assertFalse(LambdaRetryStrategy.isNonRetryable(null)); + } + + @Test + void testIsTimeoutError() { + assertTrue(LambdaRetryStrategy.isTimeoutError(InvokeResponse.builder().statusCode(408).build())); + assertTrue(LambdaRetryStrategy.isTimeoutError(InvokeResponse.builder().statusCode(429).build())); + assertFalse(LambdaRetryStrategy.isTimeoutError(InvokeResponse.builder().statusCode(200).build())); + } + +} diff --git a/data-prepper-plugins/aws-lambda/src/test/resources/lambda-processor-with-retries.yaml b/data-prepper-plugins/aws-lambda/src/test/resources/lambda-processor-with-retries.yaml new file mode 100644 index 0000000000..c518d0d335 --- /dev/null +++ b/data-prepper-plugins/aws-lambda/src/test/resources/lambda-processor-with-retries.yaml @@ -0,0 +1,15 @@ +function_name: "lambdaProcessorTest" +response_events_match: true +tags_on_failure: [ "lambda_failure" ] +batch: + key_name: "osi_key" + threshold: + event_count: 100 + maximum_size: 1mb + event_collect_timeout: 335 +client: + max_retries: 50 + max_concurrency: 5 +aws: + region: "us-east-1" + sts_role_arn: "arn:aws:iam::1234567890:role/sample-pipeine-role" \ No newline at end of file From 8f384dc1b573bce9b23c53338e4c04acaf92e870 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Wed, 29 Jan 2025 11:12:05 -0600 Subject: [PATCH 03/18] Fix postgres stream (#5367) Signed-off-by: Hai Yan --- .../plugins/source/rds/leader/LeaderScheduler.java | 10 +++++++--- .../source/rds/stream/LogicalReplicationClient.java | 9 ++++++--- .../rds/stream/LogicalReplicationEventProcessor.java | 2 +- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index d7ab374684..f2587a079d 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -16,6 +16,7 @@ import org.opensearch.dataprepper.plugins.source.rds.coordination.state.ExportProgressState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.LeaderProgressState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.MySqlStreamState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.PostgresStreamState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; import org.opensearch.dataprepper.plugins.source.rds.model.DbTableMetadata; @@ -167,17 +168,20 @@ private void createStreamPartition(RdsSourceConfig sourceConfig) { progressState.setWaitForExport(sourceConfig.isExportEnabled()); progressState.setPrimaryKeyMap(getPrimaryKeyMap()); if (sourceConfig.getEngine() == EngineType.MYSQL) { - final MySqlStreamState mySqlStreamState = progressState.getMySqlStreamState(); + final MySqlStreamState mySqlStreamState = new MySqlStreamState(); getCurrentBinlogPosition().ifPresent(mySqlStreamState::setCurrentPosition); mySqlStreamState.setForeignKeyRelations(((MySqlSchemaManager)schemaManager).getForeignKeyRelations(sourceConfig.getTableNames())); + progressState.setMySqlStreamState(mySqlStreamState); } else { // Postgres // Create replication slot, which will mark the starting point for stream final String publicationName = generatePublicationName(); final String slotName = generateReplicationSlotName(); ((PostgresSchemaManager)schemaManager).createLogicalReplicationSlot(sourceConfig.getTableNames(), publicationName, slotName); - progressState.getPostgresStreamState().setPublicationName(publicationName); - progressState.getPostgresStreamState().setReplicationSlotName(slotName); + final PostgresStreamState postgresStreamState = new PostgresStreamState(); + postgresStreamState.setPublicationName(publicationName); + postgresStreamState.setReplicationSlotName(slotName); + progressState.setPostgresStreamState(postgresStreamState); } StreamPartition streamPartition = new StreamPartition(sourceConfig.getDbIdentifier(), progressState); sourceCoordinator.createPartition(streamPartition); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java index 130f004960..22935fc6e3 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java @@ -25,6 +25,8 @@ public class LogicalReplicationClient implements ReplicationLogClient { private static final Logger LOG = LoggerFactory.getLogger(LogicalReplicationClient.class); + static final String PROTO_VERSION_KEY = "proto_version"; + static final String VERSION_ONE = "1"; static final String PUBLICATION_NAMES_KEY = "publication_names"; private final ConnectionManager connectionManager; @@ -36,10 +38,10 @@ public class LogicalReplicationClient implements ReplicationLogClient { private volatile boolean disconnectRequested = false; public LogicalReplicationClient(final ConnectionManager connectionManager, - final String replicationSlotName, - final String publicationName) { - this.publicationName = publicationName; + final String publicationName, + final String replicationSlotName) { this.connectionManager = connectionManager; + this.publicationName = publicationName; this.replicationSlotName = replicationSlotName; } @@ -54,6 +56,7 @@ public void connect() { .replicationStream() .logical() .withSlotName(replicationSlotName) + .withSlotOption(PROTO_VERSION_KEY, VERSION_ONE) .withSlotOption(PUBLICATION_NAMES_KEY, publicationName); if (startLsn != null) { logicalStreamBuilder.withStartPosition(startLsn); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java index f9881d0063..3d5c1a04b1 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java @@ -158,7 +158,7 @@ void processRelationMessage(ByteBuffer msg) { } void processCommitMessage(ByteBuffer msg) { - int flag = msg.getInt(); + int flag = msg.get(); long commitLsn = msg.getLong(); long endLsn = msg.getLong(); long epochMicro = msg.getLong(); From 49c6a2bcd94eb924cb6f3a782f09f5a07d62ea18 Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Thu, 30 Jan 2025 12:33:22 -0600 Subject: [PATCH 04/18] Change Kafka Buffer defaults for fetch.max.wait.ms, fetch.min.bytes, partition.assignment.strategy, close consumer on shutdown (#5373) Signed-off-by: Taylor Gray --- .../plugins/kafka/buffer/BufferTopicConfig.java | 4 ++-- .../dataprepper/plugins/kafka/buffer/KafkaBuffer.java | 7 ++++++- .../kafka/consumer/KafkaCustomConsumerFactory.java | 2 ++ .../plugins/kafka/util/KafkaTopicConsumerMetrics.java | 8 +++++++- .../dataprepper/plugins/kafka/buffer/KafkaBufferTest.java | 1 + 5 files changed, 18 insertions(+), 4 deletions(-) diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/BufferTopicConfig.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/BufferTopicConfig.java index 56377c1f22..a97b68d0f3 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/BufferTopicConfig.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/BufferTopicConfig.java @@ -26,8 +26,8 @@ class BufferTopicConfig extends CommonTopicConfig implements TopicProducerConfig private static final Long DEFAULT_RETENTION_PERIOD = 604800000L; static final boolean DEFAULT_AUTO_COMMIT = false; static final ByteCount DEFAULT_FETCH_MAX_BYTES = ByteCount.parse("50mb"); - static final Duration DEFAULT_FETCH_MAX_WAIT = Duration.ofMillis(500); - static final ByteCount DEFAULT_FETCH_MIN_BYTES = ByteCount.parse("1b"); + static final Duration DEFAULT_FETCH_MAX_WAIT = Duration.ofMillis(1000); + static final ByteCount DEFAULT_FETCH_MIN_BYTES = ByteCount.parse("2kb"); static final ByteCount DEFAULT_MAX_PARTITION_FETCH_BYTES = ByteCount.parse("1mb"); static final Duration DEFAULT_SESSION_TIMEOUT = Duration.ofSeconds(45); static final String DEFAULT_AUTO_OFFSET_RESET = "earliest"; diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java index f8ec9c4d91..336f29fe0f 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBuffer.java @@ -64,6 +64,8 @@ public class KafkaBuffer extends AbstractBuffer> { private final AbstractBuffer> innerBuffer; private final ExecutorService executorService; private final Duration drainTimeout; + + private final List consumers; private AtomicBoolean shutdownInProgress; private ByteDecoder byteDecoder; @@ -83,7 +85,7 @@ public KafkaBuffer(final PluginSetting pluginSetting, final KafkaBufferConfig ka innerBuffer = new BlockingBuffer<>(INNER_BUFFER_CAPACITY, INNER_BUFFER_BATCH_SIZE, pluginSetting.getPipelineName()); this.shutdownInProgress = new AtomicBoolean(false); final PluginMetrics consumerMetrics = PluginMetrics.fromNames(metricPrefixName + READ, pluginSetting.getPipelineName()); - final List consumers = kafkaCustomConsumerFactory.createConsumersForTopic(kafkaBufferConfig, kafkaBufferConfig.getTopic(), + this.consumers = kafkaCustomConsumerFactory.createConsumersForTopic(kafkaBufferConfig, kafkaBufferConfig.getTopic(), innerBuffer, consumerMetrics, acknowledgementSetManager, byteDecoder, shutdownInProgress, false, circuitBreaker); this.kafkaAdminAccessor = new KafkaAdminAccessor(kafkaBufferConfig, List.of(kafkaBufferConfig.getTopic().getGroupId())); this.executorService = Executors.newFixedThreadPool(consumers.size(), KafkaPluginThreadFactory.defaultExecutorThreadFactory(MDC_KAFKA_PLUGIN_VALUE)); @@ -233,6 +235,9 @@ public void shutdown() { executorService.shutdownNow(); } + LOG.info("Closing {} consumers", consumers.size()); + consumers.forEach(KafkaCustomConsumer::closeConsumer); + innerBuffer.shutdown(); } finally { resetMdc(); diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java index 0d091b8af7..1981f6a60a 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/consumer/KafkaCustomConsumerFactory.java @@ -12,6 +12,7 @@ import io.confluent.kafka.serializers.KafkaJsonDeserializer; import org.apache.commons.lang3.StringUtils; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.CooperativeStickyAssignor; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.errors.BrokerNotAvailableException; import org.apache.kafka.common.serialization.Deserializer; @@ -167,6 +168,7 @@ public static void setConsumerTopicProperties(final Properties properties, final properties.put(ConsumerConfig.FETCH_MAX_BYTES_CONFIG, (int)topicConfig.getFetchMaxBytes()); properties.put(ConsumerConfig.FETCH_MAX_WAIT_MS_CONFIG, topicConfig.getFetchMaxWait()); properties.put(ConsumerConfig.FETCH_MIN_BYTES_CONFIG, (int)topicConfig.getFetchMinBytes()); + properties.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, CooperativeStickyAssignor.class.getName()); } private void setSchemaRegistryProperties(final KafkaConsumerConfig kafkaConsumerConfig, final Properties properties, final TopicConfig topicConfig) { diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicConsumerMetrics.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicConsumerMetrics.java index 1fd03f8aff..60f5d282bd 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicConsumerMetrics.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/util/KafkaTopicConsumerMetrics.java @@ -10,6 +10,8 @@ import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.time.Instant; import java.util.Objects; @@ -17,6 +19,7 @@ import java.util.HashMap; public class KafkaTopicConsumerMetrics { + private static final Logger LOG = LoggerFactory.getLogger(KafkaTopicConsumerMetrics.class); static final String NUMBER_OF_POSITIVE_ACKNOWLEDGEMENTS = "numberOfPositiveAcknowledgements"; static final String NUMBER_OF_NEGATIVE_ACKNOWLEDGEMENTS = "numberOfNegativeAcknowledgements"; static final String NUMBER_OF_RECORDS_FAILED_TO_PARSE = "numberOfRecordsFailedToParse"; @@ -82,7 +85,10 @@ private void initializeMetricNamesMap(final boolean topicNameInMetrics) { double max = 0.0; for (Map.Entry> entry : metricValues.entrySet()) { Map consumerMetrics = entry.getValue(); - synchronized(consumerMetrics) { + synchronized (consumerMetrics) { + if (consumerMetrics.get(metricName) == null) { + LOG.debug("No consumer metric for recordsLagMax found"); + } max = Math.max(max, consumerMetrics.get(metricName)); } } diff --git a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBufferTest.java b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBufferTest.java index 99f2afa76b..f7cae5e416 100644 --- a/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBufferTest.java +++ b/data-prepper-plugins/kafka-plugins/src/test/java/org/opensearch/dataprepper/plugins/kafka/buffer/KafkaBufferTest.java @@ -334,6 +334,7 @@ public void testShutdown_Successful() throws InterruptedException { kafkaBuffer.shutdown(); verify(executorService).shutdown(); verify(executorService).awaitTermination(eq(EXECUTOR_SERVICE_SHUTDOWN_TIMEOUT), eq(TimeUnit.SECONDS)); + verify(consumer).closeConsumer(); } @Test From 414a7c939b2988eaa0ac59b42d61cd893f75ce86 Mon Sep 17 00:00:00 2001 From: Maxwell Brown <55033421+Galactus22625@users.noreply.github.com> Date: Thu, 30 Jan 2025 11:44:34 -0800 Subject: [PATCH 05/18] Improve Jira logging (#5351) Improve Jira logging Signed-off-by: Maxwell Brown --- .../plugins/source/jira/JiraService.java | 2 +- .../plugins/source/jira/JiraSourceConfig.java | 6 ++++++ .../source/jira/rest/JiraRestClient.java | 18 +++++++++--------- .../source/jira/rest/auth/JiraOauthConfig.java | 3 ++- .../source/jira/utils/AddressValidation.java | 2 +- .../partition/LeaderPartition.java | 2 +- .../scheduler/WorkerScheduler.java | 2 +- 7 files changed, 21 insertions(+), 14 deletions(-) diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java index 4600d1bdeb..415c4d54c8 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java @@ -166,7 +166,7 @@ private StringBuilder createIssueFilterCriteria(JiraSourceConfig configuration, .collect(Collectors.joining(DELIMITER, PREFIX, SUFFIX))) .append(CLOSING_ROUND_BRACKET); } - log.error("Created issue filter criteria JiraQl query: {}", jiraQl); + log.info("Created issue filter criteria JiraQl query: {}", jiraQl); return jiraQl; } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfig.java index 3cb7b9501c..4328181fea 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfig.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraSourceConfig.java @@ -12,6 +12,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.Valid; +import jakarta.validation.constraints.AssertTrue; import lombok.Getter; import org.opensearch.dataprepper.plugins.source.jira.configuration.AuthenticationConfig; import org.opensearch.dataprepper.plugins.source.jira.configuration.FilterConfig; @@ -30,6 +31,11 @@ public class JiraSourceConfig implements CrawlerSourceConfig { @JsonProperty("hosts") private List hosts; + @AssertTrue(message = "Jira hosts must be a list of length 1") + boolean isValidHosts() { + return hosts != null && hosts.size() == 1; + } + /** * Authentication Config to Access Jira */ diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java index 92420ac319..28a71f55ee 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java @@ -51,13 +51,11 @@ public class JiraRestClient { public static final List RETRY_ATTEMPT_SLEEP_TIME = List.of(1, 2, 5, 10, 20, 40); private static final String TICKET_FETCH_LATENCY_TIMER = "ticketFetchLatency"; private static final String SEARCH_CALL_LATENCY_TIMER = "searchCallLatency"; - private static final String PROJECTS_FETCH_LATENCY_TIMER = "projectFetchLatency"; private static final String ISSUES_REQUESTED = "issuesRequested"; private final RestTemplate restTemplate; private final JiraAuthConfig authConfig; private final Timer ticketFetchLatencyTimer; private final Timer searchCallLatencyTimer; - private final Timer projectFetchLatencyTimer; private final Counter issuesRequestedCounter; private final PluginMetrics jiraPluginMetrics = PluginMetrics.fromNames("jiraRestClient", "aws"); private int sleepTimeMultiplier = 1000; @@ -68,8 +66,6 @@ public JiraRestClient(RestTemplate restTemplate, JiraAuthConfig authConfig) { ticketFetchLatencyTimer = jiraPluginMetrics.timer(TICKET_FETCH_LATENCY_TIMER); searchCallLatencyTimer = jiraPluginMetrics.timer(SEARCH_CALL_LATENCY_TIMER); - projectFetchLatencyTimer = jiraPluginMetrics.timer(PROJECTS_FETCH_LATENCY_TIMER); - issuesRequestedCounter = jiraPluginMetrics.counter(ISSUES_REQUESTED); } @@ -119,20 +115,24 @@ private ResponseEntity invokeRestApi(URI uri, Class responseType) thro } catch (HttpClientErrorException ex) { HttpStatus statusCode = ex.getStatusCode(); String statusMessage = ex.getMessage(); - log.error("An exception has occurred while getting response from Jira search API {}", ex.getMessage()); + log.error(NOISY, "An exception has occurred while getting response from Jira search API with statusCode {} and error message: {}", statusCode, statusMessage); if (statusCode == HttpStatus.FORBIDDEN) { throw new UnAuthorizedException(statusMessage); } else if (statusCode == HttpStatus.UNAUTHORIZED) { - log.error(NOISY, "Token expired. We will try to renew the tokens now", ex); + log.warn(NOISY, "Token expired. We will try to renew the tokens now."); authConfig.renewCredentials(); - } else if (statusCode == HttpStatus.TOO_MANY_REQUESTS) { - log.error(NOISY, "Hitting API rate limit. Backing off with sleep timer.", ex); + } else if (statusCode == HttpStatus.TOO_MANY_REQUESTS || statusCode == HttpStatus.SERVICE_UNAVAILABLE || statusCode == HttpStatus.GATEWAY_TIMEOUT) { + log.error(NOISY, "Received {}. Backing off with sleep timer for {} seconds.", statusCode, RETRY_ATTEMPT_SLEEP_TIME.get(retryCount)); + } else { + log.error(NOISY, "Received an unexpected status code {} response from Jira.", statusCode, ex); } try { Thread.sleep((long) RETRY_ATTEMPT_SLEEP_TIME.get(retryCount) * sleepTimeMultiplier); } catch (InterruptedException e) { - throw new RuntimeException("Sleep in the retry attempt got interrupted", e); + throw new RuntimeException("Sleep in the retry attempt got interrupted."); } + } catch (Exception ex) { + log.error(NOISY, "An exception has occurred while getting a response from the Jira search API", ex); } retryCount++; } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfig.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfig.java index ddcf1c8468..9640ba3818 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfig.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/auth/JiraOauthConfig.java @@ -100,7 +100,7 @@ public String getJiraAccountCloudId() { if (e.getRawStatusCode() == HttpStatus.UNAUTHORIZED.value()) { renewCredentials(); } - log.error("Error occurred while accessing resources: ", e); + log.error("Error occurred while accessing resources. Status code: {}. Error message: {}", e.getStatusCode(), e.getMessage()); } } throw new UnAuthorizedException(String.format("Access token expired. Unable to renew even after %s attempts", RETRY_ATTEMPT)); @@ -153,6 +153,7 @@ public void renewCredentials() { this.accessToken = (String) oauth2Config.getAccessToken().getValue(); this.refreshToken = (String) oauth2Config.getRefreshToken().getValue(); this.expireTime = Instant.now().plusSeconds(10); + log.info("Access Token and Refresh Token pair is now refreshed."); } throw new RuntimeException("Failed to renew access token message:" + ex.getMessage(), ex); } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java index d6cc166226..e82acb2a07 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java @@ -38,7 +38,7 @@ public static InetAddress getInetAddress(String url) { try { return InetAddress.getByName(new URL(url).getHost()); } catch (UnknownHostException | MalformedURLException e) { - log.error(INVALID_URL, e); + log.error("{}: {}", INVALID_URL, url); throw new BadRequestException(e.getMessage(), e); } } diff --git a/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/partition/LeaderPartition.java b/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/partition/LeaderPartition.java index a54e50d36f..4a013e9373 100644 --- a/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/partition/LeaderPartition.java +++ b/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/partition/LeaderPartition.java @@ -75,7 +75,7 @@ public LeaderProgressState convertToPartitionState(final String serializedPartit try { return objectMapper.readValue(serializedPartitionProgressState, LeaderProgressState.class); } catch (final JsonProcessingException e) { - LOG.error("Unable to convert string to partition progress state class ", e); + LOG.error("Unable to convert string to partition progress state class due to {}. Partition progress state string: {}.", e.getMessage(), serializedPartitionProgressState); return null; } } diff --git a/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/scheduler/WorkerScheduler.java b/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/scheduler/WorkerScheduler.java index f2fc7e4b40..e738c0e19c 100644 --- a/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/scheduler/WorkerScheduler.java +++ b/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/scheduler/WorkerScheduler.java @@ -89,7 +89,7 @@ public void run() { try { Thread.sleep(RETRY_BACKOFF_ON_EXCEPTION_MILLIS); } catch (InterruptedException ex) { - log.warn("Thread interrupted while waiting to retry", ex); + log.warn("Thread interrupted while waiting to retry due to {}", ex.getMessage()); } } } From 5b16480657bb32e3a5560d63be61bf3c81c8f50a Mon Sep 17 00:00:00 2001 From: Santhosh Gandhe <1909520+san81@users.noreply.github.com> Date: Fri, 31 Jan 2025 12:15:40 -0800 Subject: [PATCH 06/18] Plugin metrics injection (#5372) * injectable plugin metrics Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * removed an unused parameter Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> * fixing a flaky test Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> --------- Signed-off-by: Santhosh Gandhe <1909520+san81@users.noreply.github.com> --- .../plugin/DefaultPluginFactoryIT.java | 15 ++++++---- .../plugin/DefaultPluginFactory.java | 8 ++--- .../plugin/PluginBeanFactoryProvider.java | 16 ++++++---- .../plugin/DefaultPluginFactoryTest.java | 26 ++++++++-------- .../plugin/PluginBeanFactoryProviderTest.java | 22 +++++++------- .../TestComponentWithConfigInject.java | 9 +++++- .../oteltrace/OTelTraceRawProcessorTest.java | 9 ++++-- .../plugins/source/jira/JiraService.java | 9 +++--- .../source/jira/rest/JiraRestClient.java | 25 ++++++++-------- .../plugins/source/jira/JiraIteratorTest.java | 20 ++++++------- .../plugins/source/jira/JiraServiceTest.java | 22 +++++++------- .../source/jira/rest/JiraRestClientTest.java | 30 ++++++++----------- .../source/source_crawler/base/Crawler.java | 4 +-- .../scheduler/WorkerScheduler.java | 5 ++-- .../source_crawler/base/CrawlerTest.java | 4 ++- 15 files changed, 118 insertions(+), 106 deletions(-) diff --git a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java index 9230aa7ff8..de81283f0b 100644 --- a/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java +++ b/data-prepper-core/src/integrationTest/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryIT.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugin; +import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -14,18 +15,19 @@ import org.opensearch.dataprepper.core.event.EventFactoryApplicationContextMarker; import org.opensearch.dataprepper.core.validation.LoggingPluginErrorsHandler; import org.opensearch.dataprepper.core.validation.PluginErrorCollector; -import org.opensearch.dataprepper.model.plugin.NoPluginFoundException; -import org.opensearch.dataprepper.plugins.configtest.TestComponentWithConfigInject; -import org.opensearch.dataprepper.plugins.configtest.TestDISourceWithConfig; -import org.opensearch.dataprepper.validation.PluginErrorsHandler; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PipelinesDataFlowModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.plugin.NoPluginFoundException; import org.opensearch.dataprepper.model.source.Source; import org.opensearch.dataprepper.plugins.TestObjectPlugin; +import org.opensearch.dataprepper.plugins.configtest.TestComponentWithConfigInject; +import org.opensearch.dataprepper.plugins.configtest.TestDISourceWithConfig; import org.opensearch.dataprepper.plugins.test.TestComponent; import org.opensearch.dataprepper.plugins.test.TestDISource; import org.opensearch.dataprepper.plugins.test.TestPlugin; +import org.opensearch.dataprepper.validation.PluginErrorsHandler; import org.springframework.context.annotation.AnnotationConfigApplicationContext; import java.util.Collections; @@ -129,7 +131,7 @@ void loadPlugin_should_return_a_new_plugin_instance_with_DI_context_initialized( } @Test - void loadPlugin_should_return_a_new_plugin_instance_with_DI_context_and_config_injected() { + void loadPlugin_should_return_a_new_plugin_instance_with_DI_context_with_config_and_plugin_metrics_injected() { final String requiredStringValue = UUID.randomUUID().toString(); final String optionalStringValue = UUID.randomUUID().toString(); @@ -152,6 +154,9 @@ void loadPlugin_should_return_a_new_plugin_instance_with_DI_context_and_config_i assertThat(pluginConfig.getRequiredString(), equalTo(requiredStringValue)); assertThat(pluginConfig.getOptionalString(), equalTo(optionalStringValue)); assertThat(plugin.getTestComponent().getIdentifier(), equalTo("test-component-with-plugin-config-injected")); + PluginMetrics pluginMetrics = plugin.getTestComponent().getPluginMetrics(); + assertInstanceOf(PluginMetrics.class, pluginMetrics); + assertInstanceOf(Counter.class, pluginMetrics.counter("testCounter")); } @Test diff --git a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/DefaultPluginFactory.java b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/DefaultPluginFactory.java index 0ec3b5a953..456a0ef7be 100644 --- a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/DefaultPluginFactory.java +++ b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/DefaultPluginFactory.java @@ -64,14 +64,14 @@ public class DefaultPluginFactory implements PluginFactory { this.pluginBeanFactoryProvider = Objects.requireNonNull(pluginBeanFactoryProvider); this.pluginConfigurationObservableFactory = pluginConfigurationObservableFactory; - if(pluginProviders.isEmpty()) { + if (pluginProviders.isEmpty()) { throw new RuntimeException("Data Prepper requires at least one PluginProvider. " + "Your Data Prepper configuration may be missing the org.opensearch.dataprepper.plugin.PluginProvider file."); } } @Override - public T loadPlugin(final Class baseClass, final PluginSetting pluginSetting, final Object ... args) { + public T loadPlugin(final Class baseClass, final PluginSetting pluginSetting, final Object... args) { final String pluginName = pluginSetting.getName(); final Class pluginClass = getPluginClass(baseClass, pluginName); @@ -100,7 +100,7 @@ public List loadPlugins( final Integer numberOfInstances = numberOfInstancesFunction.apply(pluginClass); - if(numberOfInstances == null || numberOfInstances < 0) + if (numberOfInstances == null || numberOfInstances < 0) throw new IllegalArgumentException("The numberOfInstances must be provided as a non-negative integer."); final ComponentPluginArgumentsContext constructionContext = getConstructionContext(pluginSetting, pluginClass, null); @@ -121,7 +121,7 @@ private ComponentPluginArgumentsContext getConstructionContext(final PluginS .createDefaultPluginConfigObservable(pluginConfigurationConverter, pluginConfigurationType, pluginSetting); Class[] markersToScan = pluginAnnotation.packagesToScan(); - BeanFactory beanFactory = pluginBeanFactoryProvider.createPluginSpecificContext(markersToScan, configuration); + BeanFactory beanFactory = pluginBeanFactoryProvider.createPluginSpecificContext(markersToScan, configuration, pluginSetting); return new ComponentPluginArgumentsContext.Builder() .withPluginSetting(pluginSetting) diff --git a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/PluginBeanFactoryProvider.java b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/PluginBeanFactoryProvider.java index 76762f8d8e..c544c99d3f 100644 --- a/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/PluginBeanFactoryProvider.java +++ b/data-prepper-plugin-framework/src/main/java/org/opensearch/dataprepper/plugin/PluginBeanFactoryProvider.java @@ -5,6 +5,7 @@ package org.opensearch.dataprepper.plugin; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.springframework.beans.factory.BeanFactory; import org.springframework.beans.factory.support.DefaultListableBeanFactory; @@ -20,8 +21,8 @@ /** * @since 1.3 *

- * Used to create new instances of ApplicationContext that can be used to provide a per plugin instance isolated ApplicationContext - * scope. CoreApplicationContext is unavailable to sharedPluginApplicationContext and its children. + * Used to create new instances of ApplicationContext that can be used to provide a per plugin instance isolated ApplicationContext + * scope. CoreApplicationContext is unavailable to sharedPluginApplicationContext and its children. *

*

pluginIsolatedApplicationContext inherits from {@link PluginBeanFactoryProvider#sharedPluginApplicationContext}

*

{@link PluginBeanFactoryProvider#sharedPluginApplicationContext} inherits from publicContext

@@ -53,20 +54,23 @@ GenericApplicationContext getCoreApplicationContext() { } /** + * @return BeanFactory A BeanFactory that inherits from {@link PluginBeanFactoryProvider#sharedPluginApplicationContext} * @since 1.3 * Creates a new isolated application context that inherits from * {@link PluginBeanFactoryProvider#sharedPluginApplicationContext} then returns new context's BeanFactory. * {@link PluginBeanFactoryProvider#sharedPluginApplicationContext} should not be directly accessible to plugins. * instead, a new isolated {@link ApplicationContext} should be created. - * @return BeanFactory A BeanFactory that inherits from {@link PluginBeanFactoryProvider#sharedPluginApplicationContext} */ - public BeanFactory createPluginSpecificContext(Class[] markersToScan, Object configuration) { + public BeanFactory createPluginSpecificContext(Class[] markersToScan, Object configuration, final PluginSetting pluginSetting) { AnnotationConfigApplicationContext isolatedPluginApplicationContext = new AnnotationConfigApplicationContext(); DefaultListableBeanFactory beanFactory = (DefaultListableBeanFactory) isolatedPluginApplicationContext.getBeanFactory(); - if(markersToScan !=null && markersToScan.length>0) { - if(configuration !=null && !(configuration instanceof PluginSetting)) { + if (markersToScan != null && markersToScan.length > 0) { + if (configuration != null && !(configuration instanceof PluginSetting)) { beanFactory.registerSingleton(configuration.getClass().getName(), configuration); } + if (pluginSetting != null) { + beanFactory.registerSingleton(PluginMetrics.class.getName(), PluginMetrics.fromPluginSetting(pluginSetting)); + } // If packages to scan is provided in this plugin annotation, which indicates // that this plugin is interested in using Dependency Injection isolated for its module Arrays.stream(markersToScan) diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryTest.java index 2c1bf9e0fa..3ae13b83f4 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryTest.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/DefaultPluginFactoryTest.java @@ -216,7 +216,7 @@ void loadPlugin_should_create_a_new_instance_of_the_plugin_with_di_initialized() equalTo(expectedInstance)); verify(pluginConfigurationObservableFactory).createDefaultPluginConfigObservable(eq(pluginConfigurationConverter), eq(PluginSetting.class), eq(pluginSetting)); - verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{TestDISource.class}, convertedConfiguration); + verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{TestDISource.class}, convertedConfiguration, pluginSetting); } @Test @@ -233,7 +233,7 @@ void loadPlugin_should_create_a_new_instance_of_the_first_plugin_found() { equalTo(expectedInstance)); verify(pluginConfigurationObservableFactory).createDefaultPluginConfigObservable(eq(pluginConfigurationConverter), eq(PluginSetting.class), eq(pluginSetting)); - verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration); + verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration, pluginSetting); } @Test @@ -283,7 +283,7 @@ void loadPlugins_should_return_an_empty_list_when_the_number_of_instances_is_0() assertThat(plugins, notNullValue()); assertThat(plugins.size(), equalTo(0)); - verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, null); + verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, null, pluginSetting); verifyNoInteractions(pluginCreator); } @@ -299,7 +299,7 @@ void loadPlugins_should_return_a_single_instance_when_the_the_numberOfInstances_ final List plugins = createObjectUnderTest().loadPlugins( baseClass, pluginSetting, c -> 1); - verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration); + verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration, pluginSetting); verify(pluginConfigurationObservableFactory).createDefaultPluginConfigObservable(eq(pluginConfigurationConverter), eq(PluginSetting.class), eq(pluginSetting)); final ArgumentCaptor pluginArgumentsContextArgCapture = ArgumentCaptor.forClass(ComponentPluginArgumentsContext.class); @@ -309,7 +309,7 @@ void loadPlugins_should_return_a_single_instance_when_the_the_numberOfInstances_ final Object[] pipelineDescriptionObj = actualPluginArgumentsContext.createArguments(classes.toArray(new Class[1])); assertThat(pipelineDescriptionObj.length, equalTo(1)); assertThat(pipelineDescriptionObj[0], instanceOf(PipelineDescription.class)); - final PipelineDescription actualPipelineDescription = (PipelineDescription)pipelineDescriptionObj[0]; + final PipelineDescription actualPipelineDescription = (PipelineDescription) pipelineDescriptionObj[0]; assertThat(actualPipelineDescription.getPipelineName(), is(pipelineName)); assertThat(plugins, notNullValue()); assertThat(plugins.size(), equalTo(1)); @@ -328,7 +328,7 @@ void loadPlugin_with_varargs_should_return_a_single_instance_when_the_the_number final Object plugin = createObjectUnderTest().loadPlugin(baseClass, pluginSetting, object); - verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration); + verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration, pluginSetting); verify(pluginConfigurationObservableFactory).createDefaultPluginConfigObservable(eq(pluginConfigurationConverter), eq(PluginSetting.class), eq(pluginSetting)); final ArgumentCaptor pluginArgumentsContextArgCapture = ArgumentCaptor.forClass(ComponentPluginArgumentsContext.class); @@ -338,7 +338,7 @@ void loadPlugin_with_varargs_should_return_a_single_instance_when_the_the_number final Object[] pipelineDescriptionObj = actualPluginArgumentsContext.createArguments(classes.toArray(new Class[1])); assertThat(pipelineDescriptionObj.length, equalTo(1)); assertThat(pipelineDescriptionObj[0], instanceOf(PipelineDescription.class)); - final PipelineDescription actualPipelineDescription = (PipelineDescription)pipelineDescriptionObj[0]; + final PipelineDescription actualPipelineDescription = (PipelineDescription) pipelineDescriptionObj[0]; assertThat(actualPipelineDescription.getPipelineName(), is(pipelineName)); assertThat(plugin, notNullValue()); assertThat(plugin, equalTo(expectedInstance)); @@ -380,7 +380,7 @@ void loadPlugins_should_return_an_instance_for_the_total_count() { final List plugins = createObjectUnderTest().loadPlugins( baseClass, pluginSetting, c -> 3); - verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration); + verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration, pluginSetting); final ArgumentCaptor pluginArgumentsContextArgCapture = ArgumentCaptor.forClass(ComponentPluginArgumentsContext.class); verify(pluginCreator, times(3)).newPluginInstance(eq(expectedPluginClass), pluginArgumentsContextArgCapture.capture(), eq(pluginName)); final List actualPluginArgumentsContextList = pluginArgumentsContextArgCapture.getAllValues(); @@ -390,7 +390,7 @@ void loadPlugins_should_return_an_instance_for_the_total_count() { final Object[] pipelineDescriptionObj = pluginArgumentsContext.createArguments(classes.toArray(new Class[1])); assertThat(pipelineDescriptionObj.length, equalTo(1)); assertThat(pipelineDescriptionObj[0], instanceOf(PipelineDescription.class)); - final PipelineDescription actualPipelineDescription = (PipelineDescription)pipelineDescriptionObj[0]; + final PipelineDescription actualPipelineDescription = (PipelineDescription) pipelineDescriptionObj[0]; assertThat(actualPipelineDescription.getPipelineName(), is(pipelineName)); }); assertThat(plugins, notNullValue()); @@ -416,7 +416,7 @@ void loadPlugins_should_return_a_single_instance_with_values_from_ApplicationCon final List plugins = createObjectUnderTest().loadPlugins( baseClass, pluginSetting, c -> 1); - verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration); + verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration, pluginSetting); final ArgumentCaptor pluginArgumentsContextArgCapture = ArgumentCaptor.forClass(ComponentPluginArgumentsContext.class); verify(pluginCreator).newPluginInstance(eq(expectedPluginClass), pluginArgumentsContextArgCapture.capture(), eq(pluginName)); final ComponentPluginArgumentsContext actualPluginArgumentsContext = pluginArgumentsContextArgCapture.getValue(); @@ -425,7 +425,7 @@ void loadPlugins_should_return_a_single_instance_with_values_from_ApplicationCon assertThat(pipelineDescriptionObj.length, equalTo(2)); assertThat(pipelineDescriptionObj[0], instanceOf(PipelineDescription.class)); assertThat(pipelineDescriptionObj[1], sameInstance(suppliedAdditionalArgument)); - final PipelineDescription actualPipelineDescription = (PipelineDescription)pipelineDescriptionObj[0]; + final PipelineDescription actualPipelineDescription = (PipelineDescription) pipelineDescriptionObj[0]; assertThat(actualPipelineDescription.getPipelineName(), is(pipelineName)); assertThat(plugins, notNullValue()); assertThat(plugins.size(), equalTo(1)); @@ -458,7 +458,7 @@ void loadPlugin_should_create_a_new_instance_of_the_first_plugin_found_with_corr assertThat(createObjectUnderTest().loadPlugin(baseClass, pluginSetting), equalTo(expectedInstance)); MatcherAssert.assertThat(expectedInstance.getClass().getAnnotation(DataPrepperPlugin.class).deprecatedName(), equalTo(TEST_SINK_DEPRECATED_NAME)); - verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration); + verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration, pluginSetting); } } @@ -487,7 +487,7 @@ void loadPlugin_should_create_a_new_instance_of_the_first_plugin_found_with_corr assertThat(createObjectUnderTest().loadPlugin(baseClass, pluginSetting), equalTo(expectedInstance)); MatcherAssert.assertThat(expectedInstance.getClass().getAnnotation(DataPrepperPlugin.class).alternateNames(), equalTo(new String[]{TEST_SINK_ALTERNATE_NAME})); - verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration); + verify(beanFactoryProvider).createPluginSpecificContext(new Class[]{}, convertedConfiguration, pluginSetting); } } } diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/PluginBeanFactoryProviderTest.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/PluginBeanFactoryProviderTest.java index a80e42c011..85628db855 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/PluginBeanFactoryProviderTest.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugin/PluginBeanFactoryProviderTest.java @@ -78,7 +78,7 @@ void testPluginBeanFactoryProviderGetReturnsBeanFactory() { final PluginBeanFactoryProvider beanFactoryProvider = createObjectUnderTest(); verify(context).getParent(); - assertThat(beanFactoryProvider.createPluginSpecificContext(new Class[]{}, null), is(instanceOf(BeanFactory.class))); + assertThat(beanFactoryProvider.createPluginSpecificContext(new Class[]{}, null, null), is(instanceOf(BeanFactory.class))); } @Test @@ -86,8 +86,8 @@ void testPluginBeanFactoryProviderGetReturnsUniqueBeanFactory() { doReturn(context).when(context).getParent(); final PluginBeanFactoryProvider beanFactoryProvider = createObjectUnderTest(); - final BeanFactory isolatedBeanFactoryA = beanFactoryProvider.createPluginSpecificContext(new Class[]{}, null); - final BeanFactory isolatedBeanFactoryB = beanFactoryProvider.createPluginSpecificContext(new Class[]{}, null); + final BeanFactory isolatedBeanFactoryA = beanFactoryProvider.createPluginSpecificContext(new Class[]{}, null, null); + final BeanFactory isolatedBeanFactoryB = beanFactoryProvider.createPluginSpecificContext(new Class[]{}, null, null); verify(context).getParent(); assertThat(isolatedBeanFactoryA, not(sameInstance(isolatedBeanFactoryB))); @@ -113,7 +113,7 @@ void getSharedPluginApplicationContext_called_multiple_times_returns_same_instan void testCreatePluginSpecificContext() { when(context.getParent()).thenReturn(context); final PluginBeanFactoryProvider objectUnderTest = createObjectUnderTest(); - BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{TestComponent.class}, null); + BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{TestComponent.class}, null, null); assertThat(beanFactory, notNullValue()); assertThat(beanFactory.getBean(TestComponent.class), notNullValue()); } @@ -122,7 +122,7 @@ void testCreatePluginSpecificContext() { void testCreatePluginSpecificContext_with_empty_array() { when(context.getParent()).thenReturn(context); final PluginBeanFactoryProvider objectUnderTest = createObjectUnderTest(); - BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{}, null); + BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{}, null, null); assertThat(beanFactory, notNullValue()); assertThat(beanFactory, instanceOf(ListableBeanFactory.class)); ListableBeanFactory listableBeanFactory = (ListableBeanFactory) beanFactory; @@ -137,9 +137,9 @@ void testCreatePluginSpecificContext_with_pipeline_settings() { when(context.getParent()).thenReturn(context); final PluginBeanFactoryProvider objectUnderTest = createObjectUnderTest(); PluginSetting pipelineSettings = new PluginSetting(UUID.randomUUID().toString(), Map.of("key", "val")); - BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{}, pipelineSettings); + BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{}, pipelineSettings, pipelineSettings); assertThat(beanFactory, notNullValue()); - assertThrows(NoSuchBeanDefinitionException.class, ()->beanFactory.getBean(PluginSetting.class)); + assertThrows(NoSuchBeanDefinitionException.class, () -> beanFactory.getBean(PluginSetting.class)); } @Test @@ -147,10 +147,10 @@ void testCreatePluginSpecificContext_with_empty_array_with_plugin_config() { when(context.getParent()).thenReturn(context); final PluginBeanFactoryProvider objectUnderTest = createObjectUnderTest(); TestPluginConfiguration config = new TestPluginConfiguration(); - BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{}, config); + BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{}, config, null); assertThat(beanFactory, notNullValue()); - assertThrows(NoSuchBeanDefinitionException.class, ()->beanFactory.getBean(TestComponent.class)); - assertThrows(NoSuchBeanDefinitionException.class, ()->beanFactory.getBean(TestPluginConfiguration.class)); + assertThrows(NoSuchBeanDefinitionException.class, () -> beanFactory.getBean(TestComponent.class)); + assertThrows(NoSuchBeanDefinitionException.class, () -> beanFactory.getBean(TestPluginConfiguration.class)); } @Test @@ -160,7 +160,7 @@ void testCreatePluginSpecificContext_with_plugin_config() { TestPluginConfiguration config = new TestPluginConfiguration(); String requiredStringValue = UUID.randomUUID().toString(); config.setRequiredString(requiredStringValue); - BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{TestComponent.class}, config); + BeanFactory beanFactory = objectUnderTest.createPluginSpecificContext(new Class[]{TestComponent.class}, config, null); assertThat(beanFactory, notNullValue()); assertThat(beanFactory.getBean(TestComponent.class), notNullValue()); assertThat(beanFactory.getBean(TestPluginConfiguration.class), notNullValue()); diff --git a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugins/configtest/TestComponentWithConfigInject.java b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugins/configtest/TestComponentWithConfigInject.java index 141f4f5264..21d95b5808 100644 --- a/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugins/configtest/TestComponentWithConfigInject.java +++ b/data-prepper-plugin-framework/src/test/java/org/opensearch/dataprepper/plugins/configtest/TestComponentWithConfigInject.java @@ -1,5 +1,6 @@ package org.opensearch.dataprepper.plugins.configtest; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.plugin.TestPluginConfiguration; import javax.inject.Named; @@ -7,9 +8,11 @@ @Named public class TestComponentWithConfigInject { private final TestPluginConfiguration configuration; + private final PluginMetrics pluginMetrics; - public TestComponentWithConfigInject(TestPluginConfiguration configuration) { + public TestComponentWithConfigInject(TestPluginConfiguration configuration, PluginMetrics pluginMetrics) { this.configuration = configuration; + this.pluginMetrics = pluginMetrics; } public String getIdentifier() { @@ -19,4 +22,8 @@ public String getIdentifier() { public TestPluginConfiguration getConfiguration() { return configuration; } + + public PluginMetrics getPluginMetrics() { + return pluginMetrics; + } } diff --git a/data-prepper-plugins/otel-trace-raw-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OTelTraceRawProcessorTest.java b/data-prepper-plugins/otel-trace-raw-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OTelTraceRawProcessorTest.java index 9163aad322..f934bc2a4c 100644 --- a/data-prepper-plugins/otel-trace-raw-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OTelTraceRawProcessorTest.java +++ b/data-prepper-plugins/otel-trace-raw-processor/src/test/java/org/opensearch/dataprepper/plugins/processor/oteltrace/OTelTraceRawProcessorTest.java @@ -255,7 +255,7 @@ void testMetricsOnSpanSet() { "0, 4", "2, 6" }) - void traceGroupCacheMaxSize_provides_an_upper_bound(final long cacheMaxSize, final int expectedProcessedRecords) { + void traceGroupCacheMaxSize_provides_an_upper_bound(final long cacheMaxSize, final int leastExpectedProcessedRecordCount) { reset(config); when(config.getTraceFlushIntervalSeconds()).thenReturn(TEST_TRACE_FLUSH_INTERVAL); when(config.getTraceGroupCacheMaxSize()).thenReturn(cacheMaxSize); @@ -267,7 +267,12 @@ void traceGroupCacheMaxSize_provides_an_upper_bound(final long cacheMaxSize, fin processedRecords.addAll(oTelTraceRawProcessor.doExecute(TEST_TWO_TRACE_GROUP_INTERLEAVED_PART_1_RECORDS)); processedRecords.addAll(oTelTraceRawProcessor.doExecute(TEST_TWO_TRACE_GROUP_INTERLEAVED_PART_2_RECORDS)); - MatcherAssert.assertThat(processedRecords.size(), equalTo(expectedProcessedRecords)); + // Caffeine cache eviction is not a synchronized action so trying to make an exact match with the + // expected record count may not always match depending on when the records are getting evicted. + // So changing this assertion from equals to greater than equals to expected record count + assertTrue(processedRecords.size() >= leastExpectedProcessedRecordCount, + String.format("Processed records of %d should be at least the expected record count of %d", + processedRecords.size(), leastExpectedProcessedRecordCount)); MatcherAssert.assertThat(getMissingTraceGroupFieldsSpanCount(processedRecords), equalTo(0)); } diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java index 415c4d54c8..46e51fcbaa 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java @@ -61,13 +61,12 @@ public class JiraService { private final JiraSourceConfig jiraSourceConfig; private final JiraRestClient jiraRestClient; private final Counter searchResultsFoundCounter; - private final PluginMetrics jiraPluginMetrics = PluginMetrics.fromNames("jiraService", "aws"); - public JiraService(JiraSourceConfig jiraSourceConfig, JiraRestClient jiraRestClient) { + public JiraService(JiraSourceConfig jiraSourceConfig, JiraRestClient jiraRestClient, PluginMetrics pluginMetrics) { this.jiraSourceConfig = jiraSourceConfig; this.jiraRestClient = jiraRestClient; - this.searchResultsFoundCounter = jiraPluginMetrics.counter(SEARCH_RESULTS_FOUND); + this.searchResultsFoundCounter = pluginMetrics.counter(SEARCH_RESULTS_FOUND); } /** @@ -99,7 +98,7 @@ private void searchForNewTicketsAndAddToQueue(JiraSourceConfig configuration, In int total; int startAt = 0; do { - SearchResults searchIssues = jiraRestClient.getAllIssues(jql, startAt, configuration); + SearchResults searchIssues = jiraRestClient.getAllIssues(jql, startAt); List issueList = new ArrayList<>(searchIssues.getIssues()); total = searchIssues.getTotal(); startAt += searchIssues.getIssues().size(); @@ -132,7 +131,7 @@ private void addItemsToQueue(List issueList, Queue itemInfo private StringBuilder createIssueFilterCriteria(JiraSourceConfig configuration, Instant ts) { log.info("Creating issue filter criteria"); - if (!CollectionUtils.isEmpty(JiraConfigHelper.getProjectNameIncludeFilter(configuration)) || !CollectionUtils.isEmpty(JiraConfigHelper.getProjectNameExcludeFilter(configuration)) ) { + if (!CollectionUtils.isEmpty(JiraConfigHelper.getProjectNameIncludeFilter(configuration)) || !CollectionUtils.isEmpty(JiraConfigHelper.getProjectNameExcludeFilter(configuration))) { validateProjectFilters(configuration); } StringBuilder jiraQl = new StringBuilder(UPDATED + GREATER_THAN_EQUALS + ts.toEpochMilli()); diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java index 28a71f55ee..5271c6cfb5 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClient.java @@ -16,7 +16,6 @@ import io.micrometer.core.instrument.Timer; import lombok.extern.slf4j.Slf4j; import org.opensearch.dataprepper.metrics.PluginMetrics; -import org.opensearch.dataprepper.plugins.source.jira.JiraSourceConfig; import org.opensearch.dataprepper.plugins.source.jira.exception.BadRequestException; import org.opensearch.dataprepper.plugins.source.jira.exception.UnAuthorizedException; import org.opensearch.dataprepper.plugins.source.jira.models.SearchResults; @@ -44,42 +43,42 @@ public class JiraRestClient { public static final String REST_API_SEARCH = "rest/api/3/search"; public static final String REST_API_FETCH_ISSUE = "rest/api/3/issue"; - public static final String REST_API_PROJECTS = "/rest/api/3/project/search"; + //public static final String REST_API_PROJECTS = "/rest/api/3/project/search"; public static final String FIFTY = "50"; public static final String START_AT = "startAt"; public static final String MAX_RESULT = "maxResults"; public static final List RETRY_ATTEMPT_SLEEP_TIME = List.of(1, 2, 5, 10, 20, 40); private static final String TICKET_FETCH_LATENCY_TIMER = "ticketFetchLatency"; private static final String SEARCH_CALL_LATENCY_TIMER = "searchCallLatency"; + private static final String PROJECTS_FETCH_LATENCY_TIMER = "projectFetchLatency"; private static final String ISSUES_REQUESTED = "issuesRequested"; + private int sleepTimeMultiplier = 1000; private final RestTemplate restTemplate; private final JiraAuthConfig authConfig; private final Timer ticketFetchLatencyTimer; private final Timer searchCallLatencyTimer; + private final Timer projectFetchLatencyTimer; private final Counter issuesRequestedCounter; - private final PluginMetrics jiraPluginMetrics = PluginMetrics.fromNames("jiraRestClient", "aws"); - private int sleepTimeMultiplier = 1000; - public JiraRestClient(RestTemplate restTemplate, JiraAuthConfig authConfig) { + public JiraRestClient(RestTemplate restTemplate, JiraAuthConfig authConfig, PluginMetrics pluginMetrics) { this.restTemplate = restTemplate; this.authConfig = authConfig; - ticketFetchLatencyTimer = jiraPluginMetrics.timer(TICKET_FETCH_LATENCY_TIMER); - searchCallLatencyTimer = jiraPluginMetrics.timer(SEARCH_CALL_LATENCY_TIMER); - issuesRequestedCounter = jiraPluginMetrics.counter(ISSUES_REQUESTED); + ticketFetchLatencyTimer = pluginMetrics.timer(TICKET_FETCH_LATENCY_TIMER); + searchCallLatencyTimer = pluginMetrics.timer(SEARCH_CALL_LATENCY_TIMER); + projectFetchLatencyTimer = pluginMetrics.timer(PROJECTS_FETCH_LATENCY_TIMER); + issuesRequestedCounter = pluginMetrics.counter(ISSUES_REQUESTED); } /** * Method to get Issues. * - * @param jql input parameter. - * @param startAt the start at - * @param configuration input parameter. + * @param jql input parameter. + * @param startAt the start at * @return InputStream input stream */ @Timed(SEARCH_CALL_LATENCY_TIMER) - public SearchResults getAllIssues(StringBuilder jql, int startAt, - JiraSourceConfig configuration) { + public SearchResults getAllIssues(StringBuilder jql, int startAt) { String url = authConfig.getUrl() + REST_API_SEARCH; diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraIteratorTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraIteratorTest.java index 73e184330a..bd73e3e966 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraIteratorTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraIteratorTest.java @@ -15,6 +15,7 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.plugins.source.jira.models.IssueBean; import org.opensearch.dataprepper.plugins.source.jira.models.SearchResults; import org.opensearch.dataprepper.plugins.source.jira.rest.JiraRestClient; @@ -45,8 +46,6 @@ @ExtendWith(MockitoExtension.class) public class JiraIteratorTest { - private final PluginExecutorServiceProvider executorServiceProvider = new PluginExecutorServiceProvider(); - @Mock private SearchResults mockSearchResults; @Mock @@ -54,12 +53,13 @@ public class JiraIteratorTest { private JiraService jiraService; @Mock private JiraSourceConfig jiraSourceConfig; - private JiraIterator jiraIterator; + private final PluginExecutorServiceProvider executorServiceProvider = new PluginExecutorServiceProvider(); @BeforeEach void setUp() { - jiraService = spy(new JiraService(jiraSourceConfig, jiraRestClient)); + jiraService = spy(new JiraService(jiraSourceConfig, jiraRestClient, + PluginMetrics.fromNames("jiraIteratorTest", "jira"))); } public JiraIterator createObjectUnderTest() { @@ -73,7 +73,7 @@ void testInitialization() { jiraIterator.initialize(Instant.ofEpochSecond(0)); when(mockSearchResults.getIssues()).thenReturn(new ArrayList<>()); when(mockSearchResults.getTotal()).thenReturn(0); - doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt(), any(JiraSourceConfig.class)); + doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt()); assertFalse(jiraIterator.hasNext()); } @@ -104,7 +104,7 @@ void testItemInfoQueueNotEmpty() { mockIssues.add(issue1); when(mockSearchResults.getIssues()).thenReturn(mockIssues); when(mockSearchResults.getTotal()).thenReturn(0); - doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt(), any(JiraSourceConfig.class)); + doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt()); jiraIterator.initialize(Instant.ofEpochSecond(0)); jiraIterator.setCrawlerQWaitTimeMillis(1); @@ -118,7 +118,7 @@ void testStartCrawlerThreads() { jiraIterator.initialize(Instant.ofEpochSecond(0)); jiraIterator.hasNext(); jiraIterator.hasNext(); - assertTrue(jiraIterator.showFutureList().size() == 1); + assertEquals(1, jiraIterator.showFutureList().size()); } @Test @@ -133,7 +133,7 @@ void testFuturesCompleted() throws InterruptedException { mockIssues.add(issue3); when(mockSearchResults.getIssues()).thenReturn(mockIssues); when(mockSearchResults.getTotal()).thenReturn(0); - doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt(), any(JiraSourceConfig.class)); + doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt()); jiraIterator.initialize(Instant.ofEpochSecond(0)); jiraIterator.setCrawlerQWaitTimeMillis(1); @@ -145,12 +145,12 @@ void testFuturesCompleted() throws InterruptedException { } @Test - void testItemInfoQueueEmpty(){ + void testItemInfoQueueEmpty() { jiraIterator = createObjectUnderTest(); List mockIssues = new ArrayList<>(); when(mockSearchResults.getIssues()).thenReturn(mockIssues); when(mockSearchResults.getTotal()).thenReturn(0); - doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt(), any(JiraSourceConfig.class)); + doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt()); jiraIterator.initialize(Instant.ofEpochSecond(0)); jiraIterator.setCrawlerQWaitTimeMillis(1); diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraServiceTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraServiceTest.java index 484ca2caac..80b5c50c71 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraServiceTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/JiraServiceTest.java @@ -18,6 +18,7 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.plugin.PluginConfigVariable; import org.opensearch.dataprepper.plugins.source.jira.configuration.Oauth2Config; import org.opensearch.dataprepper.plugins.source.jira.exception.BadRequestException; @@ -71,11 +72,10 @@ public class JiraServiceTest { private static final Logger log = LoggerFactory.getLogger(JiraServiceTest.class); - private final PluginExecutorServiceProvider executorServiceProvider = new PluginExecutorServiceProvider(); - @Mock private JiraRestClient jiraRestClient; - + private final PluginExecutorServiceProvider executorServiceProvider = new PluginExecutorServiceProvider(); + private final PluginMetrics pluginMetrics = PluginMetrics.fromNames("JiraServiceTest", "jira"); private static InputStream getResourceAsStream(String resourceName) { InputStream inputStream = Thread.currentThread().getContextClassLoader().getResourceAsStream(resourceName); @@ -178,7 +178,7 @@ void testJiraServiceInitialization() throws JsonProcessingException { List issueStatus = new ArrayList<>(); List projectKey = new ArrayList<>(); JiraSourceConfig jiraSourceConfig = createJiraConfiguration(BASIC, issueType, issueStatus, projectKey); - JiraService jiraService = new JiraService(jiraSourceConfig, jiraRestClient); + JiraService jiraService = new JiraService(jiraSourceConfig, jiraRestClient, pluginMetrics); assertNotNull(jiraService); when(jiraRestClient.getIssue(anyString())).thenReturn("test String"); assertNotNull(jiraService.getIssue("test Key")); @@ -193,7 +193,7 @@ public void testGetJiraEntities() throws JsonProcessingException { issueStatus.add("Done"); projectKey.add("KAN"); JiraSourceConfig jiraSourceConfig = createJiraConfiguration(BASIC, issueType, issueStatus, projectKey); - JiraService jiraService = spy(new JiraService(jiraSourceConfig, jiraRestClient)); + JiraService jiraService = spy(new JiraService(jiraSourceConfig, jiraRestClient, pluginMetrics)); List mockIssues = new ArrayList<>(); IssueBean issue1 = createIssueBean(false, false); mockIssues.add(issue1); @@ -206,7 +206,7 @@ public void testGetJiraEntities() throws JsonProcessingException { when(mockSearchResults.getIssues()).thenReturn(mockIssues); when(mockSearchResults.getTotal()).thenReturn(mockIssues.size()); - doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt(), any(JiraSourceConfig.class)); + doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt()); Instant timestamp = Instant.ofEpochSecond(0); Queue itemInfoQueue = new ConcurrentLinkedQueue<>(); @@ -221,7 +221,7 @@ public void buildIssueItemInfoMultipleFutureThreads() throws JsonProcessingExcep List projectKey = new ArrayList<>(); issueType.add("Task"); JiraSourceConfig jiraSourceConfig = createJiraConfiguration(BASIC, issueType, issueStatus, projectKey); - JiraService jiraService = spy(new JiraService(jiraSourceConfig, jiraRestClient)); + JiraService jiraService = spy(new JiraService(jiraSourceConfig, jiraRestClient, pluginMetrics)); List mockIssues = new ArrayList<>(); for (int i = 0; i < 50; i++) { IssueBean issue1 = createIssueBean(false, false); @@ -232,7 +232,7 @@ public void buildIssueItemInfoMultipleFutureThreads() throws JsonProcessingExcep when(mockSearchResults.getIssues()).thenReturn(mockIssues); when(mockSearchResults.getTotal()).thenReturn(100); - doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt(), any(JiraSourceConfig.class)); + doReturn(mockSearchResults).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt()); Instant timestamp = Instant.ofEpochSecond(0); Queue itemInfoQueue = new ConcurrentLinkedQueue<>(); @@ -253,7 +253,7 @@ public void testBadProjectKeys() throws JsonProcessingException { projectKey.add("AAAAAAAAAAAAAA"); JiraSourceConfig jiraSourceConfig = createJiraConfiguration(BASIC, issueType, issueStatus, projectKey); - JiraService jiraService = new JiraService(jiraSourceConfig, jiraRestClient); + JiraService jiraService = new JiraService(jiraSourceConfig, jiraRestClient, pluginMetrics); Instant timestamp = Instant.ofEpochSecond(0); Queue itemInfoQueue = new ConcurrentLinkedQueue<>(); @@ -268,9 +268,9 @@ public void testGetJiraEntitiesException() throws JsonProcessingException { List projectKey = new ArrayList<>(); issueType.add("Task"); JiraSourceConfig jiraSourceConfig = createJiraConfiguration(BASIC, issueType, issueStatus, projectKey); - JiraService jiraService = spy(new JiraService(jiraSourceConfig, jiraRestClient)); + JiraService jiraService = spy(new JiraService(jiraSourceConfig, jiraRestClient, pluginMetrics)); - doThrow(RuntimeException.class).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt(), any(JiraSourceConfig.class)); + doThrow(RuntimeException.class).when(jiraRestClient).getAllIssues(any(StringBuilder.class), anyInt()); Instant timestamp = Instant.ofEpochSecond(0); Queue itemInfoQueue = new ConcurrentLinkedQueue<>(); diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClientTest.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClientTest.java index 3fe0640700..a24b582e97 100644 --- a/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClientTest.java +++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/test/java/org/opensearch/dataprepper/plugins/source/jira/rest/JiraRestClientTest.java @@ -19,6 +19,7 @@ import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.plugins.source.jira.JiraServiceTest; import org.opensearch.dataprepper.plugins.source.jira.JiraSourceConfig; import org.opensearch.dataprepper.plugins.source.jira.exception.BadRequestException; @@ -43,8 +44,6 @@ import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; -import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.BASIC; -import static org.opensearch.dataprepper.plugins.source.jira.utils.Constants.OAUTH2; @ExtendWith(MockitoExtension.class) public class JiraRestClientTest { @@ -57,6 +56,7 @@ public class JiraRestClientTest { @Mock private JiraAuthConfig authConfig; + private final PluginMetrics pluginMetrics = PluginMetrics.fromNames("JiraRestClientTest", "jira"); private static Stream provideHttpStatusCodesWithExceptionClass() { return Stream.of( @@ -74,7 +74,7 @@ public void testFetchingJiraIssue(String configFileName) { doReturn(new ResponseEntity<>(exampleTicketResponse, HttpStatus.OK)).when(restTemplate).getForEntity(any(URI.class), any(Class.class)); JiraSourceConfig jiraSourceConfig = JiraServiceTest.createJiraConfigurationFromYaml(configFileName); JiraAuthConfig authConfig = new JiraAuthFactory(jiraSourceConfig).getObject(); - JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig); + JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig, pluginMetrics); String ticketDetails = jiraRestClient.getIssue("key"); assertEquals(exampleTicketResponse, ticketDetails); } @@ -82,7 +82,7 @@ public void testFetchingJiraIssue(String configFileName) { @ParameterizedTest @MethodSource("provideHttpStatusCodesWithExceptionClass") void testInvokeRestApiTokenExpired(HttpStatus statusCode, Class expectedExceptionType) { - JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig); + JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig, pluginMetrics); jiraRestClient.setSleepTimeMultiplier(1); when(authConfig.getUrl()).thenReturn("https://example.com/rest/api/2/issue/key"); when(restTemplate.getForEntity(any(URI.class), any(Class.class))).thenThrow(new HttpClientErrorException(statusCode)); @@ -91,7 +91,7 @@ void testInvokeRestApiTokenExpired(HttpStatus statusCode, Class expectedExceptio @Test void testInvokeRestApiTokenExpiredInterruptException() throws InterruptedException { - JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig); + JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig, pluginMetrics); when(authConfig.getUrl()).thenReturn("https://example.com/rest/api/2/issue/key"); when(restTemplate.getForEntity(any(URI.class), any(Class.class))).thenThrow(new HttpClientErrorException(HttpStatus.TOO_MANY_REQUESTS)); jiraRestClient.setSleepTimeMultiplier(100000); @@ -111,39 +111,33 @@ void testInvokeRestApiTokenExpiredInterruptException() throws InterruptedExcepti } @Test - public void testGetAllIssuesOauth2() throws JsonProcessingException { + public void testGetAllIssuesOauth2() { List issueType = new ArrayList<>(); - List issueStatus = new ArrayList<>(); - List projectKey = new ArrayList<>(); issueType.add("Task"); - JiraSourceConfig jiraSourceConfig = JiraServiceTest.createJiraConfiguration(OAUTH2, issueType, issueStatus, projectKey); - JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig); + JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig, pluginMetrics); SearchResults mockSearchResults = mock(SearchResults.class); doReturn("http://mock-service.jira.com/").when(authConfig).getUrl(); doReturn(new ResponseEntity<>(mockSearchResults, HttpStatus.OK)).when(restTemplate).getForEntity(any(URI.class), any(Class.class)); - SearchResults results = jiraRestClient.getAllIssues(jql, 0, jiraSourceConfig); + SearchResults results = jiraRestClient.getAllIssues(jql, 0); assertNotNull(results); } @Test - public void testGetAllIssuesBasic() throws JsonProcessingException { + public void testGetAllIssuesBasic() { List issueType = new ArrayList<>(); - List issueStatus = new ArrayList<>(); - List projectKey = new ArrayList<>(); issueType.add("Task"); - JiraSourceConfig jiraSourceConfig = JiraServiceTest.createJiraConfiguration(BASIC, issueType, issueStatus, projectKey); - JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig); + JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig, pluginMetrics); SearchResults mockSearchResults = mock(SearchResults.class); when(authConfig.getUrl()).thenReturn("https://example.com/"); doReturn(new ResponseEntity<>(mockSearchResults, HttpStatus.OK)).when(restTemplate).getForEntity(any(URI.class), any(Class.class)); - SearchResults results = jiraRestClient.getAllIssues(jql, 0, jiraSourceConfig); + SearchResults results = jiraRestClient.getAllIssues(jql, 0); assertNotNull(results); } @Test public void testRestApiAddressValidation() throws JsonProcessingException { when(authConfig.getUrl()).thenReturn("https://224.0.0.1/"); - JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig); + JiraRestClient jiraRestClient = new JiraRestClient(restTemplate, authConfig, pluginMetrics); assertThrows(BadRequestException.class, () -> jiraRestClient.getIssue("TEST-1")); } diff --git a/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/base/Crawler.java b/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/base/Crawler.java index 7a8c4f2275..d93461ee79 100644 --- a/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/base/Crawler.java +++ b/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/base/Crawler.java @@ -30,12 +30,10 @@ public class Crawler { private static final Logger log = LoggerFactory.getLogger(Crawler.class); private final Timer crawlingTimer; - private final PluginMetrics pluginMetrics = - PluginMetrics.fromNames("sourceCrawler", "crawler"); private final CrawlerClient client; - public Crawler(CrawlerClient client) { + public Crawler(CrawlerClient client, PluginMetrics pluginMetrics) { this.client = client; this.crawlingTimer = pluginMetrics.timer("crawlingTime"); } diff --git a/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/scheduler/WorkerScheduler.java b/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/scheduler/WorkerScheduler.java index e738c0e19c..6a6fbd2471 100644 --- a/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/scheduler/WorkerScheduler.java +++ b/data-prepper-plugins/saas-source-plugins/source-crawler/src/main/java/org/opensearch/dataprepper/plugins/source/source_crawler/coordination/scheduler/WorkerScheduler.java @@ -40,7 +40,6 @@ public class WorkerScheduler implements Runnable { private final Counter acknowledgementSetSuccesses; private final Counter acknowledgementSetFailures; private final String sourcePluginName; - private final String SOURCE_PLUGIN_NAME = "sourcePluginName"; public WorkerScheduler(final String sourcePluginName, @@ -58,8 +57,8 @@ public WorkerScheduler(final String sourcePluginName, this.acknowledgementSetManager = acknowledgementSetManager; this.pluginMetrics = pluginMetrics; - this.acknowledgementSetSuccesses = pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME, SOURCE_PLUGIN_NAME, sourcePluginName); - this.acknowledgementSetFailures = pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME, SOURCE_PLUGIN_NAME, sourcePluginName); + this.acknowledgementSetSuccesses = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME); + this.acknowledgementSetFailures = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME); } @Override diff --git a/data-prepper-plugins/saas-source-plugins/source-crawler/src/test/java/org/opensearch/dataprepper/plugins/source/source_crawler/base/CrawlerTest.java b/data-prepper-plugins/saas-source-plugins/source-crawler/src/test/java/org/opensearch/dataprepper/plugins/source/source_crawler/base/CrawlerTest.java index d88cd7aef1..2010a5c3b4 100644 --- a/data-prepper-plugins/saas-source-plugins/source-crawler/src/test/java/org/opensearch/dataprepper/plugins/source/source_crawler/base/CrawlerTest.java +++ b/data-prepper-plugins/saas-source-plugins/source-crawler/src/test/java/org/opensearch/dataprepper/plugins/source/source_crawler/base/CrawlerTest.java @@ -5,6 +5,7 @@ import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; @@ -50,10 +51,11 @@ public class CrawlerTest { private LeaderPartition leaderPartition; private Crawler crawler; private final Instant lastPollTime = Instant.ofEpochMilli(0); + private final PluginMetrics pluginMetrics = PluginMetrics.fromNames("CrawlerTest", "crawler"); @BeforeEach public void setup() { - crawler = new Crawler(client); + crawler = new Crawler(client, pluginMetrics); when(leaderPartition.getProgressState()).thenReturn(Optional.of(new LeaderProgressState(lastPollTime))); } From 03f5539d0bc82a3da3ab7c66cb54109ba9a70c9f Mon Sep 17 00:00:00 2001 From: Krishna Kondaka Date: Fri, 31 Jan 2025 13:25:13 -0800 Subject: [PATCH 07/18] Fixed javadoc errors (#5379) Signed-off-by: Krishna Kondaka --- .../model/annotations/AlsoRequired.java | 3 +++ .../model/annotations/ConditionalRequired.java | 6 ++++++ .../model/annotations/ExampleValues.java | 1 + .../dataprepper/model/processor/Processor.java | 1 + .../transformer/DynamicConfigTransformer.java | 6 +++--- .../processor/aggregate/AggregateAction.java | 2 +- .../aws/api/AwsCredentialsOptions.java | 2 +- .../dynamodb/converter/RecordConverter.java | 2 ++ .../source/dynamodb/leader/ShardCache.java | 2 +- .../dataprepper/plugins/dlq/DlqProvider.java | 1 + .../dataprepper/plugins/dlq/DlqWriter.java | 4 ++-- .../geoip/extension/GeoIPProcessorService.java | 2 ++ .../geoip/extension/api/GeoIPDatabaseReader.java | 2 ++ .../geoip/extension/api/GeoIpConfigSupplier.java | 2 +- .../databasedownload/DatabaseReaderBuilder.java | 1 + .../databasedownload/HttpDBDownloadService.java | 5 ++++- .../databasedownload/LocalDBDownloadService.java | 2 ++ .../extension/databasedownload/S3DBService.java | 2 ++ .../plugins/geoip/processor/GeoIPProcessor.java | 9 +++++---- .../opensearch/dataprepper/http/codec/Codec.java | 6 ++---- .../dataprepper/http/codec/JsonCodec.java | 1 - .../InMemorySourceCoordinationStore.java | 1 + .../common/thread/KafkaPluginThreadFactory.java | 1 + .../plugins/mongo/converter/RecordConverter.java | 2 ++ .../opensearch/bulk/AccumulatingBulkRequest.java | 4 ++-- .../sink/opensearch/bulk/SerializedJson.java | 1 + .../opensearch/index/AbstractIndexManager.java | 2 +- .../index/IndexTemplateAPIWrapper.java | 3 +++ .../opensearch/worker/client/SearchAccessor.java | 8 +++++--- .../otelmetrics/OTelMetricsProtoHelper.java | 10 ++++------ .../processor/parse/AbstractParseProcessor.java | 4 ++++ .../processor/parse/CommonParseConfig.java | 4 ++++ .../rds/resync/CascadingActionDetector.java | 6 ++++++ .../plugins/codec/parquet/S3OutputStream.java | 2 ++ .../plugins/sink/s3/KeyGenerator.java | 1 + .../dataprepper/plugins/sink/s3/S3Sink.java | 3 +++ .../plugins/sink/s3/S3SinkConfig.java | 1 + .../plugins/sink/s3/S3SinkService.java | 8 +++++++- .../plugins/sink/s3/accumulator/ObjectKey.java | 4 +++- .../plugins/source/s3/S3EventNotification.java | 3 ++- .../plugins/source/s3/S3ObjectHandler.java | 2 ++ .../plugins/source/jira/JiraService.java | 6 ++++-- .../source/jira/utils/AddressValidation.java | 3 ++- .../source/jira/utils/JiraConfigHelper.java | 2 ++ .../plugins/processor/OTelHelper.java | 8 ++++---- .../processor/ServiceMapRelationship.java | 16 ++++++++++++---- .../plugins/source/sqs/MessageFieldStrategy.java | 2 ++ .../test/helper/ReflectivelySetField.java | 1 + 48 files changed, 125 insertions(+), 45 deletions(-) diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/AlsoRequired.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/AlsoRequired.java index ae4275a3cc..e07a315f1d 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/AlsoRequired.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/AlsoRequired.java @@ -16,6 +16,7 @@ public @interface AlsoRequired { /** * Array of Required annotations, each representing a required property with its allowed values. + * @return returns array of required values */ Required[] values(); @@ -25,11 +26,13 @@ @interface Required { /** * Name of the required property. + * @return returns name */ String name(); /** * Allowed values for the required property. The default value of {} means any non-null value is allowed. + * @return returns array of allowed values */ String[] allowedValues() default {}; } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/ConditionalRequired.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/ConditionalRequired.java index 7808b0d321..4234c15f96 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/ConditionalRequired.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/ConditionalRequired.java @@ -13,6 +13,7 @@ public @interface ConditionalRequired { /** * Array of if-then-else requirements. + * @return returns array of if and else values */ IfThenElse[] value(); @@ -22,14 +23,17 @@ @interface IfThenElse { /** * Array of property schemas involved in if condition. + * @return returns of if schema properties */ SchemaProperty[] ifFulfilled(); /** * Array of property schemas involved in then expectation. + * @return returns of then schema properties */ SchemaProperty[] thenExpect(); /** * Array of property schemas involved in else expectation. + * @return returns of else schema properties */ SchemaProperty[] elseExpect() default {}; } @@ -40,10 +44,12 @@ @interface SchemaProperty { /** * Name of the property. + * @return returns schema field */ String field(); /** * Value of the property. Empty string means any non-null value is allowed. + * @return returns schema value */ String value() default ""; } diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/ExampleValues.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/ExampleValues.java index 5ab7593cd3..145a31a8a3 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/ExampleValues.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/annotations/ExampleValues.java @@ -44,6 +44,7 @@ /** * A description of the example value. + * @return returns description * * @since 2.11 */ diff --git a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/processor/Processor.java b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/processor/Processor.java index 784758fa95..b28dfa25df 100644 --- a/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/processor/Processor.java +++ b/data-prepper-api/src/main/java/org/opensearch/dataprepper/model/processor/Processor.java @@ -37,6 +37,7 @@ public interface Processor, OutputRecord extends R * @since 2.11 * Indicates if the processor holds the events or not * Holding events indicates that the events are not ready to be released. + * @return returns if events are held by the processor or not */ default boolean holdsEvents() { return false; diff --git a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/transformer/DynamicConfigTransformer.java b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/transformer/DynamicConfigTransformer.java index c782515ef3..ad333954f2 100644 --- a/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/transformer/DynamicConfigTransformer.java +++ b/data-prepper-pipeline-parser/src/main/java/org/opensearch/dataprepper/pipeline/parser/transformer/DynamicConfigTransformer.java @@ -486,9 +486,9 @@ public Object invokeMethod(String methodName, Class parameterType, Object arg * Replaces template node in the jsonPath with the node from * original json. * - * @param root - * @param jsonPath - * @param newNode + * @param root json root node + * @param jsonPath json path + * @param newNode new node to be repalces with */ public void replaceNode(JsonNode root, String jsonPath, JsonNode newNode) { try { diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateAction.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateAction.java index 541cd15d3d..6e17e01188 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateAction.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateAction.java @@ -32,7 +32,7 @@ default AggregateActionResponse handleEvent(final Event event, final AggregateAc /** * indicates if the action holds the events or not - * + * @return returns if events are held by the processor or not */ default boolean holdsEvents() { return false; diff --git a/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsCredentialsOptions.java b/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsCredentialsOptions.java index 1c0baea3bd..37d161cc81 100644 --- a/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsCredentialsOptions.java +++ b/data-prepper-plugins/aws-plugin-api/src/main/java/org/opensearch/dataprepper/aws/api/AwsCredentialsOptions.java @@ -139,7 +139,7 @@ public Builder withStsHeaderOverrides(final Map stsHeaderOverrid /** * Configures whether to use default credentials. * - * @param useDefaultCredentialsProvider + * @param useDefaultCredentialsProvider boolean indicating if default credentials provider should be used or not * @return The {@link Builder} for continuing to build */ public Builder withUseDefaultCredentialsProvider(final boolean useDefaultCredentialsProvider) { diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/RecordConverter.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/RecordConverter.java index dd4c6fe3e3..6e9b9c624c 100644 --- a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/RecordConverter.java +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/RecordConverter.java @@ -77,9 +77,11 @@ void flushBuffer() throws Exception { /** * Add event record to buffer * + * @param acknowledgementSet acknowledgement set * @param data A map to hold event data, note that it may be empty. * @param keys A map to hold the keys (partition key and sort key) * @param eventCreationTimeMillis Creation timestamp of the event + * @param eventVersionNumber Event version number * @param eventName Event name * @param userIdentity UserIdentity for TTL based deletes * @throws Exception Exception if failed to write to buffer. diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/leader/ShardCache.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/leader/ShardCache.java index d259bbe778..cf266b3028 100644 --- a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/leader/ShardCache.java +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/leader/ShardCache.java @@ -49,7 +49,7 @@ public void put(final String shardId, final String parentShardId) { * Get child shard ids by parent shard id from cache. * If none is found, return null. * - * @param parentShardId + * @param parentShardId parent shard id * @return a list of Child Shard IDs */ public List get(String parentShardId) { diff --git a/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/DlqProvider.java b/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/DlqProvider.java index 863a84643b..16881a8d67 100644 --- a/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/DlqProvider.java +++ b/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/DlqProvider.java @@ -21,6 +21,7 @@ public interface DlqProvider { * Allows implementors to provide a {@link DlqWriter}. This may be optional, in which case it is not used. * @param pluginMetricsScope the {@link org.opensearch.dataprepper.metrics.PluginMetrics} component scope. * This is used to place the DLQ metrics under the correct parent plugin. + * @return returns dlq writer * @since 2.2 */ default Optional getDlqWriter(final String pluginMetricsScope) { diff --git a/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/DlqWriter.java b/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/DlqWriter.java index bc1a7011df..a47c2bc3a9 100644 --- a/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/DlqWriter.java +++ b/data-prepper-plugins/failures-common/src/main/java/org/opensearch/dataprepper/plugins/dlq/DlqWriter.java @@ -23,7 +23,7 @@ public interface DlqWriter { * @param dlqObjects the list of objects to be written to the DLQ * @param pipelineName the pipeline the DLQ object is associated with. * @param pluginId the id of the plugin the DLQ object is associated with. - * @throws IOException + * @throws IOException io exception * * @since 2.2 */ @@ -31,7 +31,7 @@ public interface DlqWriter { /** * Closes any open connections to the DLQ - * @throws IOException + * @throws IOException io exception * * @since 2.2 */ diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/GeoIPProcessorService.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/GeoIPProcessorService.java index 21ab0f1190..580d677e55 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/GeoIPProcessorService.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/GeoIPProcessorService.java @@ -30,6 +30,8 @@ public class GeoIPProcessorService { * GeoIPProcessorService constructor for initialization of required attributes * * @param geoIpServiceConfig geoIpServiceConfig + * @param geoIPDatabaseManager geoIPDatabaseManager + * @param readLock readLock */ public GeoIPProcessorService(final GeoIpServiceConfig geoIpServiceConfig, final GeoIPDatabaseManager geoIPDatabaseManager, diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/api/GeoIPDatabaseReader.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/api/GeoIPDatabaseReader.java index 8f611cc9b4..52ee171eb4 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/api/GeoIPDatabaseReader.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/api/GeoIPDatabaseReader.java @@ -38,6 +38,8 @@ public interface GeoIPDatabaseReader extends AutoCloseable { * Gets the geo data from the {@link com.maxmind.geoip2.DatabaseReader} * * @param inetAddress InetAddress + * @param fields fields + * @param geoIPDatabases geo ip databases * @return Map of geo field and value pairs from IP address * * @since 2.7 diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/api/GeoIpConfigSupplier.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/api/GeoIpConfigSupplier.java index 11a9db45cf..bdb07f415d 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/api/GeoIpConfigSupplier.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/api/GeoIpConfigSupplier.java @@ -17,7 +17,7 @@ */ public interface GeoIpConfigSupplier { /** - * Returns the {@link GeoIPProcessorService} + * @return returns the {@link GeoIPProcessorService} * * @since 2.7 */ diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/DatabaseReaderBuilder.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/DatabaseReaderBuilder.java index 8a6caaabaf..cc90c7bf50 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/DatabaseReaderBuilder.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/DatabaseReaderBuilder.java @@ -24,6 +24,7 @@ public DatabaseReaderBuilder() { * @param databasePath databasePath * @param cacheSize cacheSize * @return DatabaseReader + * @throws IOException io exception */ public DatabaseReader buildReader(final Path databasePath, final int cacheSize) throws IOException { return new DatabaseReader.Builder(databasePath.toFile()) diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/HttpDBDownloadService.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/HttpDBDownloadService.java index c382e72726..4bff5ed156 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/HttpDBDownloadService.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/HttpDBDownloadService.java @@ -31,7 +31,9 @@ public class HttpDBDownloadService implements DBSource { /** * HttpDBDownloadService constructor for initialisation of attributes - * @param destinationDirectory destinationDirectory + * @param destinationDirectory destination directory + * @param geoIPFileManager geo ip file manager + * @param maxMindDatabaseConfig max min database config */ public HttpDBDownloadService(final String destinationDirectory, final GeoIPFileManager geoIPFileManager, @@ -85,6 +87,7 @@ private File decompressAndgetTarFile(final String tarFolderPath, final String do /** * Build Request And DownloadFile * @param url url + * @param downloadTarFilepath download tar file path */ public void buildRequestAndDownloadFile(final String url, final String downloadTarFilepath) { downloadDBFileFromMaxmind(url, downloadTarFilepath); diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/LocalDBDownloadService.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/LocalDBDownloadService.java index 9d60727315..4b8ea96676 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/LocalDBDownloadService.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/LocalDBDownloadService.java @@ -22,6 +22,7 @@ public class LocalDBDownloadService implements DBSource { /** * LocalDBDownloadService constructor for initialisation of attributes * @param destinationDirectory destinationDirectory + * @param maxMindDatabaseConfig maxMindDatabaseConfig */ public LocalDBDownloadService(final String destinationDirectory, final MaxMindDatabaseConfig maxMindDatabaseConfig) { this.destinationDirectory = destinationDirectory; @@ -30,6 +31,7 @@ public LocalDBDownloadService(final String destinationDirectory, final MaxMindDa /** * Initialisation of Download from local file path + * @throws Exception exception */ @Override public void initiateDownload() throws Exception { diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/S3DBService.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/S3DBService.java index c3f57e76cb..085f1d37bb 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/S3DBService.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/extension/databasedownload/S3DBService.java @@ -26,7 +26,9 @@ public class S3DBService implements DBSource { /** * S3DBService constructor for initialisation of attributes * + * @param awsAuthenticationOptionsConfig awsAuthenticationOptionsConfig * @param destinationDirectory destinationDirectory + * @param maxMindDatabaseConfig maxMindDatabaseConfig */ public S3DBService(final AwsAuthenticationOptionsConfig awsAuthenticationOptionsConfig, final String destinationDirectory, diff --git a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessor.java b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessor.java index afb0ed23d3..d039c18d22 100644 --- a/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessor.java +++ b/data-prepper-plugins/geoip-processor/src/main/java/org/opensearch/dataprepper/plugins/geoip/processor/GeoIPProcessor.java @@ -65,9 +65,10 @@ public class GeoIPProcessor extends AbstractProcessor, Record */ public interface Codec { /** * parse the request into custom type * * @param httpData The content of the original HTTP request - * @return The target data type + * @return T The target data type + * @throws IOException A failure while parsing data. */ T parse(HttpData httpData) throws IOException; @@ -33,9 +33,7 @@ public interface Codec { /* * Serializes the HttpData and split into multiple bodies based on splitLength. - *

* The serialized bodies are passed to the serializedBodyConsumer. - *

* This API will split into multiple bodies based on splitLength. Note that if a single * item is larger than this, it will be output and exceed that length. * diff --git a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java index 6306366816..792d55e50d 100644 --- a/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java +++ b/data-prepper-plugins/http-source-common/src/main/java/org/opensearch/dataprepper/http/codec/JsonCodec.java @@ -27,7 +27,6 @@ /** * JsonCodec parses the json array format HTTP data into List<{@link String}>. * TODO: replace output List<String> with List<InternalModel> type - *

*/ public class JsonCodec implements Codec> { private static final ObjectMapper mapper = new ObjectMapper(); diff --git a/data-prepper-plugins/in-memory-source-coordination-store/src/main/java/org/opensearch/dataprepper/plugins/sourcecoordinator/inmemory/InMemorySourceCoordinationStore.java b/data-prepper-plugins/in-memory-source-coordination-store/src/main/java/org/opensearch/dataprepper/plugins/sourcecoordinator/inmemory/InMemorySourceCoordinationStore.java index 2bf6922f86..857ce79bad 100644 --- a/data-prepper-plugins/in-memory-source-coordination-store/src/main/java/org/opensearch/dataprepper/plugins/sourcecoordinator/inmemory/InMemorySourceCoordinationStore.java +++ b/data-prepper-plugins/in-memory-source-coordination-store/src/main/java/org/opensearch/dataprepper/plugins/sourcecoordinator/inmemory/InMemorySourceCoordinationStore.java @@ -40,6 +40,7 @@ public InMemorySourceCoordinationStore(final PluginSetting pluginSetting) { /** * For Testing + * @param inMemoryPartitionAccessor inMemoryPartitionAccessor */ public InMemorySourceCoordinationStore(final InMemoryPartitionAccessor inMemoryPartitionAccessor) { this.inMemoryPartitionAccessor = inMemoryPartitionAccessor; diff --git a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactory.java b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactory.java index b5dede6cda..0331ffea32 100644 --- a/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactory.java +++ b/data-prepper-plugins/kafka-plugins/src/main/java/org/opensearch/dataprepper/plugins/kafka/common/thread/KafkaPluginThreadFactory.java @@ -52,6 +52,7 @@ public static KafkaPluginThreadFactory defaultExecutorThreadFactory(final String * Creates an instance specifically for use with {@link Executors}. * * @param kafkaPluginType The name of the plugin type. e.g. sink, source, buffer + * @param kafkaTopic kafka topic name * @return An instance of the {@link KafkaPluginThreadFactory}. */ public static KafkaPluginThreadFactory defaultExecutorThreadFactory( diff --git a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/converter/RecordConverter.java b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/converter/RecordConverter.java index e6516c0348..81c78cbe7c 100644 --- a/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/converter/RecordConverter.java +++ b/data-prepper-plugins/mongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/converter/RecordConverter.java @@ -57,6 +57,7 @@ private String getAttributeValue(final Map data, final String at * @param eventCreateTimeEpochMillis Creation timestamp of the event in epoch millis * @param eventVersionNumber Event version number to handle conflicts * @param eventName Event name + * @param primaryKeyBsonType primaryKeyBsonType * @return Jackson document event */ public Event convert(final String record, @@ -98,6 +99,7 @@ public Event convert(final String record, * @param record record that will be converted to Event. * @param eventCreationTimeMillis Creation timestamp of the event * @param eventVersionNumber Event version number to handle conflicts + * @param primaryKeyBsonType primaryKeyBsonType * @return Jackson document event */ public Event convert(final String record, diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/AccumulatingBulkRequest.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/AccumulatingBulkRequest.java index 34f0d489e0..73a48ae0b6 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/AccumulatingBulkRequest.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/AccumulatingBulkRequest.java @@ -10,8 +10,8 @@ /** * Accumulates Bulk Requests. * - * @param - * @param + * @param OperationType + * @param RequestType */ public interface AccumulatingBulkRequest { long estimateSizeInBytesWithDocument(O documentOrOperation); diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJson.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJson.java index d85a5992c5..c1b5ae3799 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJson.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/bulk/SerializedJson.java @@ -25,6 +25,7 @@ public interface SerializedJson extends SizedDocument { * @param jsonString The serialized JSON string which forms this JSON data. * @param docId Optional documment ID string * @param routingField Optional routing field string + * @param pipelineField pipeline Field * @return A new {@link SerializedJson}. */ static SerializedJson fromStringAndOptionals(String jsonString, String docId, String routingField, String pipelineField) { diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/AbstractIndexManager.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/AbstractIndexManager.java index bb1565dcd1..1a65396c39 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/AbstractIndexManager.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/sink/opensearch/index/AbstractIndexManager.java @@ -217,7 +217,7 @@ private String getISMEnabled(final GetClusterSettingsResponse response) { /** * Setups anything required for the index. * - * @throws IOException + * @throws IOException io exception */ @Override public void setupIndex() throws IOException { 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 index 8a267d15b3..4b967666fb 100644 --- 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 @@ -11,6 +11,7 @@ public interface IndexTemplateAPIWrapper { * Create or update the index template * * @param indexTemplate The {@link IndexTemplate} to create or update. + * @throws IOException io exception */ void putTemplate(IndexTemplate indexTemplate) throws IOException; @@ -18,6 +19,8 @@ public interface IndexTemplateAPIWrapper { * Retrieve the existing index template * * @param name The index template name to retrieve by. + * @throws IOException io exception + * @return an {@code Optional} containing the template if found, otherwise an empty {@code Optional}. */ Optional getTemplate(String name) throws IOException; } diff --git a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessor.java b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessor.java index 3b34acba97..7fa8ba2b71 100644 --- a/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessor.java +++ b/data-prepper-plugins/opensearch/src/main/java/org/opensearch/dataprepper/plugins/source/opensearch/worker/client/SearchAccessor.java @@ -40,7 +40,7 @@ public interface SearchAccessor { /** * Searches using a PIT context * @param searchPointInTimeRequest payload for searching with PIT context - * @return + * @return SearchWithSearchAfterResults seach results * @since 2.4 */ SearchWithSearchAfterResults searchWithPit(SearchPointInTimeRequest searchPointInTimeRequest); @@ -55,7 +55,7 @@ public interface SearchAccessor { /** * Creates scroll context * @param createScrollRequest payload for creating the scroll context - * @return + * @return CreateScrollResponse create scroll response * @since 2.4 */ CreateScrollResponse createScroll(CreateScrollRequest createScrollRequest); @@ -63,7 +63,7 @@ public interface SearchAccessor { /** * Search with scroll context. * @param searchScrollRequest payload for searching with scroll context - * @return + * @return SearchScrollResponse search scroll response */ SearchScrollResponse searchWithScroll(SearchScrollRequest searchScrollRequest); @@ -75,6 +75,8 @@ public interface SearchAccessor { /** * Searches with sort and search_after without using any search contexts (Point-in-Time or Scroll) + * @param noSearchContextSearchRequest nosearch context request + * @return SearchWithSearchAfterResults search results */ SearchWithSearchAfterResults searchWithoutSearchContext(NoSearchContextSearchRequest noSearchContextSearchRequest); } diff --git a/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OTelMetricsProtoHelper.java b/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OTelMetricsProtoHelper.java index a925bb4730..3a90eaea16 100644 --- a/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OTelMetricsProtoHelper.java +++ b/data-prepper-plugins/otel-metrics-raw-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/otelmetrics/OTelMetricsProtoHelper.java @@ -73,7 +73,7 @@ private OTelMetricsProtoHelper() { * Converts an {@link AnyValue} into its appropriate data type * * @param value The value to convert - * @return + * @return returns converted value object */ public static Object convertAnyValue(final AnyValue value) { switch (value.getValueCase()) { @@ -125,7 +125,6 @@ public static Map convertKeysOfDataPointAttributes(final NumberD /** * Unpacks the List of {@link KeyValue} object into a Map. - *

* Converts the keys into an os friendly format and casts the underlying data into its actual type? * * @param attributesList The list of {@link KeyValue} objects to process @@ -138,7 +137,6 @@ public static Map unpackKeyValueList(List attributesLi /** * Unpacks the List of {@link KeyValue} object into a Map. - *

* Converts the keys into an os friendly format and casts the underlying data into its actual type? * * @param attributesList The list of {@link KeyValue} objects to process @@ -194,6 +192,7 @@ public static Map getResourceAttributes(final Resource resource) /** * Extracts the name and version of the used instrumentation library used * + * @param instrumentationLibrary instrumentationLibrary * @return A map, containing information about the instrumentation library */ public static Map getInstrumentationLibraryAttributes(final InstrumentationLibrary instrumentationLibrary) { @@ -210,6 +209,7 @@ public static Map getInstrumentationLibraryAttributes(final Inst /** * Extracts the name and version of the used instrumentation scope used * + * @param instrumentationScope instrumentationScope * @return A map, containing information about the instrumentation scope */ public static Map getInstrumentationScopeAttributes(final InstrumentationScope instrumentationScope) { @@ -260,9 +260,7 @@ public static List getQuantileValues(List * the OTel metrics proto spec - *

- * The boundaries for bucket at index i are: - *

+ *

The boundaries for bucket at index i are:

*
{@code
      * (-infinity, explicit_bounds[i]) for i == 0
      * (explicit_bounds[i-1], +infinity) for i == size(explicit_bounds)
diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java
index 8f5e650a4f..e294f3c283 100644
--- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java
+++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/AbstractParseProcessor.java
@@ -84,6 +84,8 @@ protected AbstractParseProcessor(final PluginMetrics pluginMetrics,
     }
 
     /**
+     * @param message message
+     * @param context context
      * @return Optional HashMap of the parsed value - empty if the message was invalid (be sure to log the error)
      */
     protected abstract Optional> readValue(String message, Event context);
@@ -222,6 +224,8 @@ private String getEndOfPointer(final String trimmedPointer) {
 
     /**
      * Trim the pointer and change each front slash / to be a dot (.) to process
+     * @param pointer input string
+     * @return returns replaced string
      */
     private String normalizePointerStructure(final String pointer) {
         return pointer.replace('/','.');
diff --git a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java
index d52282043a..fc91f91572 100644
--- a/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java
+++ b/data-prepper-plugins/parse-json-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/parse/CommonParseConfig.java
@@ -52,23 +52,27 @@ public interface CommonParseConfig {
     /**
      * An optional setting used to specify whether the destination field should be overwritten if it already exists.
      * Defaults to true.
+     * @return returns  overwrite if destination exists flag
      */
     boolean getOverwriteIfDestinationExists();
 
     /**
      * An optional setting used to request dropping the original raw message after successfully parsing the input event.
      * Defaults to false.
+     * @return returns delete source required flag
      */
     boolean isDeleteSourceRequested();
 
     /**
      * An optional setting used to determine how to handle parsing errors. Default is skip, which includes logging the error
      * and passing the failed Event downstream to the next processor.
+     * @return returns handle failed events option
      */
     HandleFailedEventsOption getHandleFailedEventsOption();
 
     /**
      * An optional setting used to determine the depth of the input to handle
+     * @return returns depth
      */
     int getDepth();
 }
diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java
index b8e5a21662..cb7f060976 100644
--- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java
+++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java
@@ -83,6 +83,9 @@ public Map getParentTableMap(StreamPartition streamPartitio
 
     /**
      * Detects if a binlog event contains cascading updates and if detected, creates resync partitions
+     * @param event event
+     * @param parentTableMap parent table map
+     * @param tableMetadata table meta data
      */
     public void detectCascadingUpdates(Event event, Map parentTableMap, TableMetadata tableMetadata) {
         final UpdateRowsEventData data = event.getData();
@@ -140,6 +143,9 @@ public void detectCascadingUpdates(Event event, Map parentT
 
     /**
      * Detects if a binlog event contains cascading deletes and if detected, creates resync partitions
+     * @param event event
+     * @param parentTableMap parent table map
+     * @param tableMetadata table meta data
      */
     public void detectCascadingDeletes(Event event, Map parentTableMap, TableMetadata tableMetadata) {
         if (parentTableMap.containsKey(tableMetadata.getFullTableName())) {
diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java
index b4538acaf8..9830ef027b 100644
--- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java
+++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/codec/parquet/S3OutputStream.java
@@ -93,6 +93,8 @@ public class S3OutputStream extends PositionOutputStream {
      * @param s3Client the AmazonS3 client
      * @param bucketSupplier  name of the bucket
      * @param keySupplier     path within the bucket
+     * @param defaultBucket default bucket
+     * @param bucketOwnerProvider bucket owner provider
      */
     public S3OutputStream(final S3AsyncClient s3Client,
                           final Supplier bucketSupplier,
diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java
index 7a742d0217..423df4b6a4 100644
--- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java
+++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/KeyGenerator.java
@@ -30,6 +30,7 @@ public KeyGenerator(final S3SinkConfig s3SinkConfig,
     /**
      * Generate the s3 object path prefix and object file name.
      *
+     * @param event event
      * @return object key path.
      */
     public String generateKeyForEvent(final Event event) {
diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java
index c4c88dc323..ad81cf9d0c 100644
--- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java
+++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3Sink.java
@@ -60,6 +60,9 @@ public class S3Sink extends AbstractSink> {
      * @param pluginSetting dp plugin settings.
      * @param s3SinkConfig  s3 sink configurations.
      * @param pluginFactory dp plugin factory.
+     * @param sinkContext sink context
+     * @param awsCredentialsSupplier aws credentials supplier
+     * @param expressionEvaluator expression evaluator
      */
     @DataPrepperPluginConstructor
     public S3Sink(final PluginSetting pluginSetting,
diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java
index f5e36ea643..15279dae24 100644
--- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java
+++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkConfig.java
@@ -126,6 +126,7 @@ public ThresholdOptions getThresholdOptions() {
 
     /**
      * Threshold configuration for the aggregation of all S3 groups
+     * @return returns aggregate threshold options
      */
     public AggregateThresholdOptions getAggregateThresholdOptions() { return aggregateThresholdOptions; }
 
diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java
index 571a952f01..811ab3246b 100644
--- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java
+++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/S3SinkService.java
@@ -67,7 +67,10 @@ public class S3SinkService {
 
     /**
      * @param s3SinkConfig  s3 sink related configuration.
-     * @param pluginMetrics metrics.
+     * @param codecContext codecContext.
+     * @param retrySleepTime retrySleepTime
+     * @param pluginMetrics pluginMetrics
+     * @param s3GroupManager s3GroupManager
      */
     public S3SinkService(final S3SinkConfig s3SinkConfig,
                          final OutputCodecContext codecContext,
@@ -169,6 +172,9 @@ void output(Collection> records) {
     }
 
     /**
+     * @param completableFutures completableFutures
+     * @param s3Group s3 group
+     * @param forceFlush force flush flag
      * @return whether the flush was attempted
      */
     private boolean flushToS3IfNeeded(final List> completableFutures, final S3Group s3Group, final boolean forceFlush) {
diff --git a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/ObjectKey.java b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/ObjectKey.java
index 65ac8edcef..637e72c76d 100644
--- a/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/ObjectKey.java
+++ b/data-prepper-plugins/s3-sink/src/main/java/org/opensearch/dataprepper/plugins/sink/s3/accumulator/ObjectKey.java
@@ -69,6 +69,8 @@ private static String buildingPathPrefixInternal(final S3SinkConfig s3SinkConfig
      *
      * @param s3SinkConfig s3 sink configuration
      * @param codecExtension extension
+     * @param event event
+     * @param expressionEvaluator expressionEvaluator
      * @return s3 object name with prefix
      */
     public static String objectFileName(final S3SinkConfig s3SinkConfig,
@@ -86,4 +88,4 @@ public static String objectFileName(final S3SinkConfig s3SinkConfig,
                     (codecExtension!=null? codecExtension : DEFAULT_CODEC_FILE_EXTENSION);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3EventNotification.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3EventNotification.java
index 2220eea653..b81ef6d92f 100644
--- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3EventNotification.java
+++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3EventNotification.java
@@ -146,6 +146,7 @@ public String getUrlDecodedKey() {
 
     /**
      * @deprecated use {@link #getSizeAsLong()} instead.
+     * @return returns size
      */
     @Deprecated
     @JsonIgnore
@@ -397,4 +398,4 @@ public GlacierEventDataEntity getGlacierEventData() {
       return glacierEventData;
     }
   }
-}
\ No newline at end of file
+}
diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ObjectHandler.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ObjectHandler.java
index fd1122a113..4a9aab9f35 100644
--- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ObjectHandler.java
+++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/S3ObjectHandler.java
@@ -18,6 +18,8 @@ public interface S3ObjectHandler {
      * Parse S3 object content using S3 object reference and pushing to buffer
      * @param s3ObjectReference Contains bucket and s3 object details
      * @param acknowledgementSet acknowledgement set for the object
+     * @param sourceCoordinator source coordinator
+     * @param partitionKey partition key
      *
      * @throws IOException exception is thrown every time because this is not supported
      */
diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java
index 46e51fcbaa..ebeb503bd3 100644
--- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java
+++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/JiraService.java
@@ -74,6 +74,7 @@ public JiraService(JiraSourceConfig jiraSourceConfig, JiraRestClient jiraRestCli
      *
      * @param configuration the configuration.
      * @param timestamp     timestamp.
+     * @param itemInfoQueue item info queue
      */
     public void getJiraEntities(JiraSourceConfig configuration, Instant timestamp, Queue itemInfoQueue) {
         log.trace("Started to fetch entities");
@@ -90,6 +91,7 @@ public String getIssue(String issueKey) {
      *
      * @param configuration Input Parameter
      * @param timestamp     Input Parameter
+     * @param itemInfoQueue item info queue
      */
     private void searchForNewTicketsAndAddToQueue(JiraSourceConfig configuration, Instant timestamp,
                                                   Queue itemInfoQueue) {
@@ -126,7 +128,7 @@ private void addItemsToQueue(List issueList, Queue itemInfo
      *
      * @param configuration Input Parameter
      * @param ts            Input Parameter
-     * @return String Builder
+     * @return String Builder created issue filter criteria
      */
     private StringBuilder createIssueFilterCriteria(JiraSourceConfig configuration, Instant ts) {
 
@@ -213,4 +215,4 @@ private void validateProjectFilters(JiraSourceConfig configuration) {
 
     }
 
-}
\ No newline at end of file
+}
diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java
index e82acb2a07..57a6c80aff 100644
--- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java
+++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/AddressValidation.java
@@ -33,6 +33,7 @@ public class AddressValidation {
      * Method for getInetAddress.
      *
      * @param url input parameter.
+     * @return returns inet address
      */
     public static InetAddress getInetAddress(String url) {
         try {
@@ -57,4 +58,4 @@ public static void validateInetAddress(@NonNull final InetAddress address) {
             throw new BadRequestException(INVALID_URL);
         }
     }
-}
\ No newline at end of file
+}
diff --git a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraConfigHelper.java b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraConfigHelper.java
index dd815bd607..ab851d0793 100644
--- a/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraConfigHelper.java
+++ b/data-prepper-plugins/saas-source-plugins/jira-source/src/main/java/org/opensearch/dataprepper/plugins/source/jira/utils/JiraConfigHelper.java
@@ -47,6 +47,7 @@ public static List getIssueStatusExcludeFilter(JiraSourceConfig reposito
     /**
      * Get Issue Types Filter from repository configuration.
      *
+     * @param repositoryConfiguration repo config
      * @return List Issue Type Filter.
      */
     public static List getIssueTypeIncludeFilter(JiraSourceConfig repositoryConfiguration) {
@@ -67,6 +68,7 @@ public static List getIssueTypeExcludeFilter(JiraSourceConfig repository
      * Get Project Filter Types from repository configuration.
      * public static final String ST = "status";
      *
+     * @param repositoryConfiguration repo config
      * @return List Project Filter.
      */
     public static List getProjectNameIncludeFilter(JiraSourceConfig repositoryConfiguration) {
diff --git a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/OTelHelper.java b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/OTelHelper.java
index 93859adccb..8ffda8a6c7 100644
--- a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/OTelHelper.java
+++ b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/OTelHelper.java
@@ -18,8 +18,8 @@ public class OTelHelper {
     /**
      * Helper method which accepts a OpenTelemetry trace resource object and return the service name object.
      *
-     * @param resource
-     * @return
+     * @param resource resource
+     * @return returns service name
      */
     public static Optional getServiceName(final Resource resource) {
         return resource.getAttributesList().stream().filter(
@@ -31,8 +31,8 @@ public static Optional getServiceName(final Resource resource) {
     /**
      * Helper method which checks if OpenTelemetry span is valid for processing.
      *
-     * @param span
-     * @return
+     * @param span span
+     * @return returns if the span is valid or not
      */
     public static boolean checkValidSpan(final Span span) {
         return !span.getTraceId().isEmpty() && !span.getSpanId().isEmpty() && !span.getName().isEmpty();
diff --git a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapRelationship.java b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapRelationship.java
index 2d20dd95c8..1144b86c9d 100644
--- a/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapRelationship.java
+++ b/data-prepper-plugins/service-map-stateful/src/main/java/org/opensearch/dataprepper/plugins/processor/ServiceMapRelationship.java
@@ -66,6 +66,11 @@ private ServiceMapRelationship(final String serviceName, final String kind, fina
 
     /**
      * Create a destination relationship
+     * @param serviceName service name
+     * @param spanKind span kind
+     * @param domain domain
+     * @param resource resource
+     * @param traceGroupName trace group name
      * @return Relationship with the fields set, and target set to null
      */
     public static ServiceMapRelationship newDestinationRelationship (
@@ -79,6 +84,11 @@ public static ServiceMapRelationship newDestinationRelationship (
 
     /**
      * Create a target relationship
+     * @param serviceName service name
+     * @param spanKind span kind
+     * @param domain domain
+     * @param resource resource
+     * @param traceGroupName trace group name
      * @return Relationship with the fields set, and destination set to null
      */
     public static ServiceMapRelationship newTargetRelationship (
@@ -164,12 +174,10 @@ private String md5Hash() {
 
     /**
      * The endpoint follows the URL spec.
-     * 

- * Example, https://paymentservice/makePayment. - *

+ *

Example, https://paymentservice/makePayment.

* domain: paymentservice * resource: makePayment - *

+ * * */ public static class Endpoint { diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/MessageFieldStrategy.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/MessageFieldStrategy.java index f7decd3f91..91ba2be3f1 100644 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/MessageFieldStrategy.java +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/MessageFieldStrategy.java @@ -16,6 +16,8 @@ public interface MessageFieldStrategy { /** * Converts the SQS message body into one or more events. + * @param messageBody message body + * @return returns list of events fromt he body */ List parseEvents(String messageBody); } diff --git a/data-prepper-test-common/src/main/java/org/opensearch/dataprepper/test/helper/ReflectivelySetField.java b/data-prepper-test-common/src/main/java/org/opensearch/dataprepper/test/helper/ReflectivelySetField.java index 799274ca7a..6b5f0e79b6 100644 --- a/data-prepper-test-common/src/main/java/org/opensearch/dataprepper/test/helper/ReflectivelySetField.java +++ b/data-prepper-test-common/src/main/java/org/opensearch/dataprepper/test/helper/ReflectivelySetField.java @@ -17,6 +17,7 @@ private ReflectivelySetField() { /** * Reflectively set the field of a configuration object. + * @param The type * @param configurationClass The class of the configuration object. * @param configurationObject The configuration object itself (for tests you might need to specify * "NameOfTestClass.this.configurationObject" From 7a3cf8754aa01bca7f008eed39748ea2f872e7bc Mon Sep 17 00:00:00 2001 From: Jeremy Michael <60355474+jmsusanto@users.noreply.github.com> Date: Fri, 31 Jan 2025 16:20:04 -0800 Subject: [PATCH 08/18] Implementation of sqs-common plugin and refactored sqs and s3 source (#5361) * initial refactoring Signed-off-by: Jeremy Michael * refactored sqs-source to use sqs-common Signed-off-by: Jeremy Michael * refactored SqsWorker to use the common library Signed-off-by: Jeremy Michael * minor changes Signed-off-by: Jeremy Michael * another small fix Signed-off-by: Jeremy Michael * added unit tests for sqs-common Signed-off-by: Jeremy Michael * updated tests Signed-off-by: Jeremy Michael --------- Signed-off-by: Jeremy Michael Co-authored-by: Jeremy Michael --- data-prepper-plugins/s3-source/build.gradle | 2 + .../plugins/source/s3/SqsServiceIT.java | 4 +- .../plugins/source/s3/SqsWorkerIT.java | 4 +- .../plugins/source/s3/SqsService.java | 27 +- data-prepper-plugins/sqs-common/build.gradle | 28 ++ .../plugins/source/sqs/common/SqsBackoff.java | 27 ++ .../source/sqs/common/SqsClientFactory.java | 39 +++ .../common}/SqsRetriesExhaustedException.java | 2 +- .../source/sqs/common/SqsWorkerCommon.java | 212 ++++++++++++++ .../source/sqs/common/SqsBackoffTest.java | 35 +++ .../sqs/common/SqsClientFactoryTest.java | 30 ++ .../sqs/common/SqsWorkerCommonTest.java | 106 +++++++ data-prepper-plugins/sqs-source/build.gradle | 1 + .../plugins/source/sqs/SqsService.java | 31 +- .../plugins/source/sqs/SqsWorker.java | 271 ++++-------------- .../plugins/source/sqs/SqsServiceTest.java | 17 +- .../plugins/source/sqs/SqsWorkerTest.java | 23 +- settings.gradle | 1 + 18 files changed, 581 insertions(+), 279 deletions(-) create mode 100644 data-prepper-plugins/sqs-common/build.gradle create mode 100644 data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsBackoff.java create mode 100644 data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsClientFactory.java rename data-prepper-plugins/{sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs => sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common}/SqsRetriesExhaustedException.java (89%) create mode 100644 data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommon.java create mode 100644 data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsBackoffTest.java create mode 100644 data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsClientFactoryTest.java create mode 100644 data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommonTest.java diff --git a/data-prepper-plugins/s3-source/build.gradle b/data-prepper-plugins/s3-source/build.gradle index f4afbfbfe3..30bced953d 100644 --- a/data-prepper-plugins/s3-source/build.gradle +++ b/data-prepper-plugins/s3-source/build.gradle @@ -11,6 +11,8 @@ dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:buffer-common') implementation project(':data-prepper-plugins:common') + implementation project(':data-prepper-plugins:sqs-common') + implementation libs.armeria.core implementation 'io.micrometer:micrometer-core' implementation 'software.amazon.awssdk:s3' diff --git a/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/s3/SqsServiceIT.java b/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/s3/SqsServiceIT.java index 25cdb2be67..3d1e6343f0 100644 --- a/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/s3/SqsServiceIT.java +++ b/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/s3/SqsServiceIT.java @@ -22,6 +22,7 @@ import org.opensearch.dataprepper.plugins.source.s3.configuration.OnErrorOption; import org.opensearch.dataprepper.plugins.source.s3.configuration.AwsAuthenticationOptions; import org.opensearch.dataprepper.plugins.source.s3.configuration.SqsOptions; +import org.opensearch.dataprepper.plugins.source.sqs.common.SqsBackoff; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.regions.Region; @@ -154,8 +155,7 @@ public void test_sqsService(int numWorkers) throws IOException { } private void clearSqsQueue() { - Backoff backoff = Backoff.exponential(SqsService.INITIAL_DELAY, SqsService.MAXIMUM_DELAY).withJitter(SqsService.JITTER_RATE) - .withMaxAttempts(Integer.MAX_VALUE); + Backoff backoff = SqsBackoff.createExponentialBackoff(); final SqsWorker sqsWorker = new SqsWorker(acknowledgementSetManager, sqsClient, s3Service, s3SourceConfig, pluginMetrics, backoff); //final SqsService objectUnderTest = createObjectUnderTest(); int sqsMessagesProcessed; diff --git a/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerIT.java b/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerIT.java index 21475930ec..a74e1b35bb 100644 --- a/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerIT.java +++ b/data-prepper-plugins/s3-source/src/integrationTest/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerIT.java @@ -27,6 +27,7 @@ import org.opensearch.dataprepper.plugins.source.s3.configuration.NotificationSourceOption; import org.opensearch.dataprepper.plugins.source.s3.configuration.OnErrorOption; import org.opensearch.dataprepper.plugins.source.s3.configuration.SqsOptions; +import org.opensearch.dataprepper.plugins.source.sqs.common.SqsBackoff; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.sqs.SqsClient; @@ -93,8 +94,7 @@ void setUp() { .region(Region.of(System.getProperty("tests.s3source.region"))) .build(); - backoff = Backoff.exponential(SqsService.INITIAL_DELAY, SqsService.MAXIMUM_DELAY).withJitter(SqsService.JITTER_RATE) - .withMaxAttempts(Integer.MAX_VALUE); + backoff = SqsBackoff.createExponentialBackoff(); s3SourceConfig = mock(S3SourceConfig.class); s3Service = mock(S3Service.class); diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsService.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsService.java index c674be5f68..de35592b90 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsService.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsService.java @@ -12,11 +12,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; -import software.amazon.awssdk.core.retry.RetryPolicy; import software.amazon.awssdk.services.sqs.SqsClient; +import org.opensearch.dataprepper.plugins.source.sqs.common.SqsBackoff; +import org.opensearch.dataprepper.plugins.source.sqs.common.SqsClientFactory; -import java.time.Duration; import java.util.List; import java.util.concurrent.TimeUnit; import java.util.concurrent.Executors; @@ -27,10 +26,6 @@ public class SqsService { private static final Logger LOG = LoggerFactory.getLogger(SqsService.class); static final long SHUTDOWN_TIMEOUT = 30L; - static final long INITIAL_DELAY = Duration.ofSeconds(20).toMillis(); - static final long MAXIMUM_DELAY = Duration.ofMinutes(5).toMillis(); - static final double JITTER_RATE = 0.20; - private final S3SourceConfig s3SourceConfig; private final S3Service s3Accessor; private final SqsClient sqsClient; @@ -38,6 +33,7 @@ public class SqsService { private final AcknowledgementSetManager acknowledgementSetManager; private final ExecutorService executorService; private final List sqsWorkers; + private final Backoff backoff; public SqsService(final AcknowledgementSetManager acknowledgementSetManager, final S3SourceConfig s3SourceConfig, @@ -48,11 +44,9 @@ public SqsService(final AcknowledgementSetManager acknowledgementSetManager, this.s3Accessor = s3Accessor; this.pluginMetrics = pluginMetrics; this.acknowledgementSetManager = acknowledgementSetManager; - this.sqsClient = createSqsClient(credentialsProvider); + this.sqsClient = SqsClientFactory.createSqsClient(s3SourceConfig.getAwsAuthenticationOptions().getAwsRegion(), credentialsProvider); executorService = Executors.newFixedThreadPool(s3SourceConfig.getNumWorkers(), BackgroundThreadFactory.defaultExecutorThreadFactory("s3-source-sqs")); - - final Backoff backoff = Backoff.exponential(INITIAL_DELAY, MAXIMUM_DELAY).withJitter(JITTER_RATE) - .withMaxAttempts(Integer.MAX_VALUE); + backoff = SqsBackoff.createExponentialBackoff(); sqsWorkers = IntStream.range(0, s3SourceConfig.getNumWorkers()) .mapToObj(i -> new SqsWorker(acknowledgementSetManager, sqsClient, s3Accessor, s3SourceConfig, pluginMetrics, backoff)) .collect(Collectors.toList()); @@ -62,17 +56,6 @@ public void start() { sqsWorkers.forEach(executorService::submit); } - SqsClient createSqsClient(final AwsCredentialsProvider credentialsProvider) { - LOG.debug("Creating SQS client"); - return SqsClient.builder() - .region(s3SourceConfig.getAwsAuthenticationOptions().getAwsRegion()) - .credentialsProvider(credentialsProvider) - .overrideConfiguration(ClientOverrideConfiguration.builder() - .retryPolicy(RetryPolicy.builder().numRetries(5).build()) - .build()) - .build(); - } - public void stop() { executorService.shutdown(); sqsWorkers.forEach(SqsWorker::stop); diff --git a/data-prepper-plugins/sqs-common/build.gradle b/data-prepper-plugins/sqs-common/build.gradle new file mode 100644 index 0000000000..b4ffbc8e5e --- /dev/null +++ b/data-prepper-plugins/sqs-common/build.gradle @@ -0,0 +1,28 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +plugins { + id 'java' +} + +dependencies { + implementation project(':data-prepper-api') + implementation project(':data-prepper-plugins:buffer-common') + implementation project(':data-prepper-plugins:common') + implementation libs.armeria.core + implementation project(':data-prepper-plugins:aws-plugin-api') + implementation 'software.amazon.awssdk:sqs' + implementation 'software.amazon.awssdk:arns' + implementation 'software.amazon.awssdk:sts' + implementation 'io.micrometer:micrometer-core' + implementation 'com.fasterxml.jackson.core:jackson-annotations' + implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' + implementation 'org.hibernate.validator:hibernate-validator:8.0.1.Final' + testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + testImplementation project(':data-prepper-plugins:blocking-buffer') +} +test { + useJUnitPlatform() +} diff --git a/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsBackoff.java b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsBackoff.java new file mode 100644 index 0000000000..b5f85cd61d --- /dev/null +++ b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsBackoff.java @@ -0,0 +1,27 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.dataprepper.plugins.source.sqs.common; + +import com.linecorp.armeria.client.retry.Backoff; +import java.time.Duration; + +public final class SqsBackoff { + private static final long INITIAL_DELAY_MILLIS = Duration.ofSeconds(20).toMillis(); + private static final long MAX_DELAY_MILLIS = Duration.ofMinutes(5).toMillis(); + private static final double JITTER_RATE = 0.20; + + private SqsBackoff() {} + + public static Backoff createExponentialBackoff() { + return Backoff.exponential(INITIAL_DELAY_MILLIS, MAX_DELAY_MILLIS) + .withJitter(JITTER_RATE) + .withMaxAttempts(Integer.MAX_VALUE); + } +} diff --git a/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsClientFactory.java b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsClientFactory.java new file mode 100644 index 0000000000..8754d87749 --- /dev/null +++ b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsClientFactory.java @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.sqs.common; + +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sqs.SqsClient; + +/** + * A common factory to create SQS clients + */ +public final class SqsClientFactory { + + private SqsClientFactory() { + } + + public static SqsClient createSqsClient( + final Region region, + final AwsCredentialsProvider credentialsProvider) { + + return SqsClient.builder() + .region(region) + .credentialsProvider(credentialsProvider) + .overrideConfiguration(ClientOverrideConfiguration.builder() + .retryPolicy(RetryPolicy.builder().numRetries(5).build()) + .build()) + .build(); + } +} diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsRetriesExhaustedException.java b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsRetriesExhaustedException.java similarity index 89% rename from data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsRetriesExhaustedException.java rename to data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsRetriesExhaustedException.java index e1fd536cb7..6dd42ce95b 100644 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsRetriesExhaustedException.java +++ b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsRetriesExhaustedException.java @@ -8,7 +8,7 @@ * */ -package org.opensearch.dataprepper.plugins.source.sqs; +package org.opensearch.dataprepper.plugins.source.sqs.common; /** * This exception is thrown when SQS retries are exhausted diff --git a/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommon.java b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommon.java new file mode 100644 index 0000000000..9301574237 --- /dev/null +++ b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommon.java @@ -0,0 +1,212 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.sqs.common; + +import com.linecorp.armeria.client.retry.Backoff; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.model.ChangeMessageVisibilityRequest; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; +import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; +import software.amazon.awssdk.services.sqs.model.SqsException; +import software.amazon.awssdk.services.sts.model.StsException; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; + +public class SqsWorkerCommon { + private static final Logger LOG = LoggerFactory.getLogger(SqsWorkerCommon.class); + public static final String ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME = "acknowledgementSetCallbackCounter"; + public static final String SQS_MESSAGES_RECEIVED_METRIC_NAME = "sqsMessagesReceived"; + public static final String SQS_MESSAGES_DELETED_METRIC_NAME = "sqsMessagesDeleted"; + public static final String SQS_MESSAGES_FAILED_METRIC_NAME = "sqsMessagesFailed"; + public static final String SQS_MESSAGES_DELETE_FAILED_METRIC_NAME = "sqsMessagesDeleteFailed"; + public static final String SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME = "sqsVisibilityTimeoutChangedCount"; + public static final String SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME = "sqsVisibilityTimeoutChangeFailedCount"; + + private final SqsClient sqsClient; + private final Backoff standardBackoff; + private final PluginMetrics pluginMetrics; + private final AcknowledgementSetManager acknowledgementSetManager; + private volatile boolean isStopped; + private int failedAttemptCount; + private final Counter sqsMessagesReceivedCounter; + private final Counter sqsMessagesDeletedCounter; + private final Counter sqsMessagesFailedCounter; + private final Counter sqsMessagesDeleteFailedCounter; + private final Counter acknowledgementSetCallbackCounter; + private final Counter sqsVisibilityTimeoutChangedCount; + private final Counter sqsVisibilityTimeoutChangeFailedCount; + + public SqsWorkerCommon(final SqsClient sqsClient, + final Backoff standardBackoff, + final PluginMetrics pluginMetrics, + final AcknowledgementSetManager acknowledgementSetManager) { + + this.sqsClient = sqsClient; + this.standardBackoff = standardBackoff; + this.pluginMetrics = pluginMetrics; + this.acknowledgementSetManager = acknowledgementSetManager; + this.isStopped = false; + this.failedAttemptCount = 0; + + sqsMessagesReceivedCounter = pluginMetrics.counter(SQS_MESSAGES_RECEIVED_METRIC_NAME); + sqsMessagesDeletedCounter = pluginMetrics.counter(SQS_MESSAGES_DELETED_METRIC_NAME); + sqsMessagesFailedCounter = pluginMetrics.counter(SQS_MESSAGES_FAILED_METRIC_NAME); + sqsMessagesDeleteFailedCounter = pluginMetrics.counter(SQS_MESSAGES_DELETE_FAILED_METRIC_NAME); + acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME); + sqsVisibilityTimeoutChangedCount = pluginMetrics.counter(SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME); + sqsVisibilityTimeoutChangeFailedCount = pluginMetrics.counter(SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME); + } + + public List pollSqsMessages(final String queueUrl, + final Integer maxNumberOfMessages, + final Duration waitTime, + final Duration visibilityTimeout) { + try { + final ReceiveMessageRequest request = createReceiveMessageRequest(queueUrl, maxNumberOfMessages, waitTime, visibilityTimeout); + final List messages = sqsClient.receiveMessage(request).messages(); + failedAttemptCount = 0; + if (!messages.isEmpty()) { + sqsMessagesReceivedCounter.increment(messages.size()); + } + return messages; + } + catch (SqsException | StsException e) { + LOG.error("Error reading from SQS: {}. Retrying with exponential backoff.", e.getMessage()); + applyBackoff(); + return Collections.emptyList(); + } + } + + private ReceiveMessageRequest createReceiveMessageRequest(String queueUrl, Integer maxNumberOfMessages, Duration waitTime, Duration visibilityTimeout) { + ReceiveMessageRequest.Builder requestBuilder = ReceiveMessageRequest.builder() + .queueUrl(queueUrl) + .attributeNamesWithStrings("All") + .messageAttributeNames("All"); + + if (waitTime != null) { + requestBuilder.waitTimeSeconds((int) waitTime.getSeconds()); + } + if (maxNumberOfMessages != null) { + requestBuilder.maxNumberOfMessages(maxNumberOfMessages); + } + if (visibilityTimeout != null) { + requestBuilder.visibilityTimeout((int) visibilityTimeout.getSeconds()); + } + return requestBuilder.build(); + } + + public void applyBackoff() { + final long delayMillis = standardBackoff.nextDelayMillis(++failedAttemptCount); + if (delayMillis < 0) { + Thread.currentThread().interrupt(); + throw new SqsRetriesExhaustedException("SQS retries exhausted. Check your SQS configuration."); + } + + final Duration delayDuration = Duration.ofMillis(delayMillis); + LOG.info("Pausing SQS processing for {}.{} seconds due to an error.", + delayDuration.getSeconds(), delayDuration.toMillisPart()); + + try { + Thread.sleep(delayMillis); + } catch (InterruptedException e) { + LOG.error("Thread interrupted during SQS backoff sleep.", e); + Thread.currentThread().interrupt(); + } + } + + public void deleteSqsMessages(final String queueUrl, final List entries) { + if (entries == null || entries.isEmpty() || isStopped) { + return; + } + + try { + final DeleteMessageBatchRequest request = DeleteMessageBatchRequest.builder() + .queueUrl(queueUrl) + .entries(entries) + .build(); + + final DeleteMessageBatchResponse response = sqsClient.deleteMessageBatch(request); + + if (response.hasSuccessful()) { + final int successCount = response.successful().size(); + sqsMessagesDeletedCounter.increment(successCount); + LOG.debug("Deleted {} messages from SQS queue [{}]", successCount, queueUrl); + } + if (response.hasFailed()) { + final int failCount = response.failed().size(); + sqsMessagesDeleteFailedCounter.increment(failCount); + LOG.error("Failed to delete {} messages from SQS queue [{}].", failCount, queueUrl); + } + } catch (SdkException e) { + sqsMessagesDeleteFailedCounter.increment(entries.size()); + LOG.error("Failed to delete messages from SQS queue [{}]: {}", queueUrl, e.getMessage()); + } + } + + public void increaseVisibilityTimeout(final String queueUrl, + final String receiptHandle, + final int newVisibilityTimeoutSeconds, + final String messageIdForLogging) { + if (isStopped) { + LOG.info("Skipping visibility timeout extension because worker is stopping. ID: {}", messageIdForLogging); + return; + } + + try { + ChangeMessageVisibilityRequest request = ChangeMessageVisibilityRequest.builder() + .queueUrl(queueUrl) + .receiptHandle(receiptHandle) + .visibilityTimeout(newVisibilityTimeoutSeconds) + .build(); + + sqsClient.changeMessageVisibility(request); + sqsVisibilityTimeoutChangedCount.increment(); + LOG.debug("Set visibility timeout for message {} to {} seconds", messageIdForLogging, newVisibilityTimeoutSeconds); + } + catch (Exception e) { + sqsVisibilityTimeoutChangeFailedCount.increment(); + LOG.error("Failed to set visibility timeout for message {} to {}. Reason: {}", + messageIdForLogging, newVisibilityTimeoutSeconds, e.getMessage()); + } + } + + public DeleteMessageBatchRequestEntry buildDeleteMessageBatchRequestEntry(final String messageId, + final String receiptHandle) { + return DeleteMessageBatchRequestEntry.builder() + .id(messageId) + .receiptHandle(receiptHandle) + .build(); + } + + public Timer createTimer(final String timerName) { + return pluginMetrics.timer(timerName); + } + + public Counter getSqsMessagesFailedCounter() { + return sqsMessagesFailedCounter; + } + + public void stop() { + isStopped = true; + } +} diff --git a/data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsBackoffTest.java b/data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsBackoffTest.java new file mode 100644 index 0000000000..b793cf39bc --- /dev/null +++ b/data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsBackoffTest.java @@ -0,0 +1,35 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.dataprepper.plugins.source.sqs.common; + +import com.linecorp.armeria.client.retry.Backoff; +import org.junit.jupiter.api.Test; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class SqsBackoffTest { + + @Test + void testCreateExponentialBackoff() { + final Backoff backoff = SqsBackoff.createExponentialBackoff(); + assertNotNull(backoff, "Backoff should not be null"); + final long firstDelay = backoff.nextDelayMillis(1); + final long expectedBaseDelay = 20_000L; + final double jitterRate = 0.20; + final long minDelay = (long) (expectedBaseDelay * (1 - jitterRate)); + final long maxDelay = (long) (expectedBaseDelay * (1 + jitterRate)); + + assertTrue( + firstDelay >= minDelay && firstDelay <= maxDelay, + String.format("First delay %dms should be between %dms and %dms", + firstDelay, minDelay, maxDelay) + ); + } +} diff --git a/data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsClientFactoryTest.java b/data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsClientFactoryTest.java new file mode 100644 index 0000000000..5f2f64e48e --- /dev/null +++ b/data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsClientFactoryTest.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + */ + +package org.opensearch.dataprepper.plugins.source.sqs.common; + +import org.junit.jupiter.api.Test; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sqs.SqsClient; + +import static org.junit.jupiter.api.Assertions.assertNotNull; + +class SqsClientFactoryTest { + + @Test + void testCreateSqsClientReturnsNonNull() { + final StaticCredentialsProvider credentialsProvider = + StaticCredentialsProvider.create(AwsBasicCredentials.create("testKey", "testSecret")); + + final SqsClient sqsClient = SqsClientFactory.createSqsClient(Region.US_EAST_1, credentialsProvider); + assertNotNull(sqsClient, "SqsClient should not be null"); + } +} diff --git a/data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommonTest.java b/data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommonTest.java new file mode 100644 index 0000000000..32cbf3fd69 --- /dev/null +++ b/data-prepper-plugins/sqs-common/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommonTest.java @@ -0,0 +1,106 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.sqs.common; + +import com.linecorp.armeria.client.retry.Backoff; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mockito; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import software.amazon.awssdk.services.sqs.SqsClient; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; +import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse; + +import java.time.Duration; +import java.util.Collections; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.when; + +class SqsWorkerCommonTest { + private SqsClient sqsClient; + private Backoff backoff; + private PluginMetrics pluginMetrics; + private AcknowledgementSetManager acknowledgementSetManager; + private SqsWorkerCommon sqsWorkerCommon; + + @BeforeEach + void setUp() { + sqsClient = Mockito.mock(SqsClient.class); + backoff = Mockito.mock(Backoff.class); + pluginMetrics = Mockito.mock(PluginMetrics.class); + acknowledgementSetManager = Mockito.mock(AcknowledgementSetManager.class); + when(pluginMetrics.counter(Mockito.anyString())).thenReturn(Mockito.mock(Counter.class)); + when(pluginMetrics.timer(Mockito.anyString())).thenReturn(Mockito.mock(Timer.class)); + sqsWorkerCommon = new SqsWorkerCommon(sqsClient, backoff, pluginMetrics, acknowledgementSetManager); + } + + @Test + void testPollSqsMessages_handlesEmptyList() { + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))) + .thenReturn(ReceiveMessageResponse.builder() + .messages(Collections.emptyList()) + .build()); + var messages = sqsWorkerCommon.pollSqsMessages( + "testQueueUrl", + 10, + Duration.ofSeconds(5), + Duration.ofSeconds(30) + ); + + assertNotNull(messages); + assertTrue(messages.isEmpty()); + Mockito.verify(sqsClient).receiveMessage(any(ReceiveMessageRequest.class)); + Mockito.verify(backoff, Mockito.never()).nextDelayMillis(Mockito.anyInt()); + } + + @Test + void testDeleteSqsMessages_callsClientWhenNotStopped() { + var entries = Collections.singletonList( + DeleteMessageBatchRequestEntry.builder() + .id("msg-id") + .receiptHandle("receipt-handle") + .build() + ); + + when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))) + .thenReturn(DeleteMessageBatchResponse.builder().build()); + + sqsWorkerCommon.deleteSqsMessages("testQueueUrl", entries); + ArgumentCaptor captor = + ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); + Mockito.verify(sqsClient).deleteMessageBatch(captor.capture()); + assertEquals("testQueueUrl", captor.getValue().queueUrl()); + assertEquals(1, captor.getValue().entries().size()); + } + + @Test + void testStop_skipsFurtherOperations() { + sqsWorkerCommon.stop(); + sqsWorkerCommon.deleteSqsMessages("testQueueUrl", Collections.singletonList( + DeleteMessageBatchRequestEntry.builder() + .id("msg-id") + .receiptHandle("receipt-handle") + .build() + )); + Mockito.verify(sqsClient, Mockito.never()).deleteMessageBatch((DeleteMessageBatchRequest) any()); + } +} diff --git a/data-prepper-plugins/sqs-source/build.gradle b/data-prepper-plugins/sqs-source/build.gradle index b4ffbc8e5e..7a8ce38f29 100644 --- a/data-prepper-plugins/sqs-source/build.gradle +++ b/data-prepper-plugins/sqs-source/build.gradle @@ -11,6 +11,7 @@ dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:buffer-common') implementation project(':data-prepper-plugins:common') + implementation project(':data-prepper-plugins:sqs-common') implementation libs.armeria.core implementation project(':data-prepper-plugins:aws-plugin-api') implementation 'software.amazon.awssdk:sqs' diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java index 672ee9874c..5c3e3ba2d2 100644 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java @@ -18,16 +18,16 @@ import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.plugin.PluginFactory; + import org.opensearch.dataprepper.plugins.source.sqs.common.SqsBackoff; + import org.opensearch.dataprepper.plugins.source.sqs.common.SqsClientFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; - import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; - import software.amazon.awssdk.core.retry.RetryPolicy; import software.amazon.awssdk.services.sqs.SqsClient; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; - import java.time.Duration; + import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; @@ -39,10 +39,6 @@ public class SqsService { private static final Logger LOG = LoggerFactory.getLogger(SqsService.class); static final long SHUTDOWN_TIMEOUT = 30L; - static final long INITIAL_DELAY = Duration.ofSeconds(20).toMillis(); - static final long MAXIMUM_DELAY = Duration.ofMinutes(5).toMillis(); - static final double JITTER_RATE = 0.20; - private final SqsSourceConfig sqsSourceConfig; private final SqsClient sqsClient; private final PluginMetrics pluginMetrics; @@ -51,6 +47,7 @@ public class SqsService { private final List allSqsUrlExecutorServices; private final List sqsWorkers; private final Buffer> buffer; + private final Backoff backoff; public SqsService(final Buffer> buffer, final AcknowledgementSetManager acknowledgementSetManager, @@ -65,17 +62,13 @@ public SqsService(final Buffer> buffer, this.acknowledgementSetManager = acknowledgementSetManager; this.allSqsUrlExecutorServices = new ArrayList<>(); this.sqsWorkers = new ArrayList<>(); - this.sqsClient = createSqsClient(credentialsProvider); + this.sqsClient = SqsClientFactory.createSqsClient(sqsSourceConfig.getAwsAuthenticationOptions().getAwsRegion(), credentialsProvider); this.buffer = buffer; + backoff = SqsBackoff.createExponentialBackoff(); } - public void start() { - final Backoff backoff = Backoff.exponential(INITIAL_DELAY, MAXIMUM_DELAY).withJitter(JITTER_RATE) - .withMaxAttempts(Integer.MAX_VALUE); - LOG.info("Starting SqsService"); - sqsSourceConfig.getQueues().forEach(queueConfig -> { String queueUrl = queueConfig.getUrl(); String queueName = queueUrl.substring(queueUrl.lastIndexOf('/') + 1); @@ -112,17 +105,7 @@ public void start() { LOG.info("Started SQS workers for queue {} with {} workers", queueUrl, numWorkers); }); } - - SqsClient createSqsClient(final AwsCredentialsProvider credentialsProvider) { - LOG.debug("Creating SQS client"); - return SqsClient.builder() - .region(sqsSourceConfig.getAwsAuthenticationOptions().getAwsRegion()) - .credentialsProvider(credentialsProvider) - .overrideConfiguration(ClientOverrideConfiguration.builder() - .retryPolicy(RetryPolicy.builder().numRetries(5).build()) - .build()) - .build(); - } + public void stop() { allSqsUrlExecutorServices.forEach(ExecutorService::shutdown); diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java index f6de0b9ee1..cb4c168345 100644 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java @@ -17,58 +17,35 @@ import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.plugins.source.sqs.common.SqsWorkerCommon; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.sqs.SqsClient; -import software.amazon.awssdk.services.sqs.model.ChangeMessageVisibilityRequest; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; import software.amazon.awssdk.services.sqs.model.Message; -import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; -import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse; -import software.amazon.awssdk.services.sqs.model.SqsException; -import software.amazon.awssdk.services.sts.model.StsException; -import org.opensearch.dataprepper.model.buffer.Buffer; import java.time.Duration; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; - public class SqsWorker implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(SqsWorker.class); - static final String SQS_MESSAGES_RECEIVED_METRIC_NAME = "sqsMessagesReceived"; - static final String SQS_MESSAGES_DELETED_METRIC_NAME = "sqsMessagesDeleted"; - static final String SQS_MESSAGES_FAILED_METRIC_NAME = "sqsMessagesFailed"; - static final String SQS_MESSAGES_DELETE_FAILED_METRIC_NAME = "sqsMessagesDeleteFailed"; - static final String SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME = "sqsVisibilityTimeoutChangedCount"; - static final String SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME = "sqsVisibilityTimeoutChangeFailedCount"; static final String ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME = "acknowledgementSetCallbackCounter"; - - private final SqsClient sqsClient; + private final SqsWorkerCommon sqsWorkerCommon; private final SqsEventProcessor sqsEventProcessor; - private final Counter sqsMessagesReceivedCounter; - private final Counter sqsMessagesDeletedCounter; - private final Counter sqsMessagesFailedCounter; - private final Counter sqsMessagesDeleteFailedCounter; - private final Counter acknowledgementSetCallbackCounter; - private final Counter sqsVisibilityTimeoutChangedCount; - private final Counter sqsVisibilityTimeoutChangeFailedCount; - private final Backoff standardBackoff; private final QueueConfig queueConfig; - private int failedAttemptCount; private final boolean endToEndAcknowledgementsEnabled; - private final AcknowledgementSetManager acknowledgementSetManager; - private volatile boolean isStopped = false; private final Buffer> buffer; private final int bufferTimeoutMillis; - private Map messageVisibilityTimesMap; + private final AcknowledgementSetManager acknowledgementSetManager; + private final Counter acknowledgementSetCallbackCounter; + private int failedAttemptCount; + private volatile boolean isStopped = false; + private final Map messageVisibilityTimesMap; public SqsWorker(final Buffer> buffer, final AcknowledgementSetManager acknowledgementSetManager, @@ -78,24 +55,16 @@ public SqsWorker(final Buffer> buffer, final PluginMetrics pluginMetrics, final SqsEventProcessor sqsEventProcessor, final Backoff backoff) { - - this.sqsClient = sqsClient; + this.sqsWorkerCommon = new SqsWorkerCommon(sqsClient, backoff, pluginMetrics, acknowledgementSetManager); this.queueConfig = queueConfig; this.acknowledgementSetManager = acknowledgementSetManager; - this.standardBackoff = backoff; - this.endToEndAcknowledgementsEnabled = sqsSourceConfig.getAcknowledgements(); + this.sqsEventProcessor = sqsEventProcessor; this.buffer = buffer; this.bufferTimeoutMillis = (int) sqsSourceConfig.getBufferTimeout().toMillis(); - this.sqsEventProcessor = sqsEventProcessor; - messageVisibilityTimesMap = new HashMap<>(); - failedAttemptCount = 0; - sqsMessagesReceivedCounter = pluginMetrics.counter(SQS_MESSAGES_RECEIVED_METRIC_NAME); - sqsMessagesDeletedCounter = pluginMetrics.counter(SQS_MESSAGES_DELETED_METRIC_NAME); - sqsMessagesFailedCounter = pluginMetrics.counter(SQS_MESSAGES_FAILED_METRIC_NAME); - sqsMessagesDeleteFailedCounter = pluginMetrics.counter(SQS_MESSAGES_DELETE_FAILED_METRIC_NAME); - acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME); - sqsVisibilityTimeoutChangedCount = pluginMetrics.counter(SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME); - sqsVisibilityTimeoutChangeFailedCount = pluginMetrics.counter(SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME); + this.endToEndAcknowledgementsEnabled = sqsSourceConfig.getAcknowledgements(); + this.messageVisibilityTimesMap = new HashMap<>(); + this.failedAttemptCount = 0; + this.acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME); } @Override @@ -104,10 +73,9 @@ public void run() { int messagesProcessed = 0; try { messagesProcessed = processSqsMessages(); - } catch (final Exception e) { LOG.error("Unable to process SQS messages. Processing error due to: {}", e.getMessage()); - applyBackoff(); + sqsWorkerCommon.applyBackoff(); } if (messagesProcessed > 0 && queueConfig.getPollDelay().toMillis() > 0) { @@ -115,216 +83,103 @@ public void run() { Thread.sleep(queueConfig.getPollDelay().toMillis()); } catch (final InterruptedException e) { LOG.error("Thread is interrupted while polling SQS.", e); + Thread.currentThread().interrupt(); } } } } int processSqsMessages() { - final List messages = getMessagesFromSqs(); + List messages = sqsWorkerCommon.pollSqsMessages(queueConfig.getUrl(), + queueConfig.getMaximumMessages(), + queueConfig.getWaitTime(), + queueConfig.getVisibilityTimeout()); if (!messages.isEmpty()) { - sqsMessagesReceivedCounter.increment(messages.size()); final List deleteMessageBatchRequestEntries = processSqsEvents(messages); if (!deleteMessageBatchRequestEntries.isEmpty()) { - deleteSqsMessages(deleteMessageBatchRequestEntries); + sqsWorkerCommon.deleteSqsMessages(queueConfig.getUrl(), deleteMessageBatchRequestEntries); } } return messages.size(); } - private List getMessagesFromSqs() { - try { - final ReceiveMessageRequest request = createReceiveMessageRequest(); - final ReceiveMessageResponse response = sqsClient.receiveMessage(request); - List messages = response.messages(); - failedAttemptCount = 0; - return messages; - - } catch (final SqsException | StsException e) { - LOG.error("Error reading from SQS: {}. Retrying with exponential backoff.", e.getMessage()); - applyBackoff(); - return Collections.emptyList(); - } - } - - private void applyBackoff() { - final long delayMillis = standardBackoff.nextDelayMillis(++failedAttemptCount); - if (delayMillis < 0) { - Thread.currentThread().interrupt(); - throw new SqsRetriesExhaustedException("SQS retries exhausted. Make sure that SQS configuration is valid, SQS queue exists, and IAM role has required permissions."); - } - final Duration delayDuration = Duration.ofMillis(delayMillis); - LOG.info("Pausing SQS processing for {}.{} seconds due to an error in processing.", - delayDuration.getSeconds(), delayDuration.toMillisPart()); - try { - Thread.sleep(delayMillis); - } catch (final InterruptedException e){ - LOG.error("Thread is interrupted while polling SQS with retry.", e); - } - } - - private ReceiveMessageRequest createReceiveMessageRequest() { - ReceiveMessageRequest.Builder requestBuilder = ReceiveMessageRequest.builder() - .queueUrl(queueConfig.getUrl()) - .attributeNamesWithStrings("All") - .messageAttributeNames("All"); - - if (queueConfig.getWaitTime() != null) { - requestBuilder.waitTimeSeconds((int) queueConfig.getWaitTime().getSeconds()); - } - if (queueConfig.getMaximumMessages() != null) { - requestBuilder.maxNumberOfMessages(queueConfig.getMaximumMessages()); - } - if (queueConfig.getVisibilityTimeout() != null) { - requestBuilder.visibilityTimeout((int) queueConfig.getVisibilityTimeout().getSeconds()); - } - return requestBuilder.build(); - } - private List processSqsEvents(final List messages) { final List deleteMessageBatchRequestEntryCollection = new ArrayList<>(); final Map messageAcknowledgementSetMap = new HashMap<>(); final Map> messageWaitingForAcknowledgementsMap = new HashMap<>(); - + for (Message message : messages) { List waitingForAcknowledgements = new ArrayList<>(); AcknowledgementSet acknowledgementSet = null; - final int visibilityTimeout; - if (queueConfig.getVisibilityTimeout() != null) { - visibilityTimeout = (int) queueConfig.getVisibilityTimeout().getSeconds(); - } else { - visibilityTimeout = (int) Duration.ofSeconds(30).getSeconds(); - - } + final int visibilityTimeout = queueConfig.getVisibilityTimeout() != null + ? (int) queueConfig.getVisibilityTimeout().getSeconds() + : 30; - final int maxVisibilityTimeout = (int)queueConfig.getVisibilityDuplicateProtectionTimeout().getSeconds(); - final int progressCheckInterval = visibilityTimeout/2 - 1; + final int maxVisibilityTimeout = (int) queueConfig.getVisibilityDuplicateProtectionTimeout().getSeconds(); + final int progressCheckInterval = visibilityTimeout / 2 - 1; if (endToEndAcknowledgementsEnabled) { - int expiryTimeout = visibilityTimeout - 2; - final boolean visibilityDuplicateProtectionEnabled = queueConfig.getVisibilityDuplicateProtection(); - if (visibilityDuplicateProtectionEnabled) { - expiryTimeout = maxVisibilityTimeout; - } - acknowledgementSet = acknowledgementSetManager.create( - (result) -> { - acknowledgementSetCallbackCounter.increment(); - // Delete only if this is positive acknowledgement - if (visibilityDuplicateProtectionEnabled) { - messageVisibilityTimesMap.remove(message); + int expiryTimeout = queueConfig.getVisibilityDuplicateProtection() + ? maxVisibilityTimeout + : visibilityTimeout - 2; + acknowledgementSet = acknowledgementSetManager.create(result -> { + acknowledgementSetCallbackCounter.increment(); + if (queueConfig.getVisibilityDuplicateProtection()) { + messageVisibilityTimesMap.remove(message); + } + if (result) { + sqsWorkerCommon.deleteSqsMessages(queueConfig.getUrl(), waitingForAcknowledgements); + } + }, Duration.ofSeconds(expiryTimeout)); + if (queueConfig.getVisibilityDuplicateProtection()) { + acknowledgementSet.addProgressCheck(ratio -> { + int newValue = messageVisibilityTimesMap.getOrDefault(message, visibilityTimeout) + progressCheckInterval; + if (newValue >= maxVisibilityTimeout) { + return; } - if (result) { - deleteSqsMessages(waitingForAcknowledgements); - } - }, - Duration.ofSeconds(expiryTimeout)); - if (visibilityDuplicateProtectionEnabled) { - acknowledgementSet.addProgressCheck( - (ratio) -> { - int newValue = messageVisibilityTimesMap.getOrDefault(message, visibilityTimeout) + progressCheckInterval; - if (newValue >= maxVisibilityTimeout) { - return; - } - messageVisibilityTimesMap.put(message, newValue); - final int newVisibilityTimeoutSeconds = visibilityTimeout; - increaseVisibilityTimeout(message, newVisibilityTimeoutSeconds); - }, - Duration.ofSeconds(progressCheckInterval)); + messageVisibilityTimesMap.put(message, newValue); + sqsWorkerCommon.increaseVisibilityTimeout(queueConfig.getUrl(), + message.receiptHandle(), + visibilityTimeout, + message.messageId()); + }, Duration.ofSeconds(progressCheckInterval)); } messageAcknowledgementSetMap.put(message, acknowledgementSet); messageWaitingForAcknowledgementsMap.put(message, waitingForAcknowledgements); } } - - if (endToEndAcknowledgementsEnabled) { - LOG.debug("Created acknowledgement sets for {} messages.", messages.size()); - } for (Message message : messages) { final AcknowledgementSet acknowledgementSet = messageAcknowledgementSetMap.get(message); final List waitingForAcknowledgements = messageWaitingForAcknowledgementsMap.get(message); - final Optional deleteMessageBatchRequestEntry = processSqsObject(message, acknowledgementSet); + final Optional deleteEntry = processSqsObject(message, acknowledgementSet); if (endToEndAcknowledgementsEnabled) { - deleteMessageBatchRequestEntry.ifPresent(waitingForAcknowledgements::add); - acknowledgementSet.complete(); + deleteEntry.ifPresent(waitingForAcknowledgements::add); + if (acknowledgementSet != null) { + acknowledgementSet.complete(); + } } else { - deleteMessageBatchRequestEntry.ifPresent(deleteMessageBatchRequestEntryCollection::add); + deleteEntry.ifPresent(deleteMessageBatchRequestEntryCollection::add); } } - return deleteMessageBatchRequestEntryCollection; } - - private Optional processSqsObject( - final Message message, - final AcknowledgementSet acknowledgementSet) { + private Optional processSqsObject(final Message message, + final AcknowledgementSet acknowledgementSet) { try { sqsEventProcessor.addSqsObject(message, queueConfig.getUrl(), buffer, bufferTimeoutMillis, acknowledgementSet); - return Optional.of(buildDeleteMessageBatchRequestEntry(message)); + return Optional.of(sqsWorkerCommon.buildDeleteMessageBatchRequestEntry(message.messageId(), message.receiptHandle())); } catch (final Exception e) { - sqsMessagesFailedCounter.increment(); + sqsWorkerCommon.getSqsMessagesFailedCounter().increment(); LOG.error("Error processing from SQS: {}. Retrying with exponential backoff.", e.getMessage()); - applyBackoff(); + sqsWorkerCommon.applyBackoff(); return Optional.empty(); } } - private void increaseVisibilityTimeout(final Message message, final int newVisibilityTimeoutSeconds) { - if(isStopped) { - LOG.info("Some messages are pending completion of acknowledgments. Data Prepper will not increase the visibility timeout because it is shutting down. {}", message); - return; - } - final ChangeMessageVisibilityRequest changeMessageVisibilityRequest = ChangeMessageVisibilityRequest.builder() - .visibilityTimeout(newVisibilityTimeoutSeconds) - .queueUrl(queueConfig.getUrl()) - .receiptHandle(message.receiptHandle()) - .build(); - - try { - sqsClient.changeMessageVisibility(changeMessageVisibilityRequest); - sqsVisibilityTimeoutChangedCount.increment(); - LOG.debug("Set visibility timeout for message {} to {}", message.messageId(), newVisibilityTimeoutSeconds); - } catch (Exception e) { - LOG.error("Failed to set visibility timeout for message {} to {}", message.messageId(), newVisibilityTimeoutSeconds, e); - sqsVisibilityTimeoutChangeFailedCount.increment(); - } - } - - - private DeleteMessageBatchRequestEntry buildDeleteMessageBatchRequestEntry(Message message) { - return DeleteMessageBatchRequestEntry.builder() - .id(message.messageId()) - .receiptHandle(message.receiptHandle()) - .build(); - } - - private void deleteSqsMessages(final List deleteEntries) { - if (deleteEntries.isEmpty()) return; - - try { - DeleteMessageBatchRequest deleteRequest = DeleteMessageBatchRequest.builder() - .queueUrl(queueConfig.getUrl()) - .entries(deleteEntries) - .build(); - DeleteMessageBatchResponse response = sqsClient.deleteMessageBatch(deleteRequest); - - if (response.hasSuccessful()) { - int successfulDeletes = response.successful().size(); - sqsMessagesDeletedCounter.increment(successfulDeletes); - } - if (response.hasFailed()) { - int failedDeletes = response.failed().size(); - sqsMessagesDeleteFailedCounter.increment(failedDeletes); - LOG.error("Failed to delete {} messages from SQS.", failedDeletes); - } - } catch (SdkException e) { - LOG.error("Failed to delete messages from SQS: {}", e.getMessage()); - sqsMessagesDeleteFailedCounter.increment(deleteEntries.size()); - } - } - void stop() { isStopped = true; + sqsWorkerCommon.stop(); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java index 695164db82..83a12e5940 100644 --- a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java @@ -22,11 +22,9 @@ import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.sqs.SqsClient; import java.util.List; -import static org.mockito.Mockito.doReturn; + import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; import static org.mockito.Mockito.withSettings; @@ -62,26 +60,19 @@ void start_with_single_queue_starts_workers() { when(queueConfig.getNumWorkers()).thenReturn(2); when(sqsSourceConfig.getQueues()).thenReturn(List.of(queueConfig)); SqsService sqsService = spy(new SqsService(buffer, acknowledgementSetManager, sqsSourceConfig, pluginMetrics, pluginFactory, credentialsProvider)); - doReturn(sqsClient).when(sqsService).createSqsClient(credentialsProvider); sqsService.start(); // if no exception is thrown here, then workers have been started } @Test - void stop_should_shutdown_executors_and_workers_and_close_client() throws InterruptedException { + void stop_should_shutdown_executors_and_workers() throws InterruptedException { QueueConfig queueConfig = mock(QueueConfig.class); when(queueConfig.getUrl()).thenReturn("MyQueue"); when(queueConfig.getNumWorkers()).thenReturn(1); when(sqsSourceConfig.getQueues()).thenReturn(List.of(queueConfig)); SqsClient sqsClient = mock(SqsClient.class); - SqsService sqsService = new SqsService(buffer, acknowledgementSetManager, sqsSourceConfig, pluginMetrics, pluginFactory, credentialsProvider) { - @Override - SqsClient createSqsClient(final AwsCredentialsProvider credentialsProvider) { - return sqsClient; - } - }; + SqsService sqsService = new SqsService(buffer, acknowledgementSetManager, sqsSourceConfig, pluginMetrics, pluginFactory, credentialsProvider) {}; sqsService.start(); - sqsService.stop(); - verify(sqsClient, times(1)).close(); + sqsService.stop(); // again assuming that if no exception is thrown here, then workers and client have been stopped } } \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java index 22bf48596f..e7339543c2 100644 --- a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java @@ -26,6 +26,8 @@ import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.sqs.common.SqsWorkerCommon; +import org.opensearch.dataprepper.plugins.source.sqs.common.SqsRetriesExhaustedException; import software.amazon.awssdk.services.sqs.SqsClient; import software.amazon.awssdk.services.sqs.model.ChangeMessageVisibilityRequest; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; @@ -111,13 +113,20 @@ private SqsWorker createObjectUnderTest() { @BeforeEach void setUp() { - when(pluginMetrics.counter(SqsWorker.SQS_MESSAGES_RECEIVED_METRIC_NAME)).thenReturn(sqsMessagesReceivedCounter); - when(pluginMetrics.counter(SqsWorker.SQS_MESSAGES_DELETED_METRIC_NAME)).thenReturn(sqsMessagesDeletedCounter); - when(pluginMetrics.counter(SqsWorker.SQS_MESSAGES_FAILED_METRIC_NAME)).thenReturn(sqsMessagesFailedCounter); - when(pluginMetrics.counter(SqsWorker.SQS_MESSAGES_DELETE_FAILED_METRIC_NAME)).thenReturn(sqsMessagesDeleteFailedCounter); - when(pluginMetrics.counter(SqsWorker.ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME)).thenReturn(acknowledgementSetCallbackCounter); - when(pluginMetrics.counter(SqsWorker.SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME)).thenReturn(sqsVisibilityTimeoutChangedCount); - when(pluginMetrics.counter(SqsWorker.SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME)).thenReturn(sqsVisibilityTimeoutChangeFailedCount); + when(pluginMetrics.counter(SqsWorkerCommon.SQS_MESSAGES_RECEIVED_METRIC_NAME)) + .thenReturn(sqsMessagesReceivedCounter); + when(pluginMetrics.counter(SqsWorkerCommon.SQS_MESSAGES_DELETED_METRIC_NAME)) + .thenReturn(sqsMessagesDeletedCounter); + when(pluginMetrics.counter(SqsWorkerCommon.SQS_MESSAGES_FAILED_METRIC_NAME)) + .thenReturn(sqsMessagesFailedCounter); + when(pluginMetrics.counter(SqsWorkerCommon.SQS_MESSAGES_DELETE_FAILED_METRIC_NAME)) + .thenReturn(sqsMessagesDeleteFailedCounter); + when(pluginMetrics.counter(SqsWorkerCommon.ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME)) + .thenReturn(acknowledgementSetCallbackCounter); + when(pluginMetrics.counter(SqsWorkerCommon.SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME)) + .thenReturn(sqsVisibilityTimeoutChangedCount); + when(pluginMetrics.counter(SqsWorkerCommon.SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME)) + .thenReturn(sqsVisibilityTimeoutChangeFailedCount); when(sqsSourceConfig.getAcknowledgements()).thenReturn(false); when(sqsSourceConfig.getBufferTimeout()).thenReturn(Duration.ofSeconds(10)); when(queueConfig.getUrl()).thenReturn("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"); diff --git a/settings.gradle b/settings.gradle index d86bc7e1da..d2aa09b52c 100644 --- a/settings.gradle +++ b/settings.gradle @@ -167,6 +167,7 @@ include 'data-prepper-plugins:parquet-codecs' include 'data-prepper-plugins:aws-sqs-common' include 'data-prepper-plugins:buffer-common' include 'data-prepper-plugins:sqs-source' +include 'data-prepper-plugins:sqs-common' //include 'data-prepper-plugins:cloudwatch-logs' //include 'data-prepper-plugins:http-sink' //include 'data-prepper-plugins:sns-sink' From 170b39524727ac367a22f9d24fcb714282582e6d Mon Sep 17 00:00:00 2001 From: Katherine Shen <40495707+shenkw1@users.noreply.github.com> Date: Mon, 3 Feb 2025 14:22:23 -0600 Subject: [PATCH 09/18] schema revisions, add json aliases (#5349) * schema revisions, add json aliases Signed-off-by: Katherine Shen --- .../aggregate/AggregateProcessorConfig.java | 2 +- .../AnomalyDetectorProcessorConfig.java | 2 +- .../modes/RandomCutForestModeConfig.java | 2 +- .../processor/LambdaProcessorConfig.java | 4 ++-- .../processor/date/DateProcessorConfig.java | 18 ++++++++++-------- .../drop/DropEventProcessorConfig.java | 2 +- .../flatten/FlattenProcessorConfig.java | 10 ++++++---- .../keyvalue/KeyValueProcessorConfig.java | 3 +-- .../mutateevent/AddEntryProcessorConfig.java | 9 +++++---- .../ConvertEntryTypeProcessorConfig.java | 4 +++- .../mutateevent/CopyValueProcessorConfig.java | 16 +++++++--------- .../ObfuscationProcessorConfig.java | 2 +- 12 files changed, 39 insertions(+), 35 deletions(-) diff --git a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java index e1522b23ed..3f7fedb539 100644 --- a/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java +++ b/data-prepper-plugins/aggregate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/aggregate/AggregateProcessorConfig.java @@ -31,7 +31,7 @@ public class AggregateProcessorConfig { @JsonPropertyDescription("An unordered list by which to group events. Events with the same values as these keys are put into the same group. " + "If an event does not contain one of the identification_keys, then the value of that key is considered to be equal to null. " + - "At least one identification_key is required. An example configuration is [\"sourceIp\", \"destinationIp\", \"port\"].") + "At least one identification_key is required.") @JsonProperty("identification_keys") @NotEmpty @ExampleValues({ diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java index 578e22a351..a8c45316ea 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/AnomalyDetectorProcessorConfig.java @@ -34,7 +34,7 @@ public class AnomalyDetectorProcessorConfig { @UsesDataPrepperPlugin(pluginType = AnomalyDetectorMode.class) private PluginModel detectorMode; - @JsonPropertyDescription("If provided, anomalies will be detected within each unique instance of these keys. For example, if you provide the ip field, anomalies will be detected separately for each unique IP address.") + @JsonPropertyDescription("If provided, anomalies will be detected within each unique instance of these keys. For example, if you provide the IP field, anomalies will be detected separately for each unique IP address.") @JsonProperty("identification_keys") @ExampleValues({ @Example(value = "ip_address", description = "Anomalies will be detected separately for each unique IP address from the existing ip_address key of the Event.") diff --git a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java index 74c90fca50..46657946d4 100644 --- a/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java +++ b/data-prepper-plugins/anomaly-detector-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/anomalydetector/modes/RandomCutForestModeConfig.java @@ -51,7 +51,7 @@ public class RandomCutForestModeConfig { @JsonProperty(value = "time_decay", defaultValue = "" + DEFAULT_TIME_DECAY) private double timeDecay = DEFAULT_TIME_DECAY; - @JsonPropertyDescription("Output after indicates the number of events to consume before outputting anomalies. Default is 32.") + @JsonPropertyDescription("The number of events to train the model before generating an anomaly event. Default is 32.") @JsonProperty(value = "output_after", defaultValue = "" + DEFAULT_OUTPUT_AFTER) private int outputAfter = DEFAULT_OUTPUT_AFTER; diff --git a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorConfig.java b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorConfig.java index c9675ed931..49b35ebf08 100644 --- a/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorConfig.java +++ b/data-prepper-plugins/aws-lambda/src/main/java/org/opensearch/dataprepper/plugins/lambda/processor/LambdaProcessorConfig.java @@ -17,7 +17,7 @@ import org.opensearch.dataprepper.plugins.lambda.common.config.LambdaCommonConfig; @JsonPropertyOrder -@JsonClassDescription("The aws_lambda processor enables invocation of an AWS Lambda function within your Data Prepper pipeline in order to process events." + +@JsonClassDescription("The aws_lambda processor enables invocation of an AWS Lambda function within your pipeline in order to process events. " + "It supports both synchronous and asynchronous invocations based on your use case.") public class LambdaProcessorConfig extends LambdaCommonConfig { static final String DEFAULT_INVOCATION_TYPE = "request-response"; @@ -40,7 +40,7 @@ public class LambdaProcessorConfig extends LambdaCommonConfig { @JsonPropertyDescription("Defines a condition for event to use this processor.") @ExampleValues({ - @Example(value = "event['status'] == 'process'", description = "The processor will only run on this condition.") + @Example(value = "/some_key == null", description = "The processor will only run on events where this condition evaluates to true.") }) @JsonProperty("lambda_when") private String whenCondition; diff --git a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java index 2e3114d979..05deb5aa07 100644 --- a/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java +++ b/data-prepper-plugins/date-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/date/DateProcessorConfig.java @@ -10,6 +10,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.fasterxml.jackson.annotation.JsonPropertyOrder; +import com.fasterxml.jackson.annotation.JsonAlias; import jakarta.validation.constraints.AssertTrue; import org.opensearch.dataprepper.model.annotations.AlsoRequired; import org.opensearch.dataprepper.model.annotations.ConditionalRequired; @@ -133,8 +134,7 @@ public static boolean isValidPattern(final String pattern) { private Boolean fromTimeReceived = DEFAULT_FROM_TIME_RECEIVED; @JsonProperty("match") - @JsonPropertyDescription("The date match configuration. " + - "This option cannot be defined at the same time as from_time_received. " + + @JsonPropertyDescription("This option cannot be defined at the same time as from_time_received. " + "The date processor will use the first pattern that matches each event's timestamp field. " + "You must provide at least one pattern unless you have from_time_received.") @AlsoRequired(values = { @@ -155,9 +155,11 @@ public static boolean isValidPattern(final String pattern) { }) private String outputFormat = DEFAULT_OUTPUT_FORMAT; - @JsonProperty("to_origination_metadata") - @JsonPropertyDescription("When true, the matched time is also added to the event's metadata as an instance of " + - "Instant. Default is false.") + @JsonProperty("origination_timestamp_to_metadata") + @JsonAlias("to_origination_metadata") + @JsonPropertyDescription("Include the origination timestamp in the metadata. " + + "Enabling this option will use this timestamp to report the EndToEndLatency metric " + + "when events reach the sink. Default is false.") private Boolean toOriginationMetadata = DEFAULT_TO_ORIGINATION_METADATA; @JsonProperty("source_timezone") @@ -187,7 +189,7 @@ public static boolean isValidPattern(final String pattern) { "or a string representation of the " + "locale object, such as en_US. " + "A full list of locale fields, including language, country, and variant, can be found " + - "here." + + "here. " + "Default is Locale.ROOT.") @ExampleValues({ @Example("en-US"), @@ -196,10 +198,10 @@ public static boolean isValidPattern(final String pattern) { private String locale; @JsonProperty("date_when") - @JsonPropertyDescription("Specifies under what condition the date processor should perform matching. " + + @JsonPropertyDescription("Specifies under what condition the date processor should run. " + "Default is no condition.") @ExampleValues({ - @Example(value = "/some_key == null", description = "Only runs the date processor on the Event if some_key is null or doesn't exist.") + @Example(value = "/some_key == null", description = "The processor will only run on events where this condition evaluates to true.") }) private String dateWhen; diff --git a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java index dbf1e9b63d..29e5de133d 100644 --- a/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java +++ b/data-prepper-plugins/drop-events-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/drop/DropEventProcessorConfig.java @@ -27,7 +27,7 @@ public class DropEventProcessorConfig { }) private String dropWhen; - @JsonPropertyDescription("Specifies how exceptions are handled when an exception occurs while evaluating an event. Default value is skip, which drops the event so that it is not sent to further processors or sinks.") + @JsonPropertyDescription("Specifies how exceptions are handled when an exception occurs while evaluating an event. Default value is skip, which sends the events to further processors or sinks.") @JsonProperty(value = "handle_failed_events", defaultValue = "skip") private HandleFailedEventsOption handleFailedEventsOption = HandleFailedEventsOption.SKIP; diff --git a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java index e5f4cb5c1b..747eeb4ab5 100644 --- a/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java +++ b/data-prepper-plugins/flatten-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/flatten/FlattenProcessorConfig.java @@ -7,9 +7,9 @@ import com.fasterxml.jackson.annotation.JsonClassDescription; import com.fasterxml.jackson.annotation.JsonProperty; - import com.fasterxml.jackson.annotation.JsonPropertyDescription; import com.fasterxml.jackson.annotation.JsonPropertyOrder; +import com.fasterxml.jackson.annotation.JsonAlias; import jakarta.validation.constraints.AssertTrue; import jakarta.validation.constraints.NotNull; import org.opensearch.dataprepper.model.annotations.ExampleValues; @@ -20,7 +20,7 @@ import java.util.List; @JsonPropertyOrder -@JsonClassDescription("The flatten processor transforms nested objects inside of events into flattened structures.") +@JsonClassDescription("The flatten processor transforms nested objects inside of events into flattened structures.") public class FlattenProcessorConfig { static final String REMOVE_LIST_INDICES_KEY = "remove_list_indices"; @@ -44,12 +44,14 @@ public class FlattenProcessorConfig { }) private String target; - @JsonProperty("remove_processed_fields") + @JsonProperty("remove_source_keys") + @JsonAlias("remove_processed_fields") @JsonPropertyDescription("When true, the processor removes all processed fields from the source. " + "The default is false which leaves the source fields.") private boolean removeProcessedFields = false; - @JsonProperty(REMOVE_LIST_INDICES_KEY) + @JsonProperty("remove_list_elements") + @JsonAlias(REMOVE_LIST_INDICES_KEY) @JsonPropertyDescription("When true, the processor converts the fields from the source map into lists and " + "puts the lists into the target field. Default is false.") private boolean removeListIndices = false; diff --git a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java index 501dbb9d54..cfc1804a30 100644 --- a/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java +++ b/data-prepper-plugins/key-value-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/keyvalue/KeyValueProcessorConfig.java @@ -195,8 +195,7 @@ public class KeyValueProcessorConfig { "If this flag is enabled, then the content between the delimiters is considered to be one entity and " + "they are not parsed as key-value pairs. The following characters are used a group delimiters: " + "{...}, [...], <...>, (...), \"...\", '...', http://... (space), and https:// (space). " + - "Default is false. For example, if value_grouping is true, then " + - "{\"key1=[a=b,c=d]&key2=value2\"} parses to {\"key1\": \"[a=b,c=d]\", \"key2\": \"value2\"}.") + "Default is false.") @AlsoRequired(values = { @AlsoRequired.Required(name = FIELD_DELIMITER_REGEX_KEY, allowedValues = {"null"}) }) diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java index 5529428ecb..cf46fc13ae 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/AddEntryProcessorConfig.java @@ -66,7 +66,7 @@ public class AddEntryProcessorConfig { public static class Entry { @JsonPropertyDescription("The key of the new entry to be added. Some examples of keys include my_key, " + "myKey, and object/sub_Key. The key can also be a format expression, for example, ${/key1} to " + - "use the value of field key1 as the key. Either one of key or metadata_key is required.") + "use the value of field key1 as the key. Exactly one of key or metadata_key is required.") @AlsoRequired(values = { @AlsoRequired.Required(name=METADATA_KEY_KEY, allowedValues = {"null"}) }) @@ -79,7 +79,8 @@ public static class Entry { @JsonProperty(METADATA_KEY_KEY) @JsonPropertyDescription("The key for the new metadata attribute. The argument must be a literal string key " + - "and not a JSON Pointer. Either one of key or metadata_key is required.") + "and not a JSON Pointer. Adds an attribute to the Events that will not be sent to the sinks, but can be used for condition expressions and routing with the getMetadata function. " + + "Exactly one of key or metadata_key is required.") @AlsoRequired(values = { @AlsoRequired.Required(name="key", allowedValues = {"null"}) }) @@ -103,7 +104,7 @@ public static class Entry { private Object value; @JsonPropertyDescription("A format string to use as the value of the new entry, for example, " + - "${key1}-${key2}, where key1 and key2 are existing keys in the event. Required if neither" + + "${key1}-${key2}, where key1 and key2 are existing keys in the event. Required if neither " + "value nor value_expression is specified.") @AlsoRequired(values = { @AlsoRequired.Required(name="value", allowedValues = {"null"}), @@ -132,7 +133,7 @@ public static class Entry { @JsonProperty(OVERWRITE_IF_KEY_EXISTS_KEY) @JsonPropertyDescription("When set to true, the existing value is overwritten if key already exists " + - "in the event. The default value is false.") + "in the event. Only one of overwrite_if_key_exists or append_if_key_exists can be true. The default value is false.") @AlsoRequired(values = { @AlsoRequired.Required(name=APPEND_IF_KEY_EXISTS_KEY, allowedValues = {"false"}) }) diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java index f184e02896..0d938792c1 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/ConvertEntryTypeProcessorConfig.java @@ -9,6 +9,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonPropertyOrder; import com.fasterxml.jackson.annotation.JsonPropertyDescription; +import com.fasterxml.jackson.annotation.JsonAlias; import org.opensearch.dataprepper.model.annotations.AlsoRequired; import org.opensearch.dataprepper.model.annotations.ConditionalRequired; import org.opensearch.dataprepper.model.annotations.ConditionalRequired.IfThenElse; @@ -55,7 +56,8 @@ public class ConvertEntryTypeProcessorConfig implements ConverterArguments { @JsonPropertyDescription("Target type for the values. Default value is integer.") private TargetType type = TargetType.INTEGER; - @JsonProperty("null_values") + @JsonProperty("null_conversion_values") + @JsonAlias("null_values") @JsonPropertyDescription("String representation of what constitutes a null value. If the field value equals one of these strings, then the value is considered null and is converted to null.") private List nullValues; diff --git a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java index e44c1da74e..d0f991fc33 100644 --- a/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java +++ b/data-prepper-plugins/mutate-event-processors/src/main/java/org/opensearch/dataprepper/plugins/processor/mutateevent/CopyValueProcessorConfig.java @@ -20,7 +20,7 @@ import java.util.List; @JsonPropertyOrder -@JsonClassDescription("The copy_values processor copies values within an event to other fields within the event.") +@JsonClassDescription("The copy_values processor copies values from an event to another key in an event.") public class CopyValueProcessorConfig { static final String FROM_LIST_KEY = "from_list"; static final String TO_LIST_KEY = "to_list"; @@ -30,15 +30,13 @@ public static class Entry { @NotEmpty @NotNull @JsonProperty("from_key") - @JsonPropertyDescription("The key of the entry to be copied. Either from_key and " + - "to_key or from_list and to_list must be defined.") + @JsonPropertyDescription("The key of the entry to be copied. This must be configured.") private String fromKey; @NotEmpty @NotNull @JsonProperty("to_key") - @JsonPropertyDescription("The key of the new entry to be added. Either from_key and " + - "to_key or from_list and to_list must be defined.") + @JsonPropertyDescription("The key of the new entry to be added. This must be configured.") private String toKey; @JsonProperty("overwrite_if_to_key_exists") @@ -88,16 +86,16 @@ public Entry() { private List entries; @JsonProperty(FROM_LIST_KEY) - @JsonPropertyDescription("The key of the list of objects to be copied. Either from_key and " + - "to_key or from_list and to_list must be defined.") + @JsonPropertyDescription("The key of the list of objects to be copied. " + + "Both from_key and to_key must be configured and will be applied on the corresponding list.") @AlsoRequired(values = { @AlsoRequired.Required(name = TO_LIST_KEY) }) private String fromList; @JsonProperty(TO_LIST_KEY) - @JsonPropertyDescription("The key of the new list to be added. Either from_key and " + - "to_key or from_list and to_list must be defined.") + @JsonPropertyDescription("The key of the new list to be added. " + + "Both from_key and to_key must be configured and will be applied on the corresponding list.") @AlsoRequired(values = { @AlsoRequired.Required(name = FROM_LIST_KEY) }) diff --git a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java index fa0ddf355c..935d99c219 100644 --- a/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java +++ b/data-prepper-plugins/obfuscate-processor/src/main/java/org/opensearch/dataprepper/plugins/processor/obfuscation/ObfuscationProcessorConfig.java @@ -27,7 +27,7 @@ public class ObfuscationProcessorConfig { @JsonProperty("source") - @JsonPropertyDescription("The source field to obfuscate.") + @JsonPropertyDescription("The source key to obfuscate. Default action is to mask with *.") @NotEmpty @NotNull @ExampleValues({ From b181a8d281f17a8d13ff225c768ba7c00de08bdd Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Tue, 4 Feb 2025 11:58:01 -0600 Subject: [PATCH 10/18] Add e2e ack, checkpointing and metrics to Postgres stream processing (#5375) * Initial commit Signed-off-by: Hai Yan * Update unit tests Signed-off-by: Hai Yan * Add more metrics Signed-off-by: Hai Yan * Add more tests Signed-off-by: Hai Yan * Address review comments Signed-off-by: Hai Yan * Address review comments Signed-off-by: Hai Yan --------- Signed-off-by: Hai Yan --- .../rds/resync/CascadingActionDetector.java | 8 +- .../rds/stream/BinlogEventListener.java | 10 +- .../source/rds/stream/ChangeEventStatus.java | 23 ++- .../rds/stream/LogicalReplicationClient.java | 6 +- .../LogicalReplicationEventProcessor.java | 149 +++++++++++++++--- .../rds/stream/StreamCheckpointManager.java | 81 ++++++++-- .../source/rds/stream/StreamCheckpointer.java | 22 ++- .../rds/stream/StreamWorkerTaskRefresher.java | 5 +- .../source/rds/export/DataFileLoaderTest.java | 4 +- .../stream/LogicalReplicationClientTest.java | 88 +++++++++++ .../LogicalReplicationEventProcessorTest.java | 28 +++- .../stream/StreamCheckpointManagerTest.java | 61 ++++++- .../rds/stream/StreamCheckpointerTest.java | 29 +++- 13 files changed, 454 insertions(+), 60 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java index cb7f060976..4bab68b3f4 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/resync/CascadingActionDetector.java @@ -83,9 +83,9 @@ public Map getParentTableMap(StreamPartition streamPartitio /** * Detects if a binlog event contains cascading updates and if detected, creates resync partitions - * @param event event + * @param event binlog event * @param parentTableMap parent table map - * @param tableMetadata table meta data + * @param tableMetadata table metadata */ public void detectCascadingUpdates(Event event, Map parentTableMap, TableMetadata tableMetadata) { final UpdateRowsEventData data = event.getData(); @@ -143,9 +143,9 @@ public void detectCascadingUpdates(Event event, Map parentT /** * Detects if a binlog event contains cascading deletes and if detected, creates resync partitions - * @param event event + * @param event binlog event * @param parentTableMap parent table map - * @param tableMetadata table meta data + * @param tableMetadata table metadata */ public void detectCascadingDeletes(Event event, Map parentTableMap, TableMetadata tableMetadata) { if (parentTableMap.containsKey(tableMetadata.getFullTableName())) { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java index 2bc21ca786..1612e94ec3 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/BinlogEventListener.java @@ -130,7 +130,8 @@ public BinlogEventListener(final StreamPartition streamPartition, this.dbTableMetadata = dbTableMetadata; this.streamCheckpointManager = new StreamCheckpointManager( streamCheckpointer, sourceConfig.isAcknowledgmentsEnabled(), - acknowledgementSetManager, this::stopClient, sourceConfig.getStreamAcknowledgmentTimeout()); + acknowledgementSetManager, this::stopClient, sourceConfig.getStreamAcknowledgmentTimeout(), + sourceConfig.getEngine(), pluginMetrics); streamCheckpointManager.start(); this.cascadeActionDetector = cascadeActionDetector; @@ -200,7 +201,7 @@ void handleRotateEvent(com.github.shyiko.mysql.binlog.event.Event event) { // Trigger a checkpoint update for this rotate when there're no row mutation events being processed if (streamCheckpointManager.getChangeEventStatuses().isEmpty()) { - ChangeEventStatus changeEventStatus = streamCheckpointManager.saveChangeEventsStatus(currentBinlogCoordinate); + ChangeEventStatus changeEventStatus = streamCheckpointManager.saveChangeEventsStatus(currentBinlogCoordinate, 0); if (isAcknowledgmentsEnabled) { changeEventStatus.setAcknowledgmentStatus(ChangeEventStatus.AcknowledgmentStatus.POSITIVE_ACK); } @@ -347,9 +348,10 @@ void handleRowChangeEvent(com.github.shyiko.mysql.binlog.event.Event event, LOG.debug("Current binlog coordinate after receiving a row change event: " + currentBinlogCoordinate); } + final long recordCount = rows.size(); AcknowledgementSet acknowledgementSet = null; if (isAcknowledgmentsEnabled) { - acknowledgementSet = streamCheckpointManager.createAcknowledgmentSet(currentBinlogCoordinate); + acknowledgementSet = streamCheckpointManager.createAcknowledgmentSet(currentBinlogCoordinate, recordCount); } final long bytes = event.toString().getBytes().length; @@ -398,7 +400,7 @@ void handleRowChangeEvent(com.github.shyiko.mysql.binlog.event.Event event, if (isAcknowledgmentsEnabled) { acknowledgementSet.complete(); } else { - streamCheckpointManager.saveChangeEventsStatus(currentBinlogCoordinate); + streamCheckpointManager.saveChangeEventsStatus(currentBinlogCoordinate, recordCount); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ChangeEventStatus.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ChangeEventStatus.java index f2b70cbe7b..af6ef02362 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ChangeEventStatus.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/ChangeEventStatus.java @@ -6,11 +6,14 @@ package org.opensearch.dataprepper.plugins.source.rds.stream; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.postgresql.replication.LogSequenceNumber; public class ChangeEventStatus { private final BinlogCoordinate binlogCoordinate; + private final LogSequenceNumber logSequenceNumber; private final long timestamp; + private final long recordCount; private volatile AcknowledgmentStatus acknowledgmentStatus; public enum AcknowledgmentStatus { @@ -19,9 +22,19 @@ public enum AcknowledgmentStatus { NO_ACK } - public ChangeEventStatus(final BinlogCoordinate binlogCoordinate, final long timestamp) { + public ChangeEventStatus(final BinlogCoordinate binlogCoordinate, final long timestamp, final long recordCount) { this.binlogCoordinate = binlogCoordinate; + this.logSequenceNumber = null; this.timestamp = timestamp; + this.recordCount = recordCount; + acknowledgmentStatus = AcknowledgmentStatus.NO_ACK; + } + + public ChangeEventStatus(final LogSequenceNumber logSequenceNumber, final long timestamp, final long recordCount) { + this.binlogCoordinate = null; + this.logSequenceNumber = logSequenceNumber; + this.timestamp = timestamp; + this.recordCount = recordCount; acknowledgmentStatus = AcknowledgmentStatus.NO_ACK; } @@ -45,7 +58,15 @@ public BinlogCoordinate getBinlogCoordinate() { return binlogCoordinate; } + public LogSequenceNumber getLogSequenceNumber() { + return logSequenceNumber; + } + public long getTimestamp() { return timestamp; } + + public long getRecordCount() { + return recordCount; + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java index 22935fc6e3..8eb3b9cde9 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java @@ -47,6 +47,7 @@ public LogicalReplicationClient(final ConnectionManager connectionManager, @Override public void connect() { + LOG.debug("Start connecting logical replication stream. "); PGReplicationStream stream; try (Connection conn = connectionManager.getConnection()) { PGConnection pgConnection = conn.unwrap(PGConnection.class); @@ -62,6 +63,7 @@ public void connect() { logicalStreamBuilder.withStartPosition(startLsn); } stream = logicalStreamBuilder.start(); + LOG.debug("Logical replication stream started. "); if (eventProcessor != null) { while (!disconnectRequested) { @@ -88,7 +90,8 @@ public void connect() { } stream.close(); - LOG.info("Replication stream closed successfully."); + disconnectRequested = false; + LOG.debug("Replication stream closed successfully."); } catch (Exception e) { LOG.error("Exception while creating Postgres replication stream. ", e); } @@ -97,6 +100,7 @@ public void connect() { @Override public void disconnect() { disconnectRequested = true; + LOG.debug("Requested to disconnect logical replication stream."); } public void setEventProcessor(LogicalReplicationEventProcessor eventProcessor) { diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java index 3d5c1a04b1..a2a9aa1017 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java @@ -10,7 +10,13 @@ package org.opensearch.dataprepper.plugins.source.rds.stream; +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; +import io.micrometer.core.instrument.Timer; import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.event.JacksonEvent; @@ -23,6 +29,7 @@ import org.opensearch.dataprepper.plugins.source.rds.datatype.postgres.ColumnType; import org.opensearch.dataprepper.plugins.source.rds.model.MessageType; import org.opensearch.dataprepper.plugins.source.rds.model.TableMetadata; +import org.postgresql.replication.LogSequenceNumber; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -32,6 +39,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.Consumer; public class LogicalReplicationEventProcessor { enum TupleDataType { @@ -63,6 +71,14 @@ public static TupleDataType fromValue(char value) { static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); static final int DEFAULT_BUFFER_BATCH_SIZE = 1_000; + static final int NUM_OF_RETRIES = 3; + static final int BACKOFF_IN_MILLIS = 500; + static final String CHANGE_EVENTS_PROCESSED_COUNT = "changeEventsProcessed"; + static final String CHANGE_EVENTS_PROCESSING_ERROR_COUNT = "changeEventsProcessingErrors"; + static final String BYTES_RECEIVED = "bytesReceived"; + static final String BYTES_PROCESSED = "bytesProcessed"; + static final String REPLICATION_LOG_EVENT_PROCESSING_TIME = "replicationLogEntryProcessingTime"; + static final String REPLICATION_LOG_PROCESSING_ERROR_COUNT = "replicationLogEntryProcessingErrors"; private final StreamPartition streamPartition; private final RdsSourceConfig sourceConfig; @@ -70,24 +86,57 @@ public static TupleDataType fromValue(char value) { private final Buffer> buffer; private final BufferAccumulator> bufferAccumulator; private final List pipelineEvents; + private final PluginMetrics pluginMetrics; + private final AcknowledgementSetManager acknowledgementSetManager; + private final LogicalReplicationClient logicalReplicationClient; + private final StreamCheckpointer streamCheckpointer; + private final StreamCheckpointManager streamCheckpointManager; + + private final Counter changeEventSuccessCounter; + private final Counter changeEventErrorCounter; + private final DistributionSummary bytesReceivedSummary; + private final DistributionSummary bytesProcessedSummary; + private final Timer eventProcessingTimer; + private final Counter eventProcessingErrorCounter; private long currentLsn; private long currentEventTimestamp; + private long bytesReceived; private Map tableMetadataMap; public LogicalReplicationEventProcessor(final StreamPartition streamPartition, final RdsSourceConfig sourceConfig, final Buffer> buffer, - final String s3Prefix) { + final String s3Prefix, + final PluginMetrics pluginMetrics, + final LogicalReplicationClient logicalReplicationClient, + final StreamCheckpointer streamCheckpointer, + final AcknowledgementSetManager acknowledgementSetManager) { this.streamPartition = streamPartition; this.sourceConfig = sourceConfig; recordConverter = new StreamRecordConverter(s3Prefix, sourceConfig.getPartitionCount()); this.buffer = buffer; bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); + this.pluginMetrics = pluginMetrics; + this.acknowledgementSetManager = acknowledgementSetManager; + this.logicalReplicationClient = logicalReplicationClient; + this.streamCheckpointer = streamCheckpointer; + streamCheckpointManager = new StreamCheckpointManager( + streamCheckpointer, sourceConfig.isAcknowledgmentsEnabled(), + acknowledgementSetManager, this::stopClient, sourceConfig.getStreamAcknowledgmentTimeout(), + sourceConfig.getEngine(), pluginMetrics); + streamCheckpointManager.start(); tableMetadataMap = new HashMap<>(); pipelineEvents = new ArrayList<>(); + + changeEventSuccessCounter = pluginMetrics.counter(CHANGE_EVENTS_PROCESSED_COUNT); + changeEventErrorCounter = pluginMetrics.counter(CHANGE_EVENTS_PROCESSING_ERROR_COUNT); + bytesReceivedSummary = pluginMetrics.summary(BYTES_RECEIVED); + bytesProcessedSummary = pluginMetrics.summary(BYTES_PROCESSED); + eventProcessingTimer = pluginMetrics.timer(REPLICATION_LOG_EVENT_PROCESSING_TIME); + eventProcessingErrorCounter = pluginMetrics.counter(REPLICATION_LOG_PROCESSING_ERROR_COUNT); } public void process(ByteBuffer msg) { @@ -97,20 +146,36 @@ public void process(ByteBuffer msg) { // If it's INSERT/UPDATE/DELETE, prepare events // If it's a COMMIT, convert all prepared events and send to buffer MessageType messageType = MessageType.from((char) msg.get()); - if (messageType == MessageType.BEGIN) { - processBeginMessage(msg); - } else if (messageType == MessageType.RELATION) { - processRelationMessage(msg); - } else if (messageType == MessageType.INSERT) { - processInsertMessage(msg); - } else if (messageType == MessageType.UPDATE) { - processUpdateMessage(msg); - } else if (messageType == MessageType.DELETE) { - processDeleteMessage(msg); - } else if (messageType == MessageType.COMMIT) { - processCommitMessage(msg); - } else { - throw new IllegalArgumentException("Replication message type [" + messageType + "] is not supported. "); + switch (messageType) { + case BEGIN: + handleMessageWithRetries(msg, this::processBeginMessage, messageType); + break; + case RELATION: + handleMessageWithRetries(msg, this::processRelationMessage, messageType); + break; + case INSERT: + handleMessageWithRetries(msg, this::processInsertMessage, messageType); + break; + case UPDATE: + handleMessageWithRetries(msg, this::processUpdateMessage, messageType); + break; + case DELETE: + handleMessageWithRetries(msg, this::processDeleteMessage, messageType); + break; + case COMMIT: + handleMessageWithRetries(msg, this::processCommitMessage, messageType); + break; + default: + throw new IllegalArgumentException("Replication message type [" + messageType + "] is not supported. "); + } + } + + public void stopClient() { + try { + logicalReplicationClient.disconnect(); + LOG.info("Binary log client disconnected."); + } catch (Exception e) { + LOG.error("Binary log client failed to disconnect.", e); } } @@ -169,15 +234,28 @@ void processCommitMessage(ByteBuffer msg) { throw new RuntimeException("Commit LSN does not match current LSN, skipping"); } - writeToBuffer(bufferAccumulator); + final long recordCount = pipelineEvents.size(); + AcknowledgementSet acknowledgementSet = null; + if (sourceConfig.isAcknowledgmentsEnabled()) { + acknowledgementSet = streamCheckpointManager.createAcknowledgmentSet(LogSequenceNumber.valueOf(currentLsn), recordCount); + } + + writeToBuffer(bufferAccumulator, acknowledgementSet); + bytesProcessedSummary.record(bytesReceived); LOG.debug("Processed a COMMIT message with Flag: {} CommitLsn: {} EndLsn: {} Timestamp: {}", flag, commitLsn, endLsn, epochMicro); + + if (sourceConfig.isAcknowledgmentsEnabled()) { + acknowledgementSet.complete(); + } else { + streamCheckpointManager.saveChangeEventsStatus(LogSequenceNumber.valueOf(currentLsn), recordCount); + } } void processInsertMessage(ByteBuffer msg) { int tableId = msg.getInt(); char n_char = (char) msg.get(); // Skip the 'N' character - final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); + final TableMetadata tableMetadata = tableMetadataMap.get((long) tableId); final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = currentEventTimestamp; @@ -189,7 +267,7 @@ void processInsertMessage(ByteBuffer msg) { void processUpdateMessage(ByteBuffer msg) { final int tableId = msg.getInt(); - final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); + final TableMetadata tableMetadata = tableMetadataMap.get((long) tableId); final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = currentEventTimestamp; @@ -231,7 +309,7 @@ void processDeleteMessage(ByteBuffer msg) { int tableId = msg.getInt(); char n_char = (char) msg.get(); // Skip the 'N' character - final TableMetadata tableMetadata = tableMetadataMap.get((long)tableId); + final TableMetadata tableMetadata = tableMetadataMap.get((long) tableId); final List columnNames = tableMetadata.getColumnNames(); final List primaryKeys = tableMetadata.getPrimaryKeys(); final long eventTimestampMillis = currentEventTimestamp; @@ -242,6 +320,8 @@ void processDeleteMessage(ByteBuffer msg) { private void doProcess(ByteBuffer msg, List columnNames, TableMetadata tableMetadata, List primaryKeys, long eventTimestampMillis, OpenSearchBulkActions bulkAction) { + bytesReceived = msg.capacity(); + bytesReceivedSummary.record(bytesReceived); Map rowDataMap = getRowDataMap(msg, columnNames); createPipelineEvent(rowDataMap, tableMetadata, primaryKeys, eventTimestampMillis, bulkAction); @@ -284,9 +364,12 @@ private void createPipelineEvent(Map rowDataMap, TableMetadata t pipelineEvents.add(pipelineEvent); } - private void writeToBuffer(BufferAccumulator> bufferAccumulator) { + private void writeToBuffer(BufferAccumulator> bufferAccumulator, AcknowledgementSet acknowledgementSet) { for (Event pipelineEvent : pipelineEvents) { addToBufferAccumulator(bufferAccumulator, new Record<>(pipelineEvent)); + if (acknowledgementSet != null) { + acknowledgementSet.add(pipelineEvent); + } } flushBufferAccumulator(bufferAccumulator, pipelineEvents.size()); @@ -304,10 +387,12 @@ private void addToBufferAccumulator(final BufferAccumulator> buffe private void flushBufferAccumulator(BufferAccumulator> bufferAccumulator, int eventCount) { try { bufferAccumulator.flush(); + changeEventSuccessCounter.increment(eventCount); } catch (Exception e) { // this will only happen if writing to buffer gets interrupted from shutdown, // otherwise bufferAccumulator will keep retrying with backoff LOG.error("Failed to flush buffer", e); + changeEventErrorCounter.increment(eventCount); } } @@ -333,4 +418,28 @@ private List getPrimaryKeys(String schemaName, String tableName) { return progressState.getPrimaryKeyMap().get(databaseName + "." + schemaName + "." + tableName); } + + private void handleMessageWithRetries(ByteBuffer message, Consumer function, MessageType messageType) { + int retry = 0; + while (retry <= NUM_OF_RETRIES) { + try { + eventProcessingTimer.record(() -> function.accept(message)); + return; + } catch (Exception e) { + LOG.warn("Error when processing change event of type {}, will retry", messageType, e); + applyBackoff(); + } + retry++; + } + LOG.error("Failed to process change event of type {} after {} retries", messageType, NUM_OF_RETRIES); + eventProcessingErrorCounter.increment(); + } + + private void applyBackoff() { + try { + Thread.sleep(BACKOFF_IN_MILLIS); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManager.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManager.java index 3827f2b822..3880707e21 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManager.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManager.java @@ -5,9 +5,13 @@ package org.opensearch.dataprepper.plugins.source.rds.stream; +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.postgresql.replication.LogSequenceNumber; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -21,6 +25,11 @@ public class StreamCheckpointManager { private static final Logger LOG = LoggerFactory.getLogger(StreamCheckpointManager.class); static final int REGULAR_CHECKPOINT_INTERVAL_MILLIS = 60_000; static final int CHANGE_EVENT_COUNT_PER_CHECKPOINT_BATCH = 1000; + static final String POSITIVE_ACKNOWLEDGEMENT_SET_METRIC_NAME = "positiveAcknowledgementSets"; + static final String NEGATIVE_ACKNOWLEDGEMENT_SET_METRIC_NAME = "negativeAcknowledgementSets"; + static final String CHECKPOINT_COUNT = "checkpointCount"; + static final String NO_DATA_EXTEND_LEASE_COUNT = "noDataExtendLeaseCount"; + static final String GIVE_UP_PARTITION_COUNT = "giveupPartitionCount"; private final ConcurrentLinkedQueue changeEventStatuses = new ConcurrentLinkedQueue<>(); private final StreamCheckpointer streamCheckpointer; @@ -29,18 +38,35 @@ public class StreamCheckpointManager { private final boolean isAcknowledgmentEnabled; private final AcknowledgementSetManager acknowledgementSetManager; private final Duration acknowledgmentTimeout; + private final EngineType engineType; + private final PluginMetrics pluginMetrics; + private final Counter positiveAcknowledgementSets; + private final Counter negativeAcknowledgementSets; + private final Counter checkpointCount; + private final Counter noDataExtendLeaseCount; + private final Counter giveupPartitionCount; public StreamCheckpointManager(final StreamCheckpointer streamCheckpointer, final boolean isAcknowledgmentEnabled, final AcknowledgementSetManager acknowledgementSetManager, final Runnable stopStreamRunnable, - final Duration acknowledgmentTimeout) { + final Duration acknowledgmentTimeout, + final EngineType engineType, + final PluginMetrics pluginMetrics) { this.acknowledgementSetManager = acknowledgementSetManager; this.streamCheckpointer = streamCheckpointer; this.isAcknowledgmentEnabled = isAcknowledgmentEnabled; this.stopStreamRunnable = stopStreamRunnable; this.acknowledgmentTimeout = acknowledgmentTimeout; + this.engineType = engineType; + this.pluginMetrics = pluginMetrics; executorService = Executors.newSingleThreadExecutor(); + + this.positiveAcknowledgementSets = pluginMetrics.counter(POSITIVE_ACKNOWLEDGEMENT_SET_METRIC_NAME); + this.negativeAcknowledgementSets = pluginMetrics.counter(NEGATIVE_ACKNOWLEDGEMENT_SET_METRIC_NAME); + this.checkpointCount = pluginMetrics.counter(CHECKPOINT_COUNT); + this.noDataExtendLeaseCount = pluginMetrics.counter(NO_DATA_EXTEND_LEASE_COUNT); + this.giveupPartitionCount = pluginMetrics.counter(GIVE_UP_PARTITION_COUNT); } public void start() { @@ -54,6 +80,7 @@ void runCheckpointing() { try { if (changeEventStatuses.isEmpty()) { LOG.debug("No records processed. Extend the lease on stream partition."); + noDataExtendLeaseCount.increment(); streamCheckpointer.extendLease(); } else { if (isAcknowledgmentEnabled) { @@ -65,13 +92,14 @@ void runCheckpointing() { } if (lastChangeEventStatus != null) { - streamCheckpointer.checkpoint(lastChangeEventStatus.getBinlogCoordinate()); + checkpoint(engineType, lastChangeEventStatus); } // If negative ack is seen, give up partition and exit loop to stop processing stream if (currentChangeEventStatus != null && currentChangeEventStatus.isNegativeAcknowledgment()) { LOG.info("Received negative acknowledgement for change event at {}. Will restart from most recent checkpoint", currentChangeEventStatus.getBinlogCoordinate()); streamCheckpointer.giveUpPartition(); + giveupPartitionCount.increment(); break; } } else { @@ -81,10 +109,10 @@ void runCheckpointing() { changeEventCount++; // In case queue are populated faster than the poll, checkpoint when reaching certain count if (changeEventCount % CHANGE_EVENT_COUNT_PER_CHECKPOINT_BATCH == 0) { - streamCheckpointer.checkpoint(currentChangeEventStatus.getBinlogCoordinate()); + checkpoint(engineType, currentChangeEventStatus); } } while (!changeEventStatuses.isEmpty()); - streamCheckpointer.checkpoint(currentChangeEventStatus.getBinlogCoordinate()); + checkpoint(engineType, currentChangeEventStatus); } } } catch (Exception e) { @@ -107,25 +135,52 @@ public void stop() { executorService.shutdownNow(); } - public ChangeEventStatus saveChangeEventsStatus(BinlogCoordinate binlogCoordinate) { - final ChangeEventStatus changeEventStatus = new ChangeEventStatus(binlogCoordinate, Instant.now().toEpochMilli()); + public ChangeEventStatus saveChangeEventsStatus(BinlogCoordinate binlogCoordinate, long recordCount) { + final ChangeEventStatus changeEventStatus = new ChangeEventStatus(binlogCoordinate, Instant.now().toEpochMilli(), recordCount); changeEventStatuses.add(changeEventStatus); return changeEventStatus; } - public AcknowledgementSet createAcknowledgmentSet(BinlogCoordinate binlogCoordinate) { + public ChangeEventStatus saveChangeEventsStatus(LogSequenceNumber logSequenceNumber, long recordCount) { + final ChangeEventStatus changeEventStatus = new ChangeEventStatus(logSequenceNumber, Instant.now().toEpochMilli(), recordCount); + changeEventStatuses.add(changeEventStatus); + return changeEventStatus; + } + + public AcknowledgementSet createAcknowledgmentSet(BinlogCoordinate binlogCoordinate, long recordCount) { LOG.debug("Create acknowledgment set for events receive prior to {}", binlogCoordinate); - final ChangeEventStatus changeEventStatus = new ChangeEventStatus(binlogCoordinate, Instant.now().toEpochMilli()); + final ChangeEventStatus changeEventStatus = new ChangeEventStatus(binlogCoordinate, Instant.now().toEpochMilli(), recordCount); changeEventStatuses.add(changeEventStatus); + return getAcknowledgementSet(changeEventStatus); + } + + public AcknowledgementSet createAcknowledgmentSet(LogSequenceNumber logSequenceNumber, long recordCount) { + LOG.debug("Create acknowledgment set for events receive prior to {}", logSequenceNumber); + final ChangeEventStatus changeEventStatus = new ChangeEventStatus(logSequenceNumber, Instant.now().toEpochMilli(), recordCount); + changeEventStatuses.add(changeEventStatus); + return getAcknowledgementSet(changeEventStatus); + } + + private AcknowledgementSet getAcknowledgementSet(ChangeEventStatus changeEventStatus) { return acknowledgementSetManager.create((result) -> { - if (result) { - changeEventStatus.setAcknowledgmentStatus(ChangeEventStatus.AcknowledgmentStatus.POSITIVE_ACK); - } else { - changeEventStatus.setAcknowledgmentStatus(ChangeEventStatus.AcknowledgmentStatus.NEGATIVE_ACK); - } + if (result) { + positiveAcknowledgementSets.increment(); + changeEventStatus.setAcknowledgmentStatus(ChangeEventStatus.AcknowledgmentStatus.POSITIVE_ACK); + } else { + negativeAcknowledgementSets.increment(); + changeEventStatus.setAcknowledgmentStatus(ChangeEventStatus.AcknowledgmentStatus.NEGATIVE_ACK); + } }, acknowledgmentTimeout); } + private void checkpoint(final EngineType engineType, final ChangeEventStatus changeEventStatus) { + LOG.debug("Checkpoint at {} with record count {}. ", engineType == EngineType.MYSQL ? + changeEventStatus.getBinlogCoordinate() : changeEventStatus.getLogSequenceNumber(), + changeEventStatus.getRecordCount()); + streamCheckpointer.checkpoint(engineType, changeEventStatus); + checkpointCount.increment(); + } + //VisibleForTesting ConcurrentLinkedQueue getChangeEventStatuses() { return changeEventStatuses; diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java index 1f60f9715f..2875bf5544 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointer.java @@ -8,9 +8,11 @@ import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.postgresql.replication.LogSequenceNumber; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,7 +40,17 @@ public StreamCheckpointer(final EnhancedSourceCoordinator sourceCoordinator, checkpointCounter = pluginMetrics.counter(CHECKPOINT_COUNT); } - public void checkpoint(final BinlogCoordinate binlogCoordinate) { + public void checkpoint(final EngineType engineType, final ChangeEventStatus changeEventStatus) { + if (engineType == EngineType.MYSQL) { + checkpoint(changeEventStatus.getBinlogCoordinate()); + } else if (engineType == EngineType.POSTGRES) { + checkpoint(changeEventStatus.getLogSequenceNumber()); + } else { + throw new IllegalArgumentException("Unsupported engine type " + engineType); + } + } + + private void checkpoint(final BinlogCoordinate binlogCoordinate) { LOG.debug("Checkpointing stream partition {} with binlog coordinate {}", streamPartition.getPartitionKey(), binlogCoordinate); Optional progressState = streamPartition.getProgressState(); progressState.get().getMySqlStreamState().setCurrentPosition(binlogCoordinate); @@ -46,6 +58,14 @@ public void checkpoint(final BinlogCoordinate binlogCoordinate) { checkpointCounter.increment(); } + private void checkpoint(final LogSequenceNumber logSequenceNumber) { + LOG.debug("Checkpointing stream partition {} with log sequence number {}", streamPartition.getPartitionKey(), logSequenceNumber); + Optional progressState = streamPartition.getProgressState(); + progressState.get().getPostgresStreamState().setCurrentLsn(logSequenceNumber.asString()); + sourceCoordinator.saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + checkpointCounter.increment(); + } + public void extendLease() { LOG.debug("Extending lease of stream partition {}", streamPartition.getPartitionKey()); sourceCoordinator.saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java index acd8d0535f..7d89855365 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorkerTaskRefresher.java @@ -130,8 +130,8 @@ private void refreshTask(RdsSourceConfig sourceConfig) { } else { final LogicalReplicationClient logicalReplicationClient = (LogicalReplicationClient) replicationLogClient; logicalReplicationClient.setEventProcessor(new LogicalReplicationEventProcessor( - streamPartition, sourceConfig, buffer, s3Prefix - )); + streamPartition, sourceConfig, buffer, s3Prefix, pluginMetrics, logicalReplicationClient, + streamCheckpointer, acknowledgementSetManager)); } final StreamWorker streamWorker = StreamWorker.create(sourceCoordinator, replicationLogClient, pluginMetrics); executorService.submit(() -> streamWorker.processStream(streamPartition)); @@ -150,4 +150,3 @@ private DbTableMetadata getDBTableMetadata(final StreamPartition streamPartition return DbTableMetadata.fromMap(globalState.getProgressState().get()); } } - diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java index 6eeedfcd0f..efc831acfd 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoaderTest.java @@ -140,7 +140,7 @@ void test_run_success() throws Exception { ParquetReader parquetReader = mock(ParquetReader.class); BufferAccumulator> bufferAccumulator = mock(BufferAccumulator.class); when(builder.build()).thenReturn(parquetReader); - when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), (GenericRecord) null); try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class); MockedStatic bufferAccumulatorMockedStatic = mockStatic(BufferAccumulator.class)) { @@ -191,7 +191,7 @@ void test_flush_failure_then_error_metric_updated() throws Exception { BufferAccumulator> bufferAccumulator = mock(BufferAccumulator.class); doThrow(new RuntimeException("testing")).when(bufferAccumulator).flush(); when(builder.build()).thenReturn(parquetReader); - when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), null); + when(parquetReader.read()).thenReturn(mock(GenericRecord.class, RETURNS_DEEP_STUBS), (GenericRecord) null); try (MockedStatic readerMockedStatic = mockStatic(AvroParquetReader.class); MockedStatic bufferAccumulatorMockedStatic = mockStatic(BufferAccumulator.class)) { readerMockedStatic.when(() -> AvroParquetReader.builder(any(InputFile.class), any())).thenReturn(builder); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java index 9cd410ee44..45897335b5 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java @@ -33,7 +33,9 @@ import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) @@ -87,6 +89,92 @@ void test_connect() throws SQLException, InterruptedException { verify(stream).setFlushedLSN(lsn); } + @Test + void test_disconnect() throws SQLException, InterruptedException { + final Connection connection = mock(Connection.class); + final PGConnection pgConnection = mock(PGConnection.class, RETURNS_DEEP_STUBS); + final ChainedLogicalStreamBuilder logicalStreamBuilder = mock(ChainedLogicalStreamBuilder.class); + final PGReplicationStream stream = mock(PGReplicationStream.class); + final ByteBuffer message = ByteBuffer.allocate(0); + final LogSequenceNumber lsn = mock(LogSequenceNumber.class); + + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.unwrap(PGConnection.class)).thenReturn(pgConnection); + when(pgConnection.getReplicationAPI().replicationStream().logical()).thenReturn(logicalStreamBuilder); + when(logicalStreamBuilder.withSlotName(anyString())).thenReturn(logicalStreamBuilder); + when(logicalStreamBuilder.withSlotOption(anyString(), anyString())).thenReturn(logicalStreamBuilder); + when(logicalStreamBuilder.start()).thenReturn(stream); + when(stream.readPending()).thenReturn(message).thenReturn(null); + when(stream.getLastReceiveLSN()).thenReturn(lsn); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> logicalReplicationClient.connect()); + + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(eventProcessor).process(message)); + Thread.sleep(20); + verify(stream).setAppliedLSN(lsn); + verify(stream).setFlushedLSN(lsn); + + logicalReplicationClient.disconnect(); + Thread.sleep(20); + verify(stream).close(); + verifyNoMoreInteractions(stream, eventProcessor); + + executorService.shutdownNow(); + } + + @Test + void test_connect_disconnect_cycles() throws SQLException, InterruptedException { + final Connection connection = mock(Connection.class); + final PGConnection pgConnection = mock(PGConnection.class, RETURNS_DEEP_STUBS); + final ChainedLogicalStreamBuilder logicalStreamBuilder = mock(ChainedLogicalStreamBuilder.class); + final PGReplicationStream stream = mock(PGReplicationStream.class); + final ByteBuffer message = ByteBuffer.allocate(0); + final LogSequenceNumber lsn = mock(LogSequenceNumber.class); + + when(connectionManager.getConnection()).thenReturn(connection); + when(connection.unwrap(PGConnection.class)).thenReturn(pgConnection); + when(pgConnection.getReplicationAPI().replicationStream().logical()).thenReturn(logicalStreamBuilder); + when(logicalStreamBuilder.withSlotName(anyString())).thenReturn(logicalStreamBuilder); + when(logicalStreamBuilder.withSlotOption(anyString(), anyString())).thenReturn(logicalStreamBuilder); + when(logicalStreamBuilder.start()).thenReturn(stream); + when(stream.readPending()).thenReturn(message).thenReturn(null); + when(stream.getLastReceiveLSN()).thenReturn(lsn); + + // First connect + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> logicalReplicationClient.connect()); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(eventProcessor, times(1)).process(message)); + Thread.sleep(20); + verify(stream).setAppliedLSN(lsn); + verify(stream).setFlushedLSN(lsn); + + // First disconnect + logicalReplicationClient.disconnect(); + Thread.sleep(20); + verify(stream).close(); + verifyNoMoreInteractions(stream, eventProcessor); + + // Second connect + when(stream.readPending()).thenReturn(message).thenReturn(null); + executorService.submit(() -> logicalReplicationClient.connect()); + await().atMost(Duration.ofSeconds(1)) + .untilAsserted(() -> verify(eventProcessor, times(2)).process(message)); + Thread.sleep(20); + verify(stream, times(2)).setAppliedLSN(lsn); + verify(stream, times(2)).setFlushedLSN(lsn); + + // Second disconnect + logicalReplicationClient.disconnect(); + Thread.sleep(20); + verify(stream, times(2)).close(); + verifyNoMoreInteractions(stream, eventProcessor); + + executorService.shutdownNow(); + } + private LogicalReplicationClient createObjectUnderTest() { return new LogicalReplicationClient(connectionManager, replicationSlotName, publicationName); } diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java index 31ec9618a2..90e8149319 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessorTest.java @@ -10,12 +10,15 @@ package org.opensearch.dataprepper.plugins.source.rds.stream; +import io.micrometer.core.instrument.Metrics; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Answers; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; @@ -28,9 +31,11 @@ import java.util.UUID; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) class LogicalReplicationEventProcessorTest { @@ -44,6 +49,18 @@ class LogicalReplicationEventProcessorTest { @Mock private Buffer> buffer; + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private LogicalReplicationClient logicalReplicationClient; + + @Mock + private StreamCheckpointer streamCheckpointer; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + private ByteBuffer message; private String s3Prefix; @@ -56,6 +73,8 @@ class LogicalReplicationEventProcessorTest { void setUp() { s3Prefix = UUID.randomUUID().toString(); random = new Random(); + when(pluginMetrics.timer(anyString())).thenReturn(Metrics.timer("test-timer")); + when(pluginMetrics.counter(anyString())).thenReturn(Metrics.counter("test-counter")); objectUnderTest = spy(createObjectUnderTest()); } @@ -129,8 +148,15 @@ void test_unsupported_message_type_throws_exception() { assertThrows(IllegalArgumentException.class, () -> objectUnderTest.process(message)); } + @Test + void test_stopClient() { + objectUnderTest.stopClient(); + verify(logicalReplicationClient).disconnect(); + } + private LogicalReplicationEventProcessor createObjectUnderTest() { - return new LogicalReplicationEventProcessor(streamPartition, sourceConfig, buffer, s3Prefix); + return new LogicalReplicationEventProcessor(streamPartition, sourceConfig, buffer, s3Prefix, pluginMetrics, + logicalReplicationClient, streamCheckpointer, acknowledgementSetManager); } private void setMessageType(MessageType messageType) { diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManagerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManagerTest.java index deddb45e32..1b32639daf 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManagerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointManagerTest.java @@ -11,10 +11,14 @@ import org.mockito.Mock; import org.mockito.MockedStatic; import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.postgresql.replication.LogSequenceNumber; import java.time.Duration; +import java.util.Random; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.function.Consumer; @@ -42,11 +46,16 @@ class StreamCheckpointManagerTest { @Mock private Runnable stopStreamRunnable; + @Mock + private PluginMetrics pluginMetrics; + private boolean isAcknowledgmentEnabled = false; + private EngineType engineType = EngineType.MYSQL; + private Random random; @BeforeEach void setUp() { - + random = new Random(); } @Test @@ -76,29 +85,65 @@ void test_shutdown() { } @Test - void test_saveChangeEventsStatus() { + void test_saveChangeEventsStatus_mysql() { final BinlogCoordinate binlogCoordinate = mock(BinlogCoordinate.class); + final long recordCount = random.nextLong(); + final StreamCheckpointManager streamCheckpointManager = createObjectUnderTest(); + + streamCheckpointManager.saveChangeEventsStatus(binlogCoordinate, recordCount); + + assertThat(streamCheckpointManager.getChangeEventStatuses().size(), is(1)); + final ChangeEventStatus changeEventStatus = streamCheckpointManager.getChangeEventStatuses().peek(); + assertThat(changeEventStatus.getBinlogCoordinate(), is(binlogCoordinate)); + assertThat(changeEventStatus.getRecordCount(), is(recordCount)); + } + + @Test + void test_saveChangeEventsStatus_postgres() { + final LogSequenceNumber logSequenceNumber = mock(LogSequenceNumber.class); + engineType = EngineType.POSTGRES; + final long recordCount = random.nextLong(); final StreamCheckpointManager streamCheckpointManager = createObjectUnderTest(); - streamCheckpointManager.saveChangeEventsStatus(binlogCoordinate); + + streamCheckpointManager.saveChangeEventsStatus(logSequenceNumber, recordCount); assertThat(streamCheckpointManager.getChangeEventStatuses().size(), is(1)); - assertThat(streamCheckpointManager.getChangeEventStatuses().peek().getBinlogCoordinate(), is(binlogCoordinate)); + final ChangeEventStatus changeEventStatus = streamCheckpointManager.getChangeEventStatuses().peek(); + assertThat(changeEventStatus.getLogSequenceNumber(), is(logSequenceNumber)); + assertThat(changeEventStatus.getRecordCount(), is(recordCount)); } @Test - void test_createAcknowledgmentSet() { + void test_createAcknowledgmentSet_mysql() { final BinlogCoordinate binlogCoordinate = mock(BinlogCoordinate.class); + final long recordCount = random.nextLong(); final StreamCheckpointManager streamCheckpointManager = createObjectUnderTest(); - streamCheckpointManager.createAcknowledgmentSet(binlogCoordinate); + streamCheckpointManager.createAcknowledgmentSet(binlogCoordinate, recordCount); assertThat(streamCheckpointManager.getChangeEventStatuses().size(), is(1)); ChangeEventStatus changeEventStatus = streamCheckpointManager.getChangeEventStatuses().peek(); assertThat(changeEventStatus.getBinlogCoordinate(), is(binlogCoordinate)); + assertThat(changeEventStatus.getRecordCount(), is(recordCount)); + verify(acknowledgementSetManager).create(any(Consumer.class), eq(ACK_TIMEOUT)); + } + + @Test + void test_createAcknowledgmentSet_postgres() { + final LogSequenceNumber logSequenceNumber = mock(LogSequenceNumber.class); + engineType = EngineType.POSTGRES; + final long recordCount = random.nextLong(); + final StreamCheckpointManager streamCheckpointManager = createObjectUnderTest(); + streamCheckpointManager.createAcknowledgmentSet(logSequenceNumber, recordCount); + + assertThat(streamCheckpointManager.getChangeEventStatuses().size(), is(1)); + ChangeEventStatus changeEventStatus = streamCheckpointManager.getChangeEventStatuses().peek(); + assertThat(changeEventStatus.getLogSequenceNumber(), is(logSequenceNumber)); + assertThat(changeEventStatus.getRecordCount(), is(recordCount)); verify(acknowledgementSetManager).create(any(Consumer.class), eq(ACK_TIMEOUT)); } private StreamCheckpointManager createObjectUnderTest() { return new StreamCheckpointManager( - streamCheckpointer, isAcknowledgmentEnabled, acknowledgementSetManager, stopStreamRunnable, ACK_TIMEOUT); + streamCheckpointer, isAcknowledgmentEnabled, acknowledgementSetManager, stopStreamRunnable, ACK_TIMEOUT, engineType, pluginMetrics); } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java index 3327e847f5..75f16ac9fc 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamCheckpointerTest.java @@ -13,10 +13,13 @@ import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.StreamPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.MySqlStreamState; +import org.opensearch.dataprepper.plugins.source.rds.coordination.state.PostgresStreamState; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.StreamProgressState; import org.opensearch.dataprepper.plugins.source.rds.model.BinlogCoordinate; +import org.postgresql.replication.LogSequenceNumber; import java.util.Optional; @@ -39,12 +42,18 @@ class StreamCheckpointerTest { @Mock private MySqlStreamState mySqlStreamState; + @Mock + private PostgresStreamState postgresStreamState; + @Mock private PluginMetrics pluginMetrics; @Mock private Counter checkpointCounter; + @Mock + private ChangeEventStatus changeEventStatus; + private StreamCheckpointer streamCheckpointer; @@ -55,19 +64,35 @@ void setUp() { } @Test - void test_checkpoint() { + void test_checkpoint_mysql() { final BinlogCoordinate binlogCoordinate = mock(BinlogCoordinate.class); final StreamProgressState streamProgressState = mock(StreamProgressState.class); when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); when(streamProgressState.getMySqlStreamState()).thenReturn(mySqlStreamState); + when(changeEventStatus.getBinlogCoordinate()).thenReturn(binlogCoordinate); - streamCheckpointer.checkpoint(binlogCoordinate); + streamCheckpointer.checkpoint(EngineType.MYSQL, changeEventStatus); verify(mySqlStreamState).setCurrentPosition(binlogCoordinate); verify(sourceCoordinator).saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); verify(checkpointCounter).increment(); } + @Test + void test_checkpoint_postgres() { + final LogSequenceNumber logSequenceNumber = mock(LogSequenceNumber.class); + final StreamProgressState streamProgressState = mock(StreamProgressState.class); + when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); + when(streamProgressState.getPostgresStreamState()).thenReturn(postgresStreamState); + when(changeEventStatus.getLogSequenceNumber()).thenReturn(logSequenceNumber); + + streamCheckpointer.checkpoint(EngineType.POSTGRES, changeEventStatus); + + verify(postgresStreamState).setCurrentLsn(logSequenceNumber.asString()); + verify(sourceCoordinator).saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + verify(checkpointCounter).increment(); + } + @Test void test_extendLease() { streamCheckpointer.extendLease(); From 81c1c844cc6c99beeb6cd16f180e0e5fde7e5f12 Mon Sep 17 00:00:00 2001 From: Krishna Kondaka Date: Wed, 5 Feb 2025 15:28:47 -0800 Subject: [PATCH 11/18] Add cloudwatch logs sink (#5406) * Add cloudwatch logs sink Signed-off-by: Krishna Kondaka * Addressed review comments Signed-off-by: Krishna Kondaka --------- Signed-off-by: Krishna Kondaka --- .../cloudwatch-logs/build.gradle | 38 ++- .../cloudwatch_logs/CouldWatchLogsIT.java | 292 ++++++++++++++++++ .../cloudwatch_logs/CloudWatchLogsSink.java | 6 +- .../client/CloudWatchLogsService.java | 13 + settings.gradle | 2 +- 5 files changed, 344 insertions(+), 7 deletions(-) create mode 100644 data-prepper-plugins/cloudwatch-logs/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/CouldWatchLogsIT.java diff --git a/data-prepper-plugins/cloudwatch-logs/build.gradle b/data-prepper-plugins/cloudwatch-logs/build.gradle index 3bbb24f443..348275f298 100644 --- a/data-prepper-plugins/cloudwatch-logs/build.gradle +++ b/data-prepper-plugins/cloudwatch-logs/build.gradle @@ -1,8 +1,28 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + plugins { id 'java' id 'java-library' } +sourceSets { + integrationTest { + java { + compileClasspath += main.output + test.output + runtimeClasspath += main.output + test.output + srcDir file('src/integrationTest/java') + } + } +} + +configurations { + integrationTestImplementation.extendsFrom testImplementation + integrationTestRuntime.extendsFrom testRuntime +} + dependencies { implementation project(':data-prepper-plugins:aws-plugin-api') implementation project(path: ':data-prepper-plugins:common') @@ -33,4 +53,20 @@ jacocoTestCoverageVerification { test { useJUnitPlatform() -} \ No newline at end of file +} + +task integrationTest(type: Test) { + group = 'verification' + testClassesDirs = sourceSets.integrationTest.output.classesDirs + + useJUnitPlatform() + + classpath = sourceSets.integrationTest.runtimeClasspath + systemProperty 'tests.cloudwatch.log_group', System.getProperty('tests.cloudwatch.log_group') + systemProperty 'tests.cloudwatch.log_stream', System.getProperty('tests.cloudwatch.log_stream') + systemProperty 'tests.aws.region', System.getProperty('tests.aws.region') + systemProperty 'tests.aws.role', System.getProperty('tests.aws.role') + filter { + includeTestsMatching '*IT' + } +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/CouldWatchLogsIT.java b/data-prepper-plugins/cloudwatch-logs/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/CouldWatchLogsIT.java new file mode 100644 index 0000000000..5158efb18e --- /dev/null +++ b/data-prepper-plugins/cloudwatch-logs/src/integrationTest/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/CouldWatchLogsIT.java @@ -0,0 +1,292 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.sink.cloudwatch_logs; + +import io.micrometer.core.instrument.Counter; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.plugins.sink.cloudwatch_logs.config.ThresholdConfig; +import org.opensearch.dataprepper.plugins.sink.cloudwatch_logs.config.AwsConfig; +import org.opensearch.dataprepper.plugins.sink.cloudwatch_logs.config.CloudWatchLogsSinkConfig; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.sink.cloudwatch_logs.client.CloudWatchLogsClientFactory; +import software.amazon.awssdk.services.cloudwatchlogs.model.GetLogEventsResponse; +import software.amazon.awssdk.services.cloudwatchlogs.model.GetLogEventsRequest; +import software.amazon.awssdk.services.cloudwatchlogs.model.OutputLogEvent; +import software.amazon.awssdk.services.cloudwatchlogs.model.CreateLogStreamRequest; +import software.amazon.awssdk.services.cloudwatchlogs.model.CreateLogStreamResponse; +import software.amazon.awssdk.services.cloudwatchlogs.model.DeleteLogStreamRequest; +import software.amazon.awssdk.services.cloudwatchlogs.CloudWatchLogsClient; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.log.JacksonLog; +import org.opensearch.dataprepper.model.record.Record; +import software.amazon.awssdk.regions.Region; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import static org.awaitility.Awaitility.await; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.lenient; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Map; +import java.util.List; +import java.util.Collection; +import java.util.Collections; +import java.util.concurrent.atomic.AtomicInteger; + +@ExtendWith(MockitoExtension.class) +public class CouldWatchLogsIT { + static final int NUM_RECORDS = 2; + @Mock + private PluginSetting pluginSetting; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + + @Mock + private AwsConfig awsConfig; + + @Mock + private ThresholdConfig thresholdConfig; + + @Mock + private CloudWatchLogsSinkConfig cloudWatchLogsSinkConfig; + + @Mock + private Counter counter; + + private String awsRegion; + private String awsRole; + private String logGroupName; + private String logStreamName; + private CloudWatchLogsSink sink; + private AtomicInteger count; + private CloudWatchLogsClient cloudWatchLogsClient; + private ObjectMapper objectMapper; + + @BeforeEach + void setUp() { + count = new AtomicInteger(0); + objectMapper = new ObjectMapper(); + pluginSetting = mock(PluginSetting.class); + when(pluginSetting.getPipelineName()).thenReturn("pipeline"); + when(pluginSetting.getName()).thenReturn("name"); + awsRegion = System.getProperty("tests.aws.region"); + awsRole = System.getProperty("tests.aws.role"); + awsConfig = mock(AwsConfig.class); + when(awsConfig.getAwsRegion()).thenReturn(Region.of(awsRegion)); + when(awsConfig.getAwsStsRoleArn()).thenReturn(awsRole); + when(awsConfig.getAwsStsExternalId()).thenReturn(null); + when(awsConfig.getAwsStsHeaderOverrides()).thenReturn(null); + when(awsCredentialsSupplier.getProvider(any())).thenAnswer(options -> DefaultCredentialsProvider.create()); + cloudWatchLogsClient = CloudWatchLogsClientFactory.createCwlClient(awsConfig, awsCredentialsSupplier); + logGroupName = System.getProperty("tests.cloudwatch.log_group"); + logStreamName = createLogStream(logGroupName); + pluginMetrics = mock(PluginMetrics.class); + counter = mock(Counter.class); + lenient().doAnswer((a)-> { + int v = (int)(double)(a.getArgument(0)); + count.addAndGet(v); + return null; + }).when(counter).increment(any(Double.class)); + lenient().doAnswer((a)-> { + count.addAndGet(1); + return null; + }).when(counter).increment(); + when(pluginMetrics.counter(anyString())).thenReturn(counter); + cloudWatchLogsSinkConfig = mock(CloudWatchLogsSinkConfig.class); + when(cloudWatchLogsSinkConfig.getLogGroup()).thenReturn(logGroupName); + when(cloudWatchLogsSinkConfig.getLogStream()).thenReturn(logStreamName); + when(cloudWatchLogsSinkConfig.getAwsConfig()).thenReturn(awsConfig); + when(cloudWatchLogsSinkConfig.getBufferType()).thenReturn(CloudWatchLogsSinkConfig.DEFAULT_BUFFER_TYPE); + + thresholdConfig = mock(ThresholdConfig.class); + when(thresholdConfig.getBackOffTime()).thenReturn(500L); + when(thresholdConfig.getLogSendInterval()).thenReturn(60L); + when(thresholdConfig.getRetryCount()).thenReturn(10); + when(thresholdConfig.getMaxEventSizeBytes()).thenReturn(1000L); + when(cloudWatchLogsSinkConfig.getThresholdConfig()).thenReturn(thresholdConfig); + } + + @AfterEach + void tearDown() { + DeleteLogStreamRequest deleteRequest = DeleteLogStreamRequest + .builder() + .logGroupName(logGroupName) + .logStreamName(logStreamName) + .build(); + cloudWatchLogsClient.deleteLogStream(deleteRequest); + } + + private CloudWatchLogsSink createObjectUnderTest() { + return new CloudWatchLogsSink(pluginSetting, pluginMetrics, cloudWatchLogsSinkConfig, awsCredentialsSupplier); + } + + private String createLogStream(final String logGroupName) { + final String newLogStreamName = "CouldWatchLogsIT_"+RandomStringUtils.randomAlphabetic(6); + CreateLogStreamRequest createRequest = CreateLogStreamRequest + .builder() + .logGroupName(logGroupName) + .logStreamName(newLogStreamName) + .build(); + CreateLogStreamResponse response = cloudWatchLogsClient.createLogStream(createRequest); + return newLogStreamName; + + } + + @Test + void TestSinkOperationWithLogSendInterval() throws Exception { + long startTime = Instant.now().toEpochMilli(); + when(thresholdConfig.getBatchSize()).thenReturn(10); + when(thresholdConfig.getLogSendInterval()).thenReturn(10L); + when(thresholdConfig.getMaxRequestSizeBytes()).thenReturn(1000L); + + sink = createObjectUnderTest(); + Collection> records = getRecordList(NUM_RECORDS); + sink.doOutput(records); + await().atMost(Duration.ofSeconds(30)) + .untilAsserted(() -> { + sink.doOutput(Collections.emptyList()); + long endTime = Instant.now().toEpochMilli(); + GetLogEventsRequest getRequest = GetLogEventsRequest + .builder() + .logGroupName(logGroupName) + .logStreamName(logStreamName) + .startTime(startTime) + .endTime(endTime) + .build(); + GetLogEventsResponse response = cloudWatchLogsClient.getLogEvents(getRequest); + List events = response.events(); + assertThat(events.size(), equalTo(NUM_RECORDS)); + for (int i = 0; i < events.size(); i++) { + String message = events.get(i).message(); + Map event = objectMapper.readValue(message, Map.class); + assertThat(event.get("name"), equalTo("Person"+i)); + assertThat(event.get("age"), equalTo(Integer.toString(i))); + } + }); + // NUM_RECORDS success + // 1 request success + assertThat(count.get(), equalTo(NUM_RECORDS+1)); + + } + + @Test + void TestSinkOperationWithBatchSize() throws Exception { + long startTime = Instant.now().toEpochMilli(); + when(thresholdConfig.getBatchSize()).thenReturn(1); + when(thresholdConfig.getMaxEventSizeBytes()).thenReturn(1000L); + when(thresholdConfig.getMaxRequestSizeBytes()).thenReturn(1000L); + + sink = createObjectUnderTest(); + Collection> records = getRecordList(NUM_RECORDS); + sink.doOutput(records); + await().atMost(Duration.ofSeconds(30)) + .untilAsserted(() -> { + long endTime = Instant.now().toEpochMilli(); + GetLogEventsRequest getRequest = GetLogEventsRequest + .builder() + .logGroupName(logGroupName) + .logStreamName(logStreamName) + .startTime(startTime) + .endTime(endTime) + .build(); + GetLogEventsResponse response = cloudWatchLogsClient.getLogEvents(getRequest); + List events = response.events(); + assertThat(events.size(), equalTo(NUM_RECORDS)); + for (int i = 0; i < events.size(); i++) { + String message = events.get(i).message(); + Map event = objectMapper.readValue(message, Map.class); + assertThat(event.get("name"), equalTo("Person"+i)); + assertThat(event.get("age"), equalTo(Integer.toString(i))); + } + }); + // NUM_RECORDS success + // NUM_RECORDS request success + assertThat(count.get(), equalTo(NUM_RECORDS*2)); + + } + + @Test + void TestSinkOperationWithMaxRequestSize() throws Exception { + long startTime = Instant.now().toEpochMilli(); + when(thresholdConfig.getBatchSize()).thenReturn(20); + when(thresholdConfig.getMaxRequestSizeBytes()).thenReturn(108L); + + sink = createObjectUnderTest(); + Collection> records = getRecordList(NUM_RECORDS); + sink.doOutput(records); + await().atMost(Duration.ofSeconds(30)) + .untilAsserted(() -> { + long endTime = Instant.now().toEpochMilli(); + GetLogEventsRequest getRequest = GetLogEventsRequest + .builder() + .logGroupName(logGroupName) + .logStreamName(logStreamName) + .startTime(startTime) + .endTime(endTime) + .build(); + GetLogEventsResponse response = cloudWatchLogsClient.getLogEvents(getRequest); + List events = response.events(); + assertThat(events.size(), equalTo(NUM_RECORDS)); + for (int i = 0; i < events.size(); i++) { + String message = events.get(i).message(); + Map event = objectMapper.readValue(message, Map.class); + assertThat(event.get("name"), equalTo("Person"+i)); + assertThat(event.get("age"), equalTo(Integer.toString(i))); + } + }); + // NUM_RECORDS success + // 1 request success + assertThat(count.get(), equalTo(NUM_RECORDS+1)); + + } + + private Collection> getRecordList(int numberOfRecords) { + final Collection> recordList = new ArrayList<>(); + List records = generateRecords(numberOfRecords); + for (int i = 0; i < numberOfRecords; i++) { + final Event event = JacksonLog.builder().withData(records.get(i)).build(); + recordList.add(new Record<>(event)); + } + return recordList; + } + + private static List generateRecords(int numberOfRecords) { + + List recordList = new ArrayList<>(); + + for (int rows = 0; rows < numberOfRecords; rows++) { + + HashMap eventData = new HashMap<>(); + + eventData.put("name", "Person" + rows); + eventData.put("age", Integer.toString(rows)); + recordList.add((eventData)); + + } + return recordList; + } +} 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 f17d79c7af..ddc5654d27 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 @@ -87,10 +87,6 @@ public void doInitialize() { @Override public void doOutput(Collection> records) { - if (records.isEmpty()) { - return; - } - cloudWatchLogsService.processLogEvents(records); } @@ -98,4 +94,4 @@ public void doOutput(Collection> records) { public boolean isReady() { return isInitialized; } -} \ No newline at end of file +} diff --git a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/client/CloudWatchLogsService.java b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/client/CloudWatchLogsService.java index fc9963ab46..af54d19267 100644 --- a/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/client/CloudWatchLogsService.java +++ b/data-prepper-plugins/cloudwatch-logs/src/main/java/org/opensearch/dataprepper/plugins/sink/cloudwatch_logs/client/CloudWatchLogsService.java @@ -60,6 +60,19 @@ public CloudWatchLogsService(final Buffer buffer, */ public void processLogEvents(final Collection> logs) { sinkStopWatch.startIfNotRunning(); + if (logs.isEmpty() && buffer.getEventCount() > 0) { + processLock.lock(); + try { + if (cloudWatchLogsLimits.isGreaterThanLimitReached(sinkStopWatch.getElapsedTimeInSeconds(), + buffer.getBufferSize(), buffer.getEventCount())) { + stageLogEvents(); + } + } finally { + processLock.unlock(); + } + return; + } + for (Record log : logs) { String logString = log.getData().toJsonString(); int logLength = logString.length(); diff --git a/settings.gradle b/settings.gradle index d2aa09b52c..37a125aaa5 100644 --- a/settings.gradle +++ b/settings.gradle @@ -168,7 +168,7 @@ include 'data-prepper-plugins:aws-sqs-common' include 'data-prepper-plugins:buffer-common' include 'data-prepper-plugins:sqs-source' include 'data-prepper-plugins:sqs-common' -//include 'data-prepper-plugins:cloudwatch-logs' +include 'data-prepper-plugins:cloudwatch-logs' //include 'data-prepper-plugins:http-sink' //include 'data-prepper-plugins:sns-sink' //include 'data-prepper-plugins:prometheus-sink' From 782793ad225fdb716e96dba648f8ccf90203c78e Mon Sep 17 00:00:00 2001 From: David Venable Date: Wed, 5 Feb 2025 15:33:01 -0800 Subject: [PATCH 12/18] Updates the Dockerfile to use non-legacy syntax for environment variables. (#5417) Signed-off-by: David Venable --- release/docker/Dockerfile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/release/docker/Dockerfile b/release/docker/Dockerfile index 4052d59909..04d040e353 100644 --- a/release/docker/Dockerfile +++ b/release/docker/Dockerfile @@ -5,7 +5,7 @@ ARG CONFIG_FILEPATH ARG ARCHIVE_FILE ARG ARCHIVE_FILE_UNPACKED -ENV DATA_PREPPER_PATH /usr/share/data-prepper +ENV DATA_PREPPER_PATH=/usr/share/data-prepper ENV ENV_CONFIG_FILEPATH=$CONFIG_FILEPATH ENV ENV_PIPELINE_FILEPATH=$PIPELINE_FILEPATH From ae65262396bcbb8612da4f2e4a720d41ab2458df Mon Sep 17 00:00:00 2001 From: David Venable Date: Thu, 6 Feb 2025 06:43:35 -0800 Subject: [PATCH 13/18] Moves asifsmohammed to the Emeritus section. We previously removed him from the CODEOWNERS, so this keeps these in sync. (#5419) Signed-off-by: David Venable --- MAINTAINERS.md | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/MAINTAINERS.md b/MAINTAINERS.md index 85b7e816cf..0bc07143a7 100644 --- a/MAINTAINERS.md +++ b/MAINTAINERS.md @@ -14,7 +14,6 @@ This document contains a list of maintainers in this repo. See [opensearch-proje | Taylor Gray | [graytaylor0](https://github.com/graytaylor0) | Amazon | | Dinu John | [dinujoh](https://github.com/dinujoh) | Amazon | | Krishna Kondaka | [kkondaka](https://github.com/kkondaka) | Amazon | -| Asif Sohail Mohammed | [asifsmohammed](https://github.com/asifsmohammed) | Amazon | | Karsten Schnitter | [KarstenSchnitter](https://github.com/KarstenSchnitter) | SAP | | David Venable | [dlvenable](https://github.com/dlvenable) | Amazon | | Hai Yan | [oeyh](https://github.com/oeyh) | Amazon | @@ -22,10 +21,11 @@ This document contains a list of maintainers in this repo. See [opensearch-proje ## Emeritus -| Maintainer | GitHub ID | Affiliation | -| -------------------- | ----------------------------------------------------- | ----------- | -| Steven Bayer | [sbayer55](https://github.com/sbayer55) | Amazon | -| Christopher Manning | [cmanning09](https://github.com/cmanning09) | Amazon | -| David Powers | [dapowers87](https://github.com/dapowers87) | Amazon | -| Shivani Shukla | [sshivanii](https://github.com/sshivanii) | Amazon | -| Phill Treddenick | [treddeni-amazon](https://github.com/treddeni-amazon) | Amazon | +| Maintainer | GitHub ID | Affiliation | +| ---------------------- | ----------------------------------------------------- | ----------- | +| Steven Bayer | [sbayer55](https://github.com/sbayer55) | Amazon | +| Christopher Manning | [cmanning09](https://github.com/cmanning09) | Amazon | +| Asif Sohail Mohammed | [asifsmohammed](https://github.com/asifsmohammed) | Amazon | +| David Powers | [dapowers87](https://github.com/dapowers87) | Amazon | +| Shivani Shukla | [sshivanii](https://github.com/sshivanii) | Amazon | +| Phill Treddenick | [treddeni-amazon](https://github.com/treddeni-amazon) | Amazon | From 137e1e7be851a5ea773fa4b5b9953a29f6d0fa5e Mon Sep 17 00:00:00 2001 From: Jeremy Michael <60355474+jmsusanto@users.noreply.github.com> Date: Thu, 6 Feb 2025 10:42:34 -0800 Subject: [PATCH 14/18] SQS Source: Add on_error Config, Multi-Region Support, and sqsMessageDelayTimer Metric for Auto-Scaling (#5409) Signed-off-by: Jeremy Michael --- .../source/sqs/common/OnErrorOption.java | 39 ++ .../source/sqs/common/SqsWorkerCommon.java | 15 +- .../sqs/common/SqsWorkerCommonTest.java | 234 ++++++--- data-prepper-plugins/sqs-source/README.md | 60 ++- .../plugins/source/sqs/QueueConfig.java | 12 +- .../plugins/source/sqs/SqsService.java | 30 +- .../plugins/source/sqs/SqsWorker.java | 65 ++- .../source/sqs/AttributeHandlerTest.java | 70 +++ .../plugins/source/sqs/SqsServiceTest.java | 10 +- .../plugins/source/sqs/SqsWorkerTest.java | 449 +++++++++--------- 10 files changed, 643 insertions(+), 341 deletions(-) create mode 100644 data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/OnErrorOption.java create mode 100644 data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AttributeHandlerTest.java diff --git a/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/OnErrorOption.java b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/OnErrorOption.java new file mode 100644 index 0000000000..f3244f9014 --- /dev/null +++ b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/OnErrorOption.java @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.sqs.common; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +public enum OnErrorOption { + DELETE_MESSAGES("delete_messages"), + RETAIN_MESSAGES("retain_messages"); + + private static final Map OPTIONS_MAP = Arrays.stream(OnErrorOption.values()) + .collect(Collectors.toMap( + value -> value.option, + value -> value + )); + + private final String option; + + OnErrorOption(final String option) { + this.option = option; + } + + @JsonCreator + static OnErrorOption fromOptionValue(final String option) { + return OPTIONS_MAP.get(option); + } +} diff --git a/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommon.java b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommon.java index 9301574237..bda8318928 100644 --- a/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommon.java +++ b/data-prepper-plugins/sqs-common/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/common/SqsWorkerCommon.java @@ -42,7 +42,6 @@ public class SqsWorkerCommon { public static final String SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME = "sqsVisibilityTimeoutChangedCount"; public static final String SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME = "sqsVisibilityTimeoutChangeFailedCount"; - private final SqsClient sqsClient; private final Backoff standardBackoff; private final PluginMetrics pluginMetrics; private final AcknowledgementSetManager acknowledgementSetManager; @@ -56,18 +55,17 @@ public class SqsWorkerCommon { private final Counter sqsVisibilityTimeoutChangedCount; private final Counter sqsVisibilityTimeoutChangeFailedCount; - public SqsWorkerCommon(final SqsClient sqsClient, - final Backoff standardBackoff, + public SqsWorkerCommon(final Backoff standardBackoff, final PluginMetrics pluginMetrics, final AcknowledgementSetManager acknowledgementSetManager) { - this.sqsClient = sqsClient; this.standardBackoff = standardBackoff; this.pluginMetrics = pluginMetrics; this.acknowledgementSetManager = acknowledgementSetManager; this.isStopped = false; this.failedAttemptCount = 0; + sqsMessagesReceivedCounter = pluginMetrics.counter(SQS_MESSAGES_RECEIVED_METRIC_NAME); sqsMessagesDeletedCounter = pluginMetrics.counter(SQS_MESSAGES_DELETED_METRIC_NAME); sqsMessagesFailedCounter = pluginMetrics.counter(SQS_MESSAGES_FAILED_METRIC_NAME); @@ -78,6 +76,7 @@ public SqsWorkerCommon(final SqsClient sqsClient, } public List pollSqsMessages(final String queueUrl, + final SqsClient sqsClient, final Integer maxNumberOfMessages, final Duration waitTime, final Duration visibilityTimeout) { @@ -134,7 +133,7 @@ public void applyBackoff() { } } - public void deleteSqsMessages(final String queueUrl, final List entries) { + public void deleteSqsMessages(final String queueUrl, final SqsClient sqsClient, final List entries) { if (entries == null || entries.isEmpty() || isStopped) { return; } @@ -146,7 +145,6 @@ public void deleteSqsMessages(final String queueUrl, final List messages = Collections.singletonList(message); + ReceiveMessageResponse response = ReceiveMessageResponse.builder().messages(messages).build(); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(response); + List result = sqsWorkerCommon.pollSqsMessages(queueUrl, sqsClient, 10, + Duration.ofSeconds(5), Duration.ofSeconds(30)); + + verify(sqsMessagesReceivedCounter).increment(messages.size()); + assertThat(result, equalTo(messages)); } @Test - void testPollSqsMessages_handlesEmptyList() { + void testPollSqsMessages_exception() { when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))) - .thenReturn(ReceiveMessageResponse.builder() - .messages(Collections.emptyList()) - .build()); - var messages = sqsWorkerCommon.pollSqsMessages( - "testQueueUrl", - 10, - Duration.ofSeconds(5), - Duration.ofSeconds(30) - ); - - assertNotNull(messages); - assertTrue(messages.isEmpty()); - Mockito.verify(sqsClient).receiveMessage(any(ReceiveMessageRequest.class)); - Mockito.verify(backoff, Mockito.never()).nextDelayMillis(Mockito.anyInt()); + .thenThrow(SqsException.builder().message("Error").build()); + when(backoff.nextDelayMillis(anyInt())).thenReturn(100L); + List result = sqsWorkerCommon.pollSqsMessages(queueUrl, sqsClient, 10, + Duration.ofSeconds(5), Duration.ofSeconds(30)); + assertThat(result, is(empty())); + verify(sqsMessagesReceivedCounter, never()).increment(anyDouble()); } @Test - void testDeleteSqsMessages_callsClientWhenNotStopped() { - var entries = Collections.singletonList( - DeleteMessageBatchRequestEntry.builder() - .id("msg-id") - .receiptHandle("receipt-handle") - .build() - ); + void testApplyBackoff_negativeDelayThrowsException() { + when(backoff.nextDelayMillis(anyInt())).thenReturn(-1L); + assertThrows(SqsRetriesExhaustedException.class, () -> sqsWorkerCommon.applyBackoff()); + } - when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))) - .thenReturn(DeleteMessageBatchResponse.builder().build()); - - sqsWorkerCommon.deleteSqsMessages("testQueueUrl", entries); - ArgumentCaptor captor = - ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); - Mockito.verify(sqsClient).deleteMessageBatch(captor.capture()); - assertEquals("testQueueUrl", captor.getValue().queueUrl()); - assertEquals(1, captor.getValue().entries().size()); + @Test + void testApplyBackoff_positiveDelay() { + when(backoff.nextDelayMillis(anyInt())).thenReturn(50L); + assertDoesNotThrow(() -> sqsWorkerCommon.applyBackoff()); + verify(backoff).nextDelayMillis(anyInt()); } @Test - void testStop_skipsFurtherOperations() { + void testDeleteSqsMessages_noEntries() { + sqsWorkerCommon.deleteSqsMessages(queueUrl, sqsClient, null); + verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); + + sqsWorkerCommon.deleteSqsMessages(queueUrl, sqsClient, Collections.emptyList()); + verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); + sqsWorkerCommon.stop(); - sqsWorkerCommon.deleteSqsMessages("testQueueUrl", Collections.singletonList( - DeleteMessageBatchRequestEntry.builder() - .id("msg-id") - .receiptHandle("receipt-handle") - .build() - )); - Mockito.verify(sqsClient, Mockito.never()).deleteMessageBatch((DeleteMessageBatchRequest) any()); + DeleteMessageBatchRequestEntry entry = DeleteMessageBatchRequestEntry.builder() + .id("id").receiptHandle("rh").build(); + sqsWorkerCommon.deleteSqsMessages(queueUrl, sqsClient, Collections.singletonList(entry)); + verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); + } + + @Test + void testDeleteSqsMessages_successfulDeletion_withConsumerBuilder() { + DeleteMessageBatchRequestEntry entry = DeleteMessageBatchRequestEntry.builder() + .id("id") + .receiptHandle("rh") + .build(); + List entries = Collections.singletonList(entry); + DeleteMessageBatchResponse response = DeleteMessageBatchResponse.builder() + .successful(builder -> builder.id("id")) + .failed(Collections.emptyList()) + .build(); + when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenReturn(response); + sqsWorkerCommon.deleteSqsMessages(queueUrl, sqsClient, entries); + verify(sqsMessagesDeletedCounter).increment(1.0); + } + + + @Test + void testDeleteSqsMessages_failedDeletion() { + DeleteMessageBatchRequestEntry entry = DeleteMessageBatchRequestEntry.builder() + .id("id").receiptHandle("rh").build(); + List entries = Collections.singletonList(entry); + + DeleteMessageBatchResponse response = DeleteMessageBatchResponse.builder() + .successful(Collections.emptyList()) + .failed(Collections.singletonList( + BatchResultErrorEntry.builder().id("id").message("Failure").build())) + .build(); + + when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenReturn(response); + sqsWorkerCommon.deleteSqsMessages(queueUrl, sqsClient, entries); + verify(sqsMessagesDeleteFailedCounter).increment(1.0); + } + + @Test + void testDeleteSqsMessages_sdkException() { + DeleteMessageBatchRequestEntry entry1 = DeleteMessageBatchRequestEntry.builder() + .id("id-1").receiptHandle("rh-1").build(); + DeleteMessageBatchRequestEntry entry2 = DeleteMessageBatchRequestEntry.builder() + .id("id-2").receiptHandle("rh-2").build(); + List entries = Arrays.asList(entry1, entry2); + + when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))) + .thenThrow(SdkException.class); + + sqsWorkerCommon.deleteSqsMessages(queueUrl, sqsClient, entries); + verify(sqsMessagesDeleteFailedCounter).increment(entries.size()); + } + + @Test + void testIncreaseVisibilityTimeout_successful() { + String receiptHandle = "rh"; + int newVisibilityTimeout = 45; + String messageId = "msg"; + when(sqsClient.changeMessageVisibility(any(ChangeMessageVisibilityRequest.class))) + .thenReturn(ChangeMessageVisibilityResponse.builder().build()); + + sqsWorkerCommon.increaseVisibilityTimeout(queueUrl, sqsClient, receiptHandle, + newVisibilityTimeout, messageId); + + ArgumentCaptor requestCaptor = + ArgumentCaptor.forClass(ChangeMessageVisibilityRequest.class); + verify(sqsClient).changeMessageVisibility(requestCaptor.capture()); + ChangeMessageVisibilityRequest request = requestCaptor.getValue(); + assertThat(request.queueUrl(), equalTo(queueUrl)); + assertThat(request.receiptHandle(), equalTo(receiptHandle)); + assertThat(request.visibilityTimeout(), equalTo(newVisibilityTimeout)); + verify(sqsVisibilityTimeoutChangedCount).increment(); + } + + @Test + void testIncreaseVisibilityTimeout_whenException() { + String receiptHandle = "rh"; + int newVisibilityTimeout = 45; + String messageId = "msg"; + doThrow(new RuntimeException("failure")) + .when(sqsClient).changeMessageVisibility(any(ChangeMessageVisibilityRequest.class)); + + sqsWorkerCommon.increaseVisibilityTimeout(queueUrl, sqsClient, receiptHandle, + newVisibilityTimeout, messageId); + + verify(sqsVisibilityTimeoutChangeFailedCount).increment(); } } diff --git a/data-prepper-plugins/sqs-source/README.md b/data-prepper-plugins/sqs-source/README.md index ff4313605f..c3898b396d 100644 --- a/data-prepper-plugins/sqs-source/README.md +++ b/data-prepper-plugins/sqs-source/README.md @@ -2,7 +2,7 @@ This source allows Data Prepper to use SQS as a source. It reads messages from specified SQS queues and processes them into events. -## Example Configuration +## Minimal Configuration ```yaml sqs-pipeline: @@ -10,13 +10,59 @@ sqs-pipeline: sqs: queues: - url: - batch_size: 10 - workers: 1 + aws: + region: + sts_role_arn: + ``` +## Full Configuration + +```yaml +sqs-pipeline: + source: + sqs: + queues: + - url: + workers: 2 + maximum_messages: 10 + poll_delay: 0s + wait_time: 20s + visibility_timeout: 30s + visibility_duplication_protection: true + visibility_duplicate_protection_timeout: "PT1H" + on_error: "retain_messages" + codec: + json: + key_name: "events" - url: - batch_size: 10 - workers: 1 + # This queue will use the defaults for optional properties. + acknowledgments: true aws: region: sts_role_arn: - sink: - - stdout: +``` +## Key Features + +- **Multi-Queue Support:** + Process messages from multiple SQS queues simultaneously. + + +- **Configurable Polling:** + Customize batch size, poll delay, wait time, and visibility timeout per queue. + + +- **Error Handling:** + Use an `on_error` option to control behavior on errors (e.g., delete or retain messages) + + +- **Codec Support:** + Configure codecs (e.g., JSON, CSV, newline-delimited) to parse incoming messages. + + +## IAM Permissions + +Ensure that the SQS queues have the following AWS permissions +- `sqs:ReceiveMessage` + +- `sqs:DeleteMessageBatch` + +- `sqs:ChangeMessageVisibility` diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfig.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfig.java index 47e417bc27..d0715ee8c1 100644 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfig.java +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/QueueConfig.java @@ -17,7 +17,7 @@ import jakarta.validation.constraints.Min; import jakarta.validation.constraints.NotNull; import java.time.Duration; - +import org.opensearch.dataprepper.plugins.source.sqs.common.OnErrorOption; import org.hibernate.validator.constraints.time.DurationMax; import org.hibernate.validator.constraints.time.DurationMin; import org.opensearch.dataprepper.model.configuration.PluginModel; @@ -71,6 +71,9 @@ public class QueueConfig { @JsonProperty("codec") private PluginModel codec = null; + @JsonProperty("on_error") + private OnErrorOption onErrorOption = OnErrorOption.RETAIN_MESSAGES; + public String getUrl() { return url; } @@ -83,9 +86,7 @@ public int getNumWorkers() { return numWorkers; } - public Duration getVisibilityTimeout() { - return visibilityTimeout; - } + public Duration getVisibilityTimeout() {return visibilityTimeout; } public boolean getVisibilityDuplicateProtection() { return visibilityDuplicateProtection; @@ -107,5 +108,8 @@ public PluginModel getCodec() { return codec; } + public OnErrorOption getOnErrorOption() { + return onErrorOption; + } } diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java index 5c3e3ba2d2..3c48b76f4e 100644 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsService.java @@ -20,16 +20,20 @@ import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.plugins.source.sqs.common.SqsBackoff; import org.opensearch.dataprepper.plugins.source.sqs.common.SqsClientFactory; + import org.opensearch.dataprepper.plugins.source.sqs.common.SqsWorkerCommon; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; + import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.sqs.SqsClient; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; import java.util.ArrayList; + import java.util.HashMap; import java.util.List; + import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.Executors; import java.util.concurrent.ExecutorService; @@ -40,14 +44,15 @@ public class SqsService { private static final Logger LOG = LoggerFactory.getLogger(SqsService.class); static final long SHUTDOWN_TIMEOUT = 30L; private final SqsSourceConfig sqsSourceConfig; - private final SqsClient sqsClient; private final PluginMetrics pluginMetrics; private final PluginFactory pluginFactory; private final AcknowledgementSetManager acknowledgementSetManager; private final List allSqsUrlExecutorServices; private final List sqsWorkers; private final Buffer> buffer; - private final Backoff backoff; + private final Map sqsClientMap = new HashMap<>(); + private final AwsCredentialsProvider credentialsProvider; + public SqsService(final Buffer> buffer, final AcknowledgementSetManager acknowledgementSetManager, @@ -59,19 +64,23 @@ public SqsService(final Buffer> buffer, this.sqsSourceConfig = sqsSourceConfig; this.pluginMetrics = pluginMetrics; this.pluginFactory = pluginFactory; + this.credentialsProvider = credentialsProvider; this.acknowledgementSetManager = acknowledgementSetManager; this.allSqsUrlExecutorServices = new ArrayList<>(); this.sqsWorkers = new ArrayList<>(); - this.sqsClient = SqsClientFactory.createSqsClient(sqsSourceConfig.getAwsAuthenticationOptions().getAwsRegion(), credentialsProvider); this.buffer = buffer; - backoff = SqsBackoff.createExponentialBackoff(); } public void start() { LOG.info("Starting SqsService"); sqsSourceConfig.getQueues().forEach(queueConfig -> { String queueUrl = queueConfig.getUrl(); + String region = extractRegionFromQueueUrl(queueUrl); + SqsClient sqsClient = sqsClientMap.computeIfAbsent(region, + r -> SqsClientFactory.createSqsClient(Region.of(r), credentialsProvider)); String queueName = queueUrl.substring(queueUrl.lastIndexOf('/') + 1); + Backoff backoff = SqsBackoff.createExponentialBackoff(); + SqsWorkerCommon sqsWorkerCommon = new SqsWorkerCommon(backoff, pluginMetrics, acknowledgementSetManager); int numWorkers = queueConfig.getNumWorkers(); SqsEventProcessor sqsEventProcessor; MessageFieldStrategy strategy; @@ -93,11 +102,11 @@ public void start() { buffer, acknowledgementSetManager, sqsClient, + sqsWorkerCommon, sqsSourceConfig, queueConfig, pluginMetrics, - sqsEventProcessor, - backoff)) + sqsEventProcessor)) .collect(Collectors.toList()); sqsWorkers.addAll(workers); @@ -122,10 +131,15 @@ public void stop() { Thread.currentThread().interrupt(); } }); - - sqsClient.close(); + + sqsClientMap.values().forEach(SqsClient::close); LOG.info("SqsService shutdown completed."); } + + private String extractRegionFromQueueUrl(final String queueUrl) { + String[] split = queueUrl.split("\\."); + return split[1]; + } } \ No newline at end of file diff --git a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java index cb4c168345..ee82e8279d 100644 --- a/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java +++ b/data-prepper-plugins/sqs-source/src/main/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorker.java @@ -10,22 +10,26 @@ package org.opensearch.dataprepper.plugins.source.sqs; -import com.linecorp.armeria.client.retry.Backoff; import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.plugins.source.sqs.common.OnErrorOption; import org.opensearch.dataprepper.plugins.source.sqs.common.SqsWorkerCommon; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.sqs.SqsClient; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.MessageSystemAttributeName; +import software.amazon.awssdk.services.sqs.model.SqsException; import java.time.Duration; +import java.time.Instant; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -34,9 +38,12 @@ public class SqsWorker implements Runnable { private static final Logger LOG = LoggerFactory.getLogger(SqsWorker.class); + static final String SQS_MESSAGE_DELAY_METRIC_NAME = "sqsMessageDelay"; + private final Timer sqsMessageDelayTimer; static final String ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME = "acknowledgementSetCallbackCounter"; private final SqsWorkerCommon sqsWorkerCommon; private final SqsEventProcessor sqsEventProcessor; + private final SqsClient sqsClient; private final QueueConfig queueConfig; private final boolean endToEndAcknowledgementsEnabled; private final Buffer> buffer; @@ -50,21 +57,25 @@ public class SqsWorker implements Runnable { public SqsWorker(final Buffer> buffer, final AcknowledgementSetManager acknowledgementSetManager, final SqsClient sqsClient, + final SqsWorkerCommon sqsWorkerCommon, final SqsSourceConfig sqsSourceConfig, final QueueConfig queueConfig, final PluginMetrics pluginMetrics, - final SqsEventProcessor sqsEventProcessor, - final Backoff backoff) { - this.sqsWorkerCommon = new SqsWorkerCommon(sqsClient, backoff, pluginMetrics, acknowledgementSetManager); + final SqsEventProcessor sqsEventProcessor) { + + this.sqsWorkerCommon = sqsWorkerCommon; this.queueConfig = queueConfig; this.acknowledgementSetManager = acknowledgementSetManager; this.sqsEventProcessor = sqsEventProcessor; + this.sqsClient = sqsClient; this.buffer = buffer; this.bufferTimeoutMillis = (int) sqsSourceConfig.getBufferTimeout().toMillis(); this.endToEndAcknowledgementsEnabled = sqsSourceConfig.getAcknowledgements(); this.messageVisibilityTimesMap = new HashMap<>(); this.failedAttemptCount = 0; - this.acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME); + acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME); + sqsMessageDelayTimer = pluginMetrics.timer(SQS_MESSAGE_DELAY_METRIC_NAME); + } @Override @@ -90,17 +101,27 @@ public void run() { } int processSqsMessages() { - List messages = sqsWorkerCommon.pollSqsMessages(queueConfig.getUrl(), - queueConfig.getMaximumMessages(), - queueConfig.getWaitTime(), - queueConfig.getVisibilityTimeout()); - if (!messages.isEmpty()) { - final List deleteMessageBatchRequestEntries = processSqsEvents(messages); - if (!deleteMessageBatchRequestEntries.isEmpty()) { - sqsWorkerCommon.deleteSqsMessages(queueConfig.getUrl(), deleteMessageBatchRequestEntries); + try { + List messages = sqsWorkerCommon.pollSqsMessages( + queueConfig.getUrl(), + sqsClient, + queueConfig.getMaximumMessages(), + queueConfig.getWaitTime(), + queueConfig.getVisibilityTimeout()); + + if (!messages.isEmpty()) { + final List deleteEntries = processSqsEvents(messages); + if (!deleteEntries.isEmpty()) { + sqsWorkerCommon.deleteSqsMessages(queueConfig.getUrl(), sqsClient, deleteEntries); + } + } else { + sqsMessageDelayTimer.record(Duration.ZERO); } + return messages.size(); + } catch (SqsException e) { + sqsWorkerCommon.applyBackoff(); + return 0; } - return messages.size(); } private List processSqsEvents(final List messages) { @@ -128,7 +149,7 @@ private List processSqsEvents(final List processSqsEvents(final List processSqsEvents(final List waitingForAcknowledgements = messageWaitingForAcknowledgementsMap.get(message); final Optional deleteEntry = processSqsObject(message, acknowledgementSet); @@ -174,7 +203,11 @@ private Optional processSqsObject(final Message sqsWorkerCommon.getSqsMessagesFailedCounter().increment(); LOG.error("Error processing from SQS: {}. Retrying with exponential backoff.", e.getMessage()); sqsWorkerCommon.applyBackoff(); - return Optional.empty(); + if (queueConfig.getOnErrorOption().equals(OnErrorOption.DELETE_MESSAGES)) { + return Optional.of(sqsWorkerCommon.buildDeleteMessageBatchRequestEntry(message.messageId(), message.receiptHandle())); + } else { + return Optional.empty(); + } } } diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AttributeHandlerTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AttributeHandlerTest.java new file mode 100644 index 0000000000..be7c772620 --- /dev/null +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/AttributeHandlerTest.java @@ -0,0 +1,70 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.source.sqs; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import java.util.HashMap; +import java.util.Map; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.MessageAttributeValue; +import software.amazon.awssdk.services.sqs.model.MessageSystemAttributeName; + +import org.mockito.Mockito; + +public class AttributeHandlerTest { + + @Test + void testCollectMetadataAttributes() { + final Map systemAttributes = new HashMap<>(); + systemAttributes.put(MessageSystemAttributeName.SENT_TIMESTAMP, "1234567890"); + systemAttributes.put(MessageSystemAttributeName.APPROXIMATE_RECEIVE_COUNT, "5"); + final Map messageAttributes = new HashMap<>(); + + messageAttributes.put("CustomKey", MessageAttributeValue.builder() + .stringValue("customValue") + .dataType("String") + .build()); + + final Message message = Message.builder() + .messageId("id-1") + .receiptHandle("rh-1") + .body("Test message") + .attributes(systemAttributes) + .messageAttributes(messageAttributes) + .build(); + + final String queueUrl = "https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"; + final Map metadata = AttributeHandler.collectMetadataAttributes(message, queueUrl); + assertThat(metadata.get("queueUrl"), equalTo(queueUrl)); + assertThat(metadata.get("sentTimestamp"), equalTo("1234567890")); + assertThat(metadata.get("approximateReceiveCount"), equalTo("5")); + assertThat(metadata.get("customKey"), equalTo("customValue")); + } + + @Test + void testApplyMetadataAttributes() { + final Event event = Mockito.mock(Event.class); + final EventMetadata metadata = Mockito.mock(EventMetadata.class); + when(event.getMetadata()).thenReturn(metadata); + final Map attributes = new HashMap<>(); + attributes.put("key1", "value1"); + attributes.put("key2", "value2"); + AttributeHandler.applyMetadataAttributes(event, attributes); + verify(metadata).setAttribute("key1", "value1"); + verify(metadata).setAttribute("key2", "value2"); + } +} diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java index 83a12e5940..4b4777d434 100644 --- a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsServiceTest.java @@ -20,18 +20,15 @@ import org.opensearch.dataprepper.model.record.Record; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.sqs.SqsClient; -import java.util.List; +import java.util.List; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; -import static org.mockito.Mockito.withSettings; class SqsServiceTest { private SqsSourceConfig sqsSourceConfig; - private SqsClient sqsClient; private PluginMetrics pluginMetrics; private PluginFactory pluginFactory; private AcknowledgementSetManager acknowledgementSetManager; @@ -41,13 +38,11 @@ class SqsServiceTest { @BeforeEach void setUp() { sqsSourceConfig = mock(SqsSourceConfig.class); - sqsClient = mock(SqsClient.class, withSettings()); pluginMetrics = mock(PluginMetrics.class); pluginFactory = mock(PluginFactory.class); acknowledgementSetManager = mock(AcknowledgementSetManager.class); buffer = mock(Buffer.class); credentialsProvider = mock(AwsCredentialsProvider.class); - AwsAuthenticationOptions awsAuthenticationOptions = mock(AwsAuthenticationOptions.class); when(awsAuthenticationOptions.getAwsRegion()).thenReturn(Region.US_EAST_1); when(sqsSourceConfig.getAwsAuthenticationOptions()).thenReturn(awsAuthenticationOptions); @@ -66,10 +61,9 @@ void start_with_single_queue_starts_workers() { @Test void stop_should_shutdown_executors_and_workers() throws InterruptedException { QueueConfig queueConfig = mock(QueueConfig.class); - when(queueConfig.getUrl()).thenReturn("MyQueue"); + when(queueConfig.getUrl()).thenReturn("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"); when(queueConfig.getNumWorkers()).thenReturn(1); when(sqsSourceConfig.getQueues()).thenReturn(List.of(queueConfig)); - SqsClient sqsClient = mock(SqsClient.class); SqsService sqsService = new SqsService(buffer, acknowledgementSetManager, sqsSourceConfig, pluginMetrics, pluginFactory, credentialsProvider) {}; sqsService.start(); sqsService.stop(); // again assuming that if no exception is thrown here, then workers and client have been stopped diff --git a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java index e7339543c2..9f7a7f1f30 100644 --- a/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java +++ b/data-prepper-plugins/sqs-source/src/test/java/org/opensearch/dataprepper/plugins/source/sqs/SqsWorkerTest.java @@ -10,8 +10,8 @@ package org.opensearch.dataprepper.plugins.source.sqs; -import com.linecorp.armeria.client.retry.Backoff; import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.Timer; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -24,47 +24,42 @@ import org.opensearch.dataprepper.model.acknowledgements.ProgressCheck; import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.sqs.common.OnErrorOption; import org.opensearch.dataprepper.plugins.source.sqs.common.SqsWorkerCommon; -import org.opensearch.dataprepper.plugins.source.sqs.common.SqsRetriesExhaustedException; import software.amazon.awssdk.services.sqs.SqsClient; -import software.amazon.awssdk.services.sqs.model.ChangeMessageVisibilityRequest; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequest; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; -import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResultEntry; +import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchRequestEntry; import software.amazon.awssdk.services.sqs.model.Message; -import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; +import software.amazon.awssdk.services.sqs.model.MessageSystemAttributeName; import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse; import software.amazon.awssdk.services.sqs.model.SqsException; -import software.amazon.awssdk.services.sqs.model.BatchResultErrorEntry; import java.io.IOException; import java.time.Duration; +import java.time.Instant; import java.util.Collections; -import java.util.UUID; +import java.util.Map; import java.util.function.Consumer; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.MatcherAssert.assertThat; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyDouble; import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyList; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.isNull; +import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.lenient; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) class SqsWorkerTest { - @Mock private Buffer> buffer; @Mock @@ -72,6 +67,8 @@ class SqsWorkerTest { @Mock private SqsClient sqsClient; @Mock + private SqsWorkerCommon sqsWorkerCommon; + @Mock private SqsEventProcessor sqsEventProcessor; @Mock private SqsSourceConfig sqsSourceConfig; @@ -80,125 +77,119 @@ class SqsWorkerTest { @Mock private PluginMetrics pluginMetrics; @Mock - private PluginFactory pluginFactory; - @Mock - private Backoff backoff; - @Mock - private Counter sqsMessagesReceivedCounter; - @Mock - private Counter sqsMessagesDeletedCounter; + private Timer sqsMessageDelayTimer; @Mock private Counter sqsMessagesFailedCounter; - @Mock - private Counter sqsMessagesDeleteFailedCounter; - @Mock - private Counter acknowledgementSetCallbackCounter; - @Mock - private Counter sqsVisibilityTimeoutChangedCount; - @Mock - private Counter sqsVisibilityTimeoutChangeFailedCount; - private final int mockBufferTimeoutMillis = 10000; + private final int bufferTimeoutMillis = 10000; + private SqsWorker sqsWorker; private SqsWorker createObjectUnderTest() { return new SqsWorker( buffer, acknowledgementSetManager, sqsClient, + sqsWorkerCommon, sqsSourceConfig, queueConfig, pluginMetrics, - sqsEventProcessor, - backoff); + sqsEventProcessor + ); } @BeforeEach void setUp() { - when(pluginMetrics.counter(SqsWorkerCommon.SQS_MESSAGES_RECEIVED_METRIC_NAME)) - .thenReturn(sqsMessagesReceivedCounter); - when(pluginMetrics.counter(SqsWorkerCommon.SQS_MESSAGES_DELETED_METRIC_NAME)) - .thenReturn(sqsMessagesDeletedCounter); - when(pluginMetrics.counter(SqsWorkerCommon.SQS_MESSAGES_FAILED_METRIC_NAME)) - .thenReturn(sqsMessagesFailedCounter); - when(pluginMetrics.counter(SqsWorkerCommon.SQS_MESSAGES_DELETE_FAILED_METRIC_NAME)) - .thenReturn(sqsMessagesDeleteFailedCounter); - when(pluginMetrics.counter(SqsWorkerCommon.ACKNOWLEDGEMENT_SET_CALLACK_METRIC_NAME)) - .thenReturn(acknowledgementSetCallbackCounter); - when(pluginMetrics.counter(SqsWorkerCommon.SQS_VISIBILITY_TIMEOUT_CHANGED_COUNT_METRIC_NAME)) - .thenReturn(sqsVisibilityTimeoutChangedCount); - when(pluginMetrics.counter(SqsWorkerCommon.SQS_VISIBILITY_TIMEOUT_CHANGE_FAILED_COUNT_METRIC_NAME)) - .thenReturn(sqsVisibilityTimeoutChangeFailedCount); when(sqsSourceConfig.getAcknowledgements()).thenReturn(false); when(sqsSourceConfig.getBufferTimeout()).thenReturn(Duration.ofSeconds(10)); - when(queueConfig.getUrl()).thenReturn("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"); - when(queueConfig.getWaitTime()).thenReturn(Duration.ofSeconds(1)); - } - - @Test - void processSqsMessages_should_return_number_of_messages_processed_and_increment_counters() throws IOException { + lenient().when(queueConfig.getUrl()).thenReturn("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"); + lenient().when(queueConfig.getWaitTime()).thenReturn(Duration.ofSeconds(10)); + lenient().when(queueConfig.getMaximumMessages()).thenReturn(10); + lenient().when(queueConfig.getVisibilityTimeout()).thenReturn(Duration.ofSeconds(30)); + when(pluginMetrics.timer(SqsWorker.SQS_MESSAGE_DELAY_METRIC_NAME)).thenReturn(sqsMessageDelayTimer); + lenient().doNothing().when(sqsMessageDelayTimer).record(any(Duration.class)); + sqsWorker = new SqsWorker( + buffer, + acknowledgementSetManager, + sqsClient, + sqsWorkerCommon, + sqsSourceConfig, + queueConfig, + pluginMetrics, + sqsEventProcessor + ); final Message message = Message.builder() - .messageId(UUID.randomUUID().toString()) - .receiptHandle(UUID.randomUUID().toString()) + .messageId("msg-1") + .receiptHandle("rh-1") .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") + .attributes(Map.of( + MessageSystemAttributeName.SENT_TIMESTAMP, "1234567890", + MessageSystemAttributeName.APPROXIMATE_RECEIVE_COUNT, "0" + )) .build(); - final ReceiveMessageResponse response = ReceiveMessageResponse.builder().messages(message).build(); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(response); - final DeleteMessageBatchResultEntry successfulDelete = DeleteMessageBatchResultEntry.builder().id(message.messageId()).build(); - final DeleteMessageBatchResponse deleteResponse = DeleteMessageBatchResponse.builder().successful(successfulDelete).build(); - when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenReturn(deleteResponse); - - int messagesProcessed = createObjectUnderTest().processSqsMessages(); - assertThat(messagesProcessed, equalTo(1)); - - verify(sqsMessagesReceivedCounter).increment(1); - verify(sqsMessagesDeletedCounter).increment(1); - verify(sqsMessagesDeleteFailedCounter, never()).increment(anyDouble()); + lenient().when(sqsWorkerCommon.pollSqsMessages( + anyString(), + eq(sqsClient), + any(), + any(), + any() + )).thenReturn(Collections.singletonList(message)); + + lenient().when(sqsWorkerCommon.buildDeleteMessageBatchRequestEntry(anyString(), anyString())) + .thenAnswer(invocation -> { + String messageId = invocation.getArgument(0); + String receiptHandle = invocation.getArgument(1); + return DeleteMessageBatchRequestEntry.builder() + .id(messageId) + .receiptHandle(receiptHandle) + .build(); + }); } @Test - void processSqsMessages_should_invoke_processSqsEvent_and_deleteSqsMessages_when_entries_non_empty() throws IOException { - final Message message = Message.builder() - .messageId(UUID.randomUUID().toString()) - .receiptHandle(UUID.randomUUID().toString()) - .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") - .build(); - - final ReceiveMessageResponse response = ReceiveMessageResponse.builder() - .messages(message) - .build(); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(response); - - final DeleteMessageBatchResultEntry successfulDelete = DeleteMessageBatchResultEntry.builder() - .id(message.messageId()) - .build(); - final DeleteMessageBatchResponse deleteResponse = DeleteMessageBatchResponse.builder() - .successful(successfulDelete) - .build(); - when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenReturn(deleteResponse); - - SqsWorker sqsWorker = createObjectUnderTest(); + void processSqsMessages_should_call_addSqsObject_and_deleteSqsMessages_for_valid_message() throws IOException { int messagesProcessed = sqsWorker.processSqsMessages(); - assertThat(messagesProcessed, equalTo(1)); - verify(sqsEventProcessor, times(1)).addSqsObject(eq(message), eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), eq(buffer), eq(mockBufferTimeoutMillis), isNull()); - verify(sqsClient, times(1)).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); - verify(sqsMessagesReceivedCounter).increment(1); - verify(sqsMessagesDeletedCounter).increment(1); - verify(sqsMessagesDeleteFailedCounter, never()).increment(anyDouble()); + verify(sqsEventProcessor, times(1)).addSqsObject( + any(), + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(buffer), + anyInt(), + isNull()); + verify(sqsWorkerCommon, atLeastOnce()).deleteSqsMessages( + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(sqsClient), + anyList() + ); + verify(sqsMessageDelayTimer, times(1)).record(any(Duration.class)); } + @Test void processSqsMessages_should_not_invoke_processSqsEvent_and_deleteSqsMessages_when_entries_are_empty() throws IOException { - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))) - .thenReturn(ReceiveMessageResponse.builder().messages(Collections.emptyList()).build()); - SqsWorker sqsWorker = createObjectUnderTest(); + when(sqsWorkerCommon.pollSqsMessages( + anyString(), + eq(sqsClient), + any(), + any(), + any() + )).thenReturn(ReceiveMessageResponse.builder().messages(Collections.emptyList()).build().messages()); + int messagesProcessed = sqsWorker.processSqsMessages(); + assertThat(messagesProcessed, equalTo(0)); - verify(sqsEventProcessor, never()).addSqsObject(any(), anyString(), any(), anyInt(), any()); - verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); - verify(sqsMessagesReceivedCounter, never()).increment(anyDouble()); - verify(sqsMessagesDeletedCounter, never()).increment(anyDouble()); + verify(sqsEventProcessor, times(0)).addSqsObject( + any(), + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(buffer), + anyInt(), + isNull()); + verify(sqsWorkerCommon, times(0)).deleteSqsMessages( + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(sqsClient), + anyList() + ); + verify(sqsMessageDelayTimer, times(1)).record(any(Duration.class)); } @@ -208,188 +199,176 @@ void processSqsMessages_should_not_delete_messages_if_acknowledgements_enabled_u AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class); when(acknowledgementSetManager.create(any(), any())).thenReturn(acknowledgementSet); when(queueConfig.getUrl()).thenReturn("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"); - - final Message message = Message.builder() - .messageId("msg-1") - .receiptHandle("rh-1") - .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") - .build(); - - final ReceiveMessageResponse response = ReceiveMessageResponse.builder().messages(message).build(); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(response); int messagesProcessed = createObjectUnderTest().processSqsMessages(); assertThat(messagesProcessed, equalTo(1)); - verify(sqsEventProcessor, times(1)).addSqsObject(eq(message), + verify(sqsEventProcessor, times(1)).addSqsObject(any(), eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), eq(buffer), - eq(mockBufferTimeoutMillis), + eq(bufferTimeoutMillis), eq(acknowledgementSet)); - verify(sqsMessagesReceivedCounter).increment(1); - verifyNoInteractions(sqsMessagesDeletedCounter); + verify(sqsWorkerCommon, times(0)).deleteSqsMessages( + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(sqsClient), + anyList() + ); + verify(sqsMessageDelayTimer, times(1)).record(any(Duration.class)); } @Test void acknowledgementsEnabled_and_visibilityDuplicateProtectionEnabled_should_create_ack_sets_and_progress_check() { when(sqsSourceConfig.getAcknowledgements()).thenReturn(true); when(queueConfig.getVisibilityDuplicateProtection()).thenReturn(true); - - SqsWorker worker = new SqsWorker(buffer, acknowledgementSetManager, sqsClient, sqsSourceConfig, queueConfig, pluginMetrics, sqsEventProcessor, backoff); - Message message = Message.builder().messageId("msg-dup").receiptHandle("handle-dup").build(); - ReceiveMessageResponse response = ReceiveMessageResponse.builder().messages(message).build(); - when(sqsClient.receiveMessage((ReceiveMessageRequest) any())).thenReturn(response); - AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class); when(acknowledgementSetManager.create(any(), any())).thenReturn(acknowledgementSet); - - int processed = worker.processSqsMessages(); - assertThat(processed, equalTo(1)); - + createObjectUnderTest().processSqsMessages(); verify(acknowledgementSetManager).create(any(), any()); verify(acknowledgementSet).addProgressCheck(any(), any()); } @Test void processSqsMessages_should_return_zero_messages_with_backoff_when_a_SqsException_is_thrown() { - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenThrow(SqsException.class); + when(sqsWorkerCommon.pollSqsMessages( + anyString(), + eq(sqsClient), + any(), + any(), + any() + )).thenThrow(SqsException.class); final int messagesProcessed = createObjectUnderTest().processSqsMessages(); - verify(backoff).nextDelayMillis(1); + verify(sqsWorkerCommon, times(1)).applyBackoff(); assertThat(messagesProcessed, equalTo(0)); } - @Test - void processSqsMessages_should_throw_when_a_SqsException_is_thrown_with_max_retries() { - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenThrow(SqsException.class); - when(backoff.nextDelayMillis(anyInt())).thenReturn((long) -1); - SqsWorker objectUnderTest = createObjectUnderTest(); - assertThrows(SqsRetriesExhaustedException.class, objectUnderTest::processSqsMessages); - } @Test void processSqsMessages_should_update_visibility_timeout_when_progress_changes() throws IOException { - AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class); - when(queueConfig.getVisibilityDuplicateProtection()).thenReturn(true); - when(queueConfig.getVisibilityTimeout()).thenReturn(Duration.ofMillis(1)); - when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); when(sqsSourceConfig.getAcknowledgements()).thenReturn(true); - final Message message = mock(Message.class); - final String testReceiptHandle = UUID.randomUUID().toString(); - when(message.messageId()).thenReturn(testReceiptHandle); - when(message.receiptHandle()).thenReturn(testReceiptHandle); - - final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); - when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); - - final int messagesProcessed = createObjectUnderTest().processSqsMessages(); + when(queueConfig.getVisibilityDuplicateProtection()).thenReturn(true); + when(queueConfig.getVisibilityDuplicateProtectionTimeout()).thenReturn(Duration.ofSeconds(60)); + when(queueConfig.getVisibilityTimeout()).thenReturn(Duration.ofSeconds(30)); + when(queueConfig.getUrl()).thenReturn("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"); + AcknowledgementSet acknowledgementSet = mock(AcknowledgementSet.class); + when(acknowledgementSetManager.create(any(), any(Duration.class))) + .thenReturn(acknowledgementSet); + final String testMessageId = "msg-1"; + final String testReceiptHandle = "rh-1"; + SqsWorker sqsWorker = createObjectUnderTest(); // your builder method + final int messagesProcessed = sqsWorker.processSqsMessages(); assertThat(messagesProcessed, equalTo(1)); - verify(sqsEventProcessor).addSqsObject(any(), anyString(), any(), anyInt(), any()); + + verify(sqsEventProcessor).addSqsObject( + any(), + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(buffer), + eq(bufferTimeoutMillis), + eq(acknowledgementSet) + ); verify(acknowledgementSetManager).create(any(), any(Duration.class)); - ArgumentCaptor> progressConsumerArgumentCaptor = ArgumentCaptor.forClass(Consumer.class); - verify(acknowledgementSet).addProgressCheck(progressConsumerArgumentCaptor.capture(), any(Duration.class)); - final Consumer actualConsumer = progressConsumerArgumentCaptor.getValue(); - final ProgressCheck progressCheck = mock(ProgressCheck.class); + @SuppressWarnings("unchecked") + ArgumentCaptor> progressConsumerCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSet).addProgressCheck(progressConsumerCaptor.capture(), any(Duration.class)); + final Consumer actualConsumer = progressConsumerCaptor.getValue(); + ProgressCheck progressCheck = mock(ProgressCheck.class); actualConsumer.accept(progressCheck); - - ArgumentCaptor changeMessageVisibilityRequestArgumentCaptor = ArgumentCaptor.forClass(ChangeMessageVisibilityRequest.class); - verify(sqsClient).changeMessageVisibility(changeMessageVisibilityRequestArgumentCaptor.capture()); - ChangeMessageVisibilityRequest actualChangeVisibilityRequest = changeMessageVisibilityRequestArgumentCaptor.getValue(); - assertThat(actualChangeVisibilityRequest.queueUrl(), equalTo(queueConfig.getUrl())); - assertThat(actualChangeVisibilityRequest.receiptHandle(), equalTo(testReceiptHandle)); - verify(sqsMessagesReceivedCounter).increment(1); + verify(sqsWorkerCommon).increaseVisibilityTimeout( + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(sqsClient), + eq(testReceiptHandle), + eq(30), + eq(testMessageId) + ); } @Test - void increaseVisibilityTimeout_doesNothing_whenIsStopped() throws IOException { - when(sqsSourceConfig.getAcknowledgements()).thenReturn(true); - when(queueConfig.getVisibilityDuplicateProtection()).thenReturn(false); - when(queueConfig.getVisibilityTimeout()).thenReturn(Duration.ofSeconds(30)); - AcknowledgementSet mockAcknowledgementSet = mock(AcknowledgementSet.class); - when(acknowledgementSetManager.create(any(), any())).thenReturn(mockAcknowledgementSet); - Message message = Message.builder() - .messageId(UUID.randomUUID().toString()) - .receiptHandle(UUID.randomUUID().toString()) - .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") - .build(); - ReceiveMessageResponse response = ReceiveMessageResponse.builder() - .messages(message) - .build(); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(response); - SqsWorker sqsWorker = createObjectUnderTest(); - sqsWorker.stop(); - int messagesProcessed = sqsWorker.processSqsMessages(); + void processSqsMessages_should_return_delete_message_entry_when_exception_thrown_and_onErrorOption_is_DELETE_MESSAGES() throws IOException { + when(queueConfig.getOnErrorOption()).thenReturn(OnErrorOption.DELETE_MESSAGES); + doThrow(new RuntimeException("Processing error")) + .when(sqsEventProcessor).addSqsObject(any(), + anyString(), + eq(buffer), + anyInt(), + isNull()); + + when(sqsWorkerCommon.getSqsMessagesFailedCounter()).thenReturn(sqsMessagesFailedCounter); + SqsWorker worker = createObjectUnderTest(); + int messagesProcessed = worker.processSqsMessages(); assertThat(messagesProcessed, equalTo(1)); - verify(sqsEventProcessor, times(1)).addSqsObject(eq(message), + verify(sqsMessagesFailedCounter, times(1)).increment(); + verify(sqsWorkerCommon, atLeastOnce()).applyBackoff(); + verify(sqsWorkerCommon, atLeastOnce()).deleteSqsMessages( eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), - eq(buffer), - eq(mockBufferTimeoutMillis), - eq(mockAcknowledgementSet)); - verify(sqsClient, never()).changeMessageVisibility(any(ChangeMessageVisibilityRequest.class)); - verify(sqsVisibilityTimeoutChangeFailedCount, never()).increment(); + eq(sqsClient), + anyList()); } @Test - void deleteSqsMessages_incrementsFailedCounter_whenDeleteResponseHasFailedDeletes() throws IOException { - final Message message1 = Message.builder() - .messageId(UUID.randomUUID().toString()) - .receiptHandle(UUID.randomUUID().toString()) - .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World 1\"}]}") - .build(); - final Message message2 = Message.builder() - .messageId(UUID.randomUUID().toString()) - .receiptHandle(UUID.randomUUID().toString()) - .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World 2\"}]}") - .build(); - - final ReceiveMessageResponse receiveResponse = ReceiveMessageResponse.builder() - .messages(message1, message2) - .build(); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveResponse); - - DeleteMessageBatchResultEntry successfulDelete = DeleteMessageBatchResultEntry.builder() - .id(message1.messageId()) - .build(); + void processSqsMessages_should_not_delete_message_entry_when_exception_thrown_and_onErrorOption_is_RETAIN_MESSAGES() throws IOException { + when(queueConfig.getOnErrorOption()).thenReturn(OnErrorOption.RETAIN_MESSAGES); + doThrow(new RuntimeException("Processing error")) + .when(sqsEventProcessor).addSqsObject(any(), + anyString(), + eq(buffer), + anyInt(), + isNull()); + + when(sqsWorkerCommon.getSqsMessagesFailedCounter()).thenReturn(sqsMessagesFailedCounter); + SqsWorker worker = createObjectUnderTest(); + int messagesProcessed = worker.processSqsMessages(); + assertThat(messagesProcessed, equalTo(1)); + verify(sqsMessagesFailedCounter, times(1)).increment(); + verify(sqsWorkerCommon, atLeastOnce()).applyBackoff(); + verify(sqsWorkerCommon, times(0)).deleteSqsMessages( + eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), + eq(sqsClient), + anyList()); + } - BatchResultErrorEntry failedDelete = BatchResultErrorEntry.builder() - .id(message2.messageId()) - .code("ReceiptHandleIsInvalid") - .senderFault(true) - .message("Failed to delete message due to invalid receipt handle.") - .build(); + @Test + void stop_should_set_isStopped_and_call_stop_on_sqsWorkerCommon() { + SqsWorker worker = createObjectUnderTest(); + worker.stop(); + verify(sqsWorkerCommon, times(1)).stop(); + } - DeleteMessageBatchResponse deleteResponse = DeleteMessageBatchResponse.builder() - .successful(successfulDelete) - .failed(failedDelete) + @Test + void processSqsMessages_should_record_sqsMessageDelayTimer_when_approximateReceiveCount_less_than_or_equal_to_one() throws IOException { + final long sentTimestampMillis = Instant.now().minusSeconds(5).toEpochMilli(); + final Message message = Message.builder() + .messageId("msg-1") + .receiptHandle("rh-1") + .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") + .attributes(Map.of( + MessageSystemAttributeName.SENT_TIMESTAMP, String.valueOf(sentTimestampMillis), + MessageSystemAttributeName.APPROXIMATE_RECEIVE_COUNT, "1" + )) .build(); + when(sqsWorkerCommon.pollSqsMessages(anyString(), eq(sqsClient), anyInt(), any(), any())) + .thenReturn(Collections.singletonList(message)); - when(sqsClient.deleteMessageBatch(any(DeleteMessageBatchRequest.class))).thenReturn(deleteResponse); - SqsWorker sqsWorker = createObjectUnderTest(); - int messagesProcessed = sqsWorker.processSqsMessages(); - assertThat(messagesProcessed, equalTo(2)); - verify(sqsMessagesReceivedCounter).increment(2); - verify(sqsMessagesDeletedCounter).increment(1); - verify(sqsMessagesDeleteFailedCounter).increment(1); + SqsWorker worker = createObjectUnderTest(); + worker.processSqsMessages(); + ArgumentCaptor durationCaptor = ArgumentCaptor.forClass(Duration.class); + verify(sqsMessageDelayTimer).record(durationCaptor.capture()); } + @Test - void processSqsMessages_handlesException_correctly_when_addSqsObject_throwsException() throws IOException { + void processSqsMessages_should_not_record_sqsMessageDelayTimer_when_approximateReceiveCount_greater_than_one() throws IOException { + final long sentTimestampMillis = Instant.now().minusSeconds(5).toEpochMilli(); final Message message = Message.builder() - .messageId(UUID.randomUUID().toString()) - .receiptHandle(UUID.randomUUID().toString()) + .messageId("msg-1") + .receiptHandle("rh-1") .body("{\"Records\":[{\"eventSource\":\"custom\",\"message\":\"Hello World\"}]}") + .attributes(Map.of( + MessageSystemAttributeName.SENT_TIMESTAMP, String.valueOf(sentTimestampMillis), + MessageSystemAttributeName.APPROXIMATE_RECEIVE_COUNT, "2" + )) .build(); - when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn( - ReceiveMessageResponse.builder().messages(message).build() - ); - doThrow(new RuntimeException("Processing failed")).when(sqsEventProcessor) - .addSqsObject(eq(message), eq("https://sqs.us-east-1.amazonaws.com/123456789012/MyQueue"), - any(), anyInt(), any()); - SqsWorker sqsWorker = createObjectUnderTest(); - int messagesProcessed = sqsWorker.processSqsMessages(); - assertThat(messagesProcessed, equalTo(1)); - verify(sqsMessagesReceivedCounter).increment(1); - verify(sqsMessagesFailedCounter).increment(); - verify(backoff).nextDelayMillis(anyInt()); - verify(sqsClient, never()).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); - verify(sqsMessagesDeletedCounter, never()).increment(anyInt()); + when(sqsWorkerCommon.pollSqsMessages(anyString(), eq(sqsClient), anyInt(), any(), any())) + .thenReturn(Collections.singletonList(message)); + + SqsWorker worker = createObjectUnderTest(); + worker.processSqsMessages(); + verify(sqsMessageDelayTimer, never()).record(any(Duration.class)); } } \ No newline at end of file From 705c0d94dcf0b397ce57eae2718c48518627b561 Mon Sep 17 00:00:00 2001 From: Hai Yan <8153134+oeyh@users.noreply.github.com> Date: Fri, 7 Feb 2025 10:41:42 -0600 Subject: [PATCH 15/18] Support export from postgres (#5414) Signed-off-by: Hai Yan --- .../source/rds/configuration/EngineType.java | 2 +- .../state/DataFileProgressState.java | 15 ++++++++++++ .../state/ExportProgressState.java | 11 +++++++++ .../source/rds/export/DataFileLoader.java | 19 ++++++++------- .../source/rds/export/ExportScheduler.java | 9 ++++++- .../source/rds/leader/LeaderScheduler.java | 2 ++ .../source/rds/model/ExportObjectKey.java | 24 +++++++++++++++---- .../rds/stream/LogicalReplicationClient.java | 5 ++++ .../LogicalReplicationEventProcessor.java | 8 +++++-- .../source/rds/stream/StreamWorker.java | 2 ++ .../rds/leader/LeaderSchedulerTest.java | 9 +++++-- .../source/rds/model/ExportObjectKeyTest.java | 19 ++++++++++++++- .../stream/LogicalReplicationClientTest.java | 16 +++++++++++++ 13 files changed, 122 insertions(+), 19 deletions(-) diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java index 20f7f3b534..99980eae71 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/configuration/EngineType.java @@ -33,7 +33,7 @@ public String toString() { } @JsonCreator - public static EngineType fromOptionValue(final String option) { + public static EngineType fromString(final String option) { return ENGINE_TYPE_MAP.get(option); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java index 9fee60105f..e865e73af8 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/DataFileProgressState.java @@ -12,6 +12,9 @@ public class DataFileProgressState { + @JsonProperty("engineType") + private String engineType; + @JsonProperty("isLoaded") private boolean isLoaded = false; @@ -21,6 +24,10 @@ public class DataFileProgressState { @JsonProperty("sourceDatabase") private String sourceDatabase; + /** + * For MySQL, sourceTable is in the format of tableName + * For Postgres, sourceTable is in the format of schemaName.tableName + */ @JsonProperty("sourceTable") private String sourceTable; @@ -33,6 +40,14 @@ public class DataFileProgressState { @JsonProperty("snapshotTime") private long snapshotTime; + public String getEngineType() { + return engineType; + } + + public void setEngineType(String engineType) { + this.engineType = engineType; + } + public int getTotalRecords() { return totalRecords; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java index e4bbeb4c98..74b387d3b4 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/coordination/state/ExportProgressState.java @@ -15,6 +15,9 @@ */ public class ExportProgressState { + @JsonProperty("engineType") + private String engineType; + @JsonProperty("snapshotId") private String snapshotId; @@ -48,6 +51,14 @@ public class ExportProgressState { @JsonProperty("status") private String status; + public String getEngineType() { + return engineType; + } + + public void setEngineType(String engineType) { + this.engineType = engineType; + } + public String getSnapshotId() { return snapshotId; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java index b0a205c9ec..75f4dd0dc5 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/DataFileLoader.java @@ -16,6 +16,7 @@ import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.converter.ExportRecordConverter; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.state.DataFileProgressState; @@ -127,7 +128,7 @@ public void run() { final String fullTableName = progressState.getSourceDatabase() + DOT_DELIMITER + progressState.getSourceTable(); final List primaryKeys = progressState.getPrimaryKeyMap().getOrDefault(fullTableName, List.of()); - transformEvent(event, fullTableName); + transformEvent(event, fullTableName, EngineType.fromString(progressState.getEngineType())); final long snapshotTime = progressState.getSnapshotTime(); final long eventVersionNumber = snapshotTime - VERSION_OVERLAP_TIME_FOR_EXPORT.toMillis(); @@ -173,13 +174,15 @@ public void run() { } } - private void transformEvent(final Event event, final String fullTableName) { - Map columnDataTypeMap = dbTableMetadata.getTableColumnDataTypeMap().get(fullTableName); - for (Map.Entry entry : event.toMap().entrySet()) { - final Object data = DataTypeHelper.getDataByColumnType(MySQLDataType.byDataType(columnDataTypeMap.get(entry.getKey())), entry.getKey(), - entry.getValue(), null); - event.put(entry.getKey(), data); + private void transformEvent(final Event event, final String fullTableName, final EngineType engineType) { + // TODO: support data type mapping in Postgres + if (engineType == EngineType.MYSQL) { + Map columnDataTypeMap = dbTableMetadata.getTableColumnDataTypeMap().get(fullTableName); + for (Map.Entry entry : event.toMap().entrySet()) { + final Object data = DataTypeHelper.getDataByColumnType(MySQLDataType.byDataType(columnDataTypeMap.get(entry.getKey())), entry.getKey(), + entry.getValue(), null); + event.put(entry.getKey(), data); + } } } - } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java index ce534747e1..97821a64b7 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/export/ExportScheduler.java @@ -9,6 +9,7 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.DataFilePartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; @@ -53,6 +54,7 @@ public class ExportScheduler implements Runnable { static final String EXPORT_JOB_SUCCESS_COUNT = "exportJobSuccess"; static final String EXPORT_JOB_FAILURE_COUNT = "exportJobFailure"; static final String EXPORT_S3_OBJECTS_TOTAL_COUNT = "exportS3ObjectsTotal"; + static final String DOT_DELIMITER = "."; private final S3Client s3Client; private final PluginMetrics pluginMetrics; @@ -65,6 +67,8 @@ public class ExportScheduler implements Runnable { private final Counter exportJobFailureCounter; private final Counter exportS3ObjectsTotalCounter; + private EngineType engineType; + private volatile boolean shutdownRequested = false; public ExportScheduler(final EnhancedSourceCoordinator sourceCoordinator, @@ -133,6 +137,7 @@ public void shutdown() { private String getOrCreateExportTaskId(ExportPartition exportPartition) { ExportProgressState progressState = exportPartition.getProgressState().get(); + engineType = EngineType.fromString(progressState.getEngineType()); if (progressState.getExportTaskId() != null) { LOG.info("Export task has already created for db {}", exportPartition.getDbIdentifier()); @@ -316,7 +321,9 @@ private void createDataFilePartitions(String bucket, final DataFileProgressState progressState = new DataFileProgressState(); final ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKey); final String database = exportObjectKey.getDatabaseName(); - final String table = exportObjectKey.getTableName(); + final String table = engineType == EngineType.MYSQL ? + exportObjectKey.getTableName() : + exportObjectKey.getSchemaName() + DOT_DELIMITER + exportObjectKey.getTableName(); progressState.setSourceDatabase(database); progressState.setSourceTable(table); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java index f2587a079d..79489211e5 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderScheduler.java @@ -81,6 +81,7 @@ public void run() { if (leaderPartition != null) { LeaderProgressState leaderProgressState = leaderPartition.getProgressState().get(); if (!leaderProgressState.isInitialized()) { + LOG.info("Performing initialization as LEADER node."); init(); } } @@ -139,6 +140,7 @@ private void init() { private void createExportPartition(RdsSourceConfig sourceConfig) { ExportProgressState progressState = new ExportProgressState(); + progressState.setEngineType(sourceConfig.getEngine().toString()); progressState.setIamRoleArn(sourceConfig.getExport().getIamRoleArn()); progressState.setBucket(sourceConfig.getS3Bucket()); // This prefix is for data exported from RDS diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java index dab6cc8d40..ee53b552d5 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKey.java @@ -10,7 +10,7 @@ /** * Represents the object key for an object exported to S3 by RDS. - * The object key has this structure: "{prefix}/{export task ID}/{database name}/{table name}/{numbered folder}/{file name}" + * The object key has this structure: "{prefix}/{export task ID}/{database name}/{full table name}/{numbered folder}/{file name}" */ public class ExportObjectKey { @@ -18,14 +18,21 @@ public class ExportObjectKey { private final String prefix; private final String exportTaskId; private final String databaseName; + + /** + * schemaName is specific for Postgres; For MySQL, this schemaName has the same value as databaseName + */ + private final String schemaName; + private final String tableName; private final String numberedFolder; private final String fileName; - ExportObjectKey(final String prefix, final String exportTaskId, final String databaseName, final String tableName, final String numberedFolder, final String fileName) { + ExportObjectKey(final String prefix, final String exportTaskId, final String databaseName, final String schemaName, final String tableName, final String numberedFolder, final String fileName) { this.prefix = prefix; this.exportTaskId = exportTaskId; this.databaseName = databaseName; + this.schemaName = schemaName; this.tableName = tableName; this.numberedFolder = numberedFolder; this.fileName = fileName; @@ -42,13 +49,14 @@ public static ExportObjectKey fromString(final String objectKeyString) { .collect(Collectors.joining(S3_PATH_DELIMITER)); final String exportTaskId = parts[parts.length - 5]; final String databaseName = parts[parts.length - 4]; - // fullTableName is in the format of "databaseName.tableName" + // fullTableName is in the format of "databaseName.tableName" for MySQL and "schemaName.tableName" for Postgres final String fullTableName = parts[parts.length - 3]; + final String schemaName = fullTableName.split("\\.")[0]; final String tableName = fullTableName.split("\\.")[1]; final String numberedFolder = parts[parts.length - 2]; final String fileName = parts[parts.length - 1]; - return new ExportObjectKey(prefix, exportTaskId, databaseName, tableName, numberedFolder, fileName); + return new ExportObjectKey(prefix, exportTaskId, databaseName, schemaName, tableName, numberedFolder, fileName); } public String getPrefix() { @@ -63,6 +71,14 @@ public String getDatabaseName() { return databaseName; } + /** + * schemaName is specific for Postgres; For MySQL, this schemaName has the same value as databaseName + * @return schemaName + */ + public String getSchemaName() { + return schemaName; + } + public String getTableName() { return tableName; } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java index 8eb3b9cde9..95838ff586 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClient.java @@ -85,15 +85,20 @@ public void connect() { stream.setAppliedLSN(lsn); } catch (Exception e) { LOG.error("Exception while processing Postgres replication stream. ", e); + throw e; } } } stream.close(); disconnectRequested = false; + if (eventProcessor != null) { + eventProcessor.stopCheckpointManager(); + } LOG.debug("Replication stream closed successfully."); } catch (Exception e) { LOG.error("Exception while creating Postgres replication stream. ", e); + throw new RuntimeException(e); } } diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java index a2a9aa1017..f49f2acc66 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationEventProcessor.java @@ -173,12 +173,16 @@ public void process(ByteBuffer msg) { public void stopClient() { try { logicalReplicationClient.disconnect(); - LOG.info("Binary log client disconnected."); + LOG.info("Logical replication client disconnected."); } catch (Exception e) { - LOG.error("Binary log client failed to disconnect.", e); + LOG.error("Logical replication client failed to disconnect.", e); } } + public void stopCheckpointManager() { + streamCheckpointManager.stop(); + } + void processBeginMessage(ByteBuffer msg) { currentLsn = msg.getLong(); long epochMicro = msg.getLong(); diff --git a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java index 4da8798c90..d1f5c8b31c 100644 --- a/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java +++ b/data-prepper-plugins/rds-source/src/main/java/org/opensearch/dataprepper/plugins/source/rds/stream/StreamWorker.java @@ -62,6 +62,8 @@ public void processStream(final StreamPartition streamPartition) { LOG.info("Connect to database to read change events."); replicationLogClient.connect(); } catch (Exception e) { + LOG.warn("Error while connecting to replication stream, will retry."); + sourceCoordinator.giveUpPartition(streamPartition); throw new RuntimeException(e); } finally { try { diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java index 060e3e9a29..e75f3dc28f 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/leader/LeaderSchedulerTest.java @@ -9,12 +9,15 @@ import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; import org.mockito.Answers; import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; import org.opensearch.dataprepper.plugins.source.rds.RdsSourceConfig; import org.opensearch.dataprepper.plugins.source.rds.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.rds.configuration.EngineType; import org.opensearch.dataprepper.plugins.source.rds.configuration.ExportConfig; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.ExportPartition; import org.opensearch.dataprepper.plugins.source.rds.coordination.partition.GlobalState; @@ -92,12 +95,14 @@ void non_leader_node_should_not_perform_init() throws InterruptedException { verify(sourceCoordinator, never()).createPartition(any(ExportPartition.class)); } - @Test - void leader_node_should_perform_init_if_not_initialized() throws InterruptedException { + @ParameterizedTest + @EnumSource(EngineType.class) + void leader_node_should_perform_init_if_not_initialized(EngineType engineType) throws InterruptedException { when(sourceCoordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)).thenReturn(Optional.of(leaderPartition)); when(leaderPartition.getProgressState()).thenReturn(Optional.of(leaderProgressState)); when(leaderProgressState.isInitialized()).thenReturn(false); when(sourceConfig.isExportEnabled()).thenReturn(true); + when(sourceConfig.getEngine()).thenReturn(engineType); final ExecutorService executorService = Executors.newSingleThreadExecutor(); executorService.submit(leaderScheduler); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java index 18a66bd6e2..1a207a9031 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/model/ExportObjectKeyTest.java @@ -15,13 +15,28 @@ class ExportObjectKeyTest { @Test - void test_fromString_with_valid_input_string() { + void test_fromString_with_valid_input_string_mysql() { final String objectKeyString = "prefix/export-task-id/db-name/db-name.table-name/1/file-name.parquet"; final ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKeyString); assertThat(exportObjectKey.getPrefix(), equalTo("prefix")); assertThat(exportObjectKey.getExportTaskId(), equalTo("export-task-id")); assertThat(exportObjectKey.getDatabaseName(), equalTo("db-name")); + assertThat(exportObjectKey.getSchemaName(), equalTo("db-name")); + assertThat(exportObjectKey.getTableName(), equalTo("table-name")); + assertThat(exportObjectKey.getNumberedFolder(), equalTo("1")); + assertThat(exportObjectKey.getFileName(), equalTo("file-name.parquet")); + } + + @Test + void test_fromString_with_valid_input_string_postgres() { + final String objectKeyString = "prefix/export-task-id/db-name/schema-name.table-name/1/file-name.parquet"; + final ExportObjectKey exportObjectKey = ExportObjectKey.fromString(objectKeyString); + + assertThat(exportObjectKey.getPrefix(), equalTo("prefix")); + assertThat(exportObjectKey.getExportTaskId(), equalTo("export-task-id")); + assertThat(exportObjectKey.getDatabaseName(), equalTo("db-name")); + assertThat(exportObjectKey.getSchemaName(), equalTo("schema-name")); assertThat(exportObjectKey.getTableName(), equalTo("table-name")); assertThat(exportObjectKey.getNumberedFolder(), equalTo("1")); assertThat(exportObjectKey.getFileName(), equalTo("file-name.parquet")); @@ -35,6 +50,7 @@ void test_fromString_with_path_with_empty_prefix() { assertThat(exportObjectKey.getPrefix(), equalTo("")); assertThat(exportObjectKey.getExportTaskId(), equalTo("export-task-id")); assertThat(exportObjectKey.getDatabaseName(), equalTo("db-name")); + assertThat(exportObjectKey.getSchemaName(), equalTo("db-name")); assertThat(exportObjectKey.getTableName(), equalTo("table-name")); assertThat(exportObjectKey.getNumberedFolder(), equalTo("1")); assertThat(exportObjectKey.getFileName(), equalTo("file-name.parquet")); @@ -48,6 +64,7 @@ void test_fromString_with_path_with_multilevel_prefix() { assertThat(exportObjectKey.getPrefix(), equalTo("prefix1/prefix2/prefix3")); assertThat(exportObjectKey.getExportTaskId(), equalTo("export-task-id")); assertThat(exportObjectKey.getDatabaseName(), equalTo("db-name")); + assertThat(exportObjectKey.getSchemaName(), equalTo("db-name")); assertThat(exportObjectKey.getTableName(), equalTo("table-name")); assertThat(exportObjectKey.getNumberedFolder(), equalTo("1")); assertThat(exportObjectKey.getFileName(), equalTo("file-name.parquet")); diff --git a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java index 45897335b5..be87bf5bda 100644 --- a/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java +++ b/data-prepper-plugins/rds-source/src/test/java/org/opensearch/dataprepper/plugins/source/rds/stream/LogicalReplicationClientTest.java @@ -30,6 +30,7 @@ import java.util.concurrent.Executors; import static org.awaitility.Awaitility.await; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.anyString; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.mock; @@ -89,6 +90,18 @@ void test_connect() throws SQLException, InterruptedException { verify(stream).setFlushedLSN(lsn); } + @Test + void test_connect_exception_should_throw() throws SQLException { + when(connectionManager.getConnection()).thenThrow(RuntimeException.class); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> logicalReplicationClient.connect()); + + assertThrows(RuntimeException.class, () -> logicalReplicationClient.connect()); + + executorService.shutdownNow(); + } + @Test void test_disconnect() throws SQLException, InterruptedException { final Connection connection = mock(Connection.class); @@ -119,6 +132,7 @@ void test_disconnect() throws SQLException, InterruptedException { logicalReplicationClient.disconnect(); Thread.sleep(20); verify(stream).close(); + verify(eventProcessor).stopCheckpointManager(); verifyNoMoreInteractions(stream, eventProcessor); executorService.shutdownNow(); @@ -155,6 +169,7 @@ void test_connect_disconnect_cycles() throws SQLException, InterruptedException logicalReplicationClient.disconnect(); Thread.sleep(20); verify(stream).close(); + verify(eventProcessor).stopCheckpointManager(); verifyNoMoreInteractions(stream, eventProcessor); // Second connect @@ -170,6 +185,7 @@ void test_connect_disconnect_cycles() throws SQLException, InterruptedException logicalReplicationClient.disconnect(); Thread.sleep(20); verify(stream, times(2)).close(); + verify(eventProcessor, times(2)).stopCheckpointManager(); verifyNoMoreInteractions(stream, eventProcessor); executorService.shutdownNow(); From 1ce9dbf3f12cbaa1042bfc544b3bdfc8343fe85c Mon Sep 17 00:00:00 2001 From: Taylor Gray Date: Fri, 7 Feb 2025 10:47:12 -0600 Subject: [PATCH 16/18] Add max_receive_count configuration option in S3-SQS source to delete messages that have been received many times (#5408) Signed-off-by: Taylor Gray --- .../plugins/source/s3/SqsWorker.java | 29 ++++++++++++-- .../source/s3/configuration/SqsOptions.java | 8 ++++ .../source/s3/parser/ParsedMessage.java | 10 +++++ .../plugins/source/s3/SqsWorkerTest.java | 38 +++++++++++++++++++ .../source/s3/parser/ParsedMessageTest.java | 1 + 5 files changed, 82 insertions(+), 4 deletions(-) diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java index 81c7f9ca9c..14451be31a 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorker.java @@ -30,6 +30,7 @@ import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResultEntry; import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.MessageSystemAttributeName; import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; import software.amazon.awssdk.services.sqs.model.SqsException; import software.amazon.awssdk.services.sts.model.StsException; @@ -189,6 +190,7 @@ private ReceiveMessageRequest createReceiveMessageRequest() { .maxNumberOfMessages(sqsOptions.getMaximumMessages()) .visibilityTimeout((int) sqsOptions.getVisibilityTimeout().getSeconds()) .waitTimeSeconds((int) sqsOptions.getWaitTime().getSeconds()) + .attributeNamesWithStrings(MessageSystemAttributeName.APPROXIMATE_RECEIVE_COUNT.toString()) .build(); } @@ -232,10 +234,20 @@ && isEventBridgeEventTypeCreated(parsedMessage)) { LOG.info("Received {} messages from SQS. Processing {} messages.", s3EventNotificationRecords.size(), parsedMessagesToRead.size()); for (ParsedMessage parsedMessage : parsedMessagesToRead) { - sqsMessageDelayTimer.record(Duration.between( - Instant.ofEpochMilli(parsedMessage.getEventTime().toInstant().getMillis()), - Instant.now() - )); + final int approximateReceiveCount = getApproximateReceiveCount(parsedMessage.getMessage()); + if (s3SourceConfig.getSqsOptions().getMaxReceiveAttempts() != null && + approximateReceiveCount > s3SourceConfig.getSqsOptions().getMaxReceiveAttempts()) { + deleteSqsMessages(List.of(buildDeleteMessageBatchRequestEntry(parsedMessage.getMessage()))); + parsedMessage.setShouldSkipProcessing(true); + continue; + } + + if (approximateReceiveCount <= 1) { + sqsMessageDelayTimer.record(Duration.between( + Instant.ofEpochMilli(parsedMessage.getEventTime().toInstant().getMillis()), + Instant.now() + )); + } List waitingForAcknowledgements = new ArrayList<>(); List s3ObjectDeletionWaitingForAcknowledgments = new ArrayList<>(); AcknowledgementSet acknowledgementSet = null; @@ -288,6 +300,10 @@ && isEventBridgeEventTypeCreated(parsedMessage)) { // Use a separate loop for processing the S3 objects for (ParsedMessage parsedMessage : parsedMessagesToRead) { + if (parsedMessage.isShouldSkipProcessing()) { + continue; + } + final AcknowledgementSet acknowledgementSet = messageAcknowledgementSetMap.get(parsedMessage); final List waitingForAcknowledgements = messageWaitingForAcknowledgementsMap.get(parsedMessage); final List s3ObjectDeletionsWaitingForAcknowledgments = messagesWaitingForS3ObjectDeletion.get(parsedMessage); @@ -428,6 +444,11 @@ private S3ObjectReference populateS3Reference(final String bucketName, final Str .build(); } + private int getApproximateReceiveCount(final Message message) { + return message.attributes() != null && message.attributes().get(MessageSystemAttributeName.APPROXIMATE_RECEIVE_COUNT) != null ? + Integer.parseInt(message.attributes().get(MessageSystemAttributeName.APPROXIMATE_RECEIVE_COUNT)) : 0; + } + void stop() { isStopped = true; } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/configuration/SqsOptions.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/configuration/SqsOptions.java index 1242a6525b..8d4eebdd08 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/configuration/SqsOptions.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/configuration/SqsOptions.java @@ -6,6 +6,7 @@ package org.opensearch.dataprepper.plugins.source.s3.configuration; import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyDescription; import jakarta.validation.constraints.Max; import jakarta.validation.constraints.Min; import jakarta.validation.constraints.NotBlank; @@ -53,6 +54,11 @@ public class SqsOptions { @DurationMin(seconds = 0) private Duration pollDelay = DEFAULT_POLL_DELAY_SECONDS; + @JsonPropertyDescription("Messages that contain an ApproximateReceiveCount greater than this value will be deleted") + @JsonProperty("max_receive_attempts") + @Min(1) + private Integer maxReceiveAttempts; + public String getSqsUrl() { return sqsUrl; } @@ -80,4 +86,6 @@ public Duration getWaitTime() { public Duration getPollDelay() { return pollDelay; } + + public Integer getMaxReceiveAttempts() { return maxReceiveAttempts; } } diff --git a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessage.java b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessage.java index ed68dff063..095e244f1d 100644 --- a/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessage.java +++ b/data-prepper-plugins/s3-source/src/main/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessage.java @@ -24,6 +24,8 @@ public class ParsedMessage { private boolean emptyNotification; private String detailType; + private boolean shouldSkipProcessing; + public ParsedMessage(final Message message, final boolean failedParsing) { this.message = Objects.requireNonNull(message); this.failedParsing = failedParsing; @@ -40,6 +42,7 @@ public ParsedMessage(final Message message, final boolean failedParsing) { this.eventTime = notificationRecords.get(0).getEventTime(); this.failedParsing = false; this.emptyNotification = notificationRecords.isEmpty(); + this.shouldSkipProcessing = false; } ParsedMessage(final Message message, final S3EventBridgeNotification eventBridgeNotification) { @@ -49,6 +52,7 @@ public ParsedMessage(final Message message, final boolean failedParsing) { this.objectSize = eventBridgeNotification.getDetail().getObject().getSize(); this.detailType = eventBridgeNotification.getDetailType(); this.eventTime = eventBridgeNotification.getTime(); + this.shouldSkipProcessing = false; } public Message getMessage() { @@ -87,6 +91,12 @@ public String getDetailType() { return detailType; } + public boolean isShouldSkipProcessing () { return shouldSkipProcessing; } + + public void setShouldSkipProcessing(final boolean shouldSkipProcessing) { + this.shouldSkipProcessing = shouldSkipProcessing; + } + @Override public String toString() { return "Message{" + diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerTest.java index 8c0522619e..c506d295c3 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/SqsWorkerTest.java @@ -40,6 +40,7 @@ import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResponse; import software.amazon.awssdk.services.sqs.model.DeleteMessageBatchResultEntry; import software.amazon.awssdk.services.sqs.model.Message; +import software.amazon.awssdk.services.sqs.model.MessageSystemAttributeName; import software.amazon.awssdk.services.sqs.model.ReceiveMessageRequest; import software.amazon.awssdk.services.sqs.model.ReceiveMessageResponse; import software.amazon.awssdk.services.sqs.model.SqsException; @@ -53,6 +54,7 @@ import java.time.temporal.ChronoUnit; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -202,6 +204,42 @@ void processSqsMessages_should_return_number_of_messages_processed(final String assertThat(actualDelay, greaterThanOrEqualTo(Duration.ofHours(1).minus(Duration.ofSeconds(5)))); } + @Test + void processSqsMessages_with_max_receive_count_reached_deletes_message_and_skips_processing() throws IOException { + when(sqsOptions.getMaxReceiveAttempts()).thenReturn(4); + + final String eventName = "ObjectCreated:Put"; + Instant startTime = Instant.now().minus(1, ChronoUnit.HOURS); + final Message message = mock(Message.class); + when(message.attributes()).thenReturn(Map.of(MessageSystemAttributeName.APPROXIMATE_RECEIVE_COUNT, "5")); + when(message.body()).thenReturn(createEventNotification(eventName, startTime)); + final String testReceiptHandle = UUID.randomUUID().toString(); + when(message.messageId()).thenReturn(testReceiptHandle); + when(message.receiptHandle()).thenReturn(testReceiptHandle); + + final ReceiveMessageResponse receiveMessageResponse = mock(ReceiveMessageResponse.class); + when(sqsClient.receiveMessage(any(ReceiveMessageRequest.class))).thenReturn(receiveMessageResponse); + when(receiveMessageResponse.messages()).thenReturn(Collections.singletonList(message)); + + final int messagesProcessed = createObjectUnderTest().processSqsMessages(); + final ArgumentCaptor deleteMessageBatchRequestArgumentCaptor = ArgumentCaptor.forClass(DeleteMessageBatchRequest.class); + verify(sqsClient).deleteMessageBatch(deleteMessageBatchRequestArgumentCaptor.capture()); + final DeleteMessageBatchRequest actualDeleteMessageBatchRequest = deleteMessageBatchRequestArgumentCaptor.getValue(); + + verifyNoInteractions(sqsMessageDelayTimer); + + assertThat(actualDeleteMessageBatchRequest, notNullValue()); + assertThat(actualDeleteMessageBatchRequest.entries().size(), equalTo(1)); + assertThat(actualDeleteMessageBatchRequest.queueUrl(), equalTo(s3SourceConfig.getSqsOptions().getSqsUrl())); + assertThat(actualDeleteMessageBatchRequest.entries().get(0).id(), equalTo(message.messageId())); + assertThat(actualDeleteMessageBatchRequest.entries().get(0).receiptHandle(), equalTo(message.receiptHandle())); + assertThat(messagesProcessed, equalTo(1)); + verifyNoInteractions(s3Service); + verify(sqsClient).deleteMessageBatch(any(DeleteMessageBatchRequest.class)); + verify(sqsMessagesReceivedCounter).increment(1); + verify(sqsMessagesDeletedCounter).increment(1); + } + @Test void processSqsMessages_should_not_interact_with_S3Service_and_delete_message_if_TestEvent() { final String messageId = UUID.randomUUID().toString(); diff --git a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessageTest.java b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessageTest.java index 51f3abad06..4a33f8f12a 100644 --- a/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessageTest.java +++ b/data-prepper-plugins/s3-source/src/test/java/org/opensearch/dataprepper/plugins/source/s3/parser/ParsedMessageTest.java @@ -120,6 +120,7 @@ void test_parsed_message_with_S3EventNotificationRecord() { assertThat(parsedMessage.getEventTime(), equalTo(testEventTime)); assertThat(parsedMessage.isFailedParsing(), equalTo(false)); assertThat(parsedMessage.isEmptyNotification(), equalTo(false)); + assertThat(parsedMessage.isShouldSkipProcessing(), equalTo(false)); } @Test From 815ddc0041f3c7fee8f23f6123e6e15187db6889 Mon Sep 17 00:00:00 2001 From: David Venable Date: Fri, 7 Feb 2025 12:23:15 -0800 Subject: [PATCH 17/18] Set the JVM file encoding to UTF-8. Resolves #5238. (#5420) Signed-off-by: David Venable --- release/archives/linux/data-prepper-jdk-x64.sh | 2 +- release/archives/linux/data-prepper-x64.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/release/archives/linux/data-prepper-jdk-x64.sh b/release/archives/linux/data-prepper-jdk-x64.sh index 1622e597cb..18c8951b09 100644 --- a/release/archives/linux/data-prepper-jdk-x64.sh +++ b/release/archives/linux/data-prepper-jdk-x64.sh @@ -34,7 +34,7 @@ echo "JAVA_HOME is set to $JAVA_HOME" export PATH=$JAVA_HOME/bin:$PATH DATA_PREPPER_HOME_OPTS="-Ddata-prepper.dir=$DATA_PREPPER_HOME" -DATA_PREPPER_JAVA_OPTS="-Dlog4j.configurationFile=$DATA_PREPPER_HOME/config/log4j2-rolling.properties" +DATA_PREPPER_JAVA_OPTS="-Dfile.encoding=UTF-8 -Dlog4j.configurationFile=$DATA_PREPPER_HOME/config/log4j2-rolling.properties" if [[ $# == 0 ]]; then exec java $DATA_PREPPER_JAVA_OPTS $JAVA_OPTS $DATA_PREPPER_HOME_OPTS -cp "$DATA_PREPPER_CLASSPATH" org.opensearch.dataprepper.DataPrepperExecute diff --git a/release/archives/linux/data-prepper-x64.sh b/release/archives/linux/data-prepper-x64.sh index 1bfdbbb051..6650702670 100755 --- a/release/archives/linux/data-prepper-x64.sh +++ b/release/archives/linux/data-prepper-x64.sh @@ -62,7 +62,7 @@ then fi DATA_PREPPER_HOME_OPTS="-Ddata-prepper.dir=$DATA_PREPPER_HOME" -DATA_PREPPER_JAVA_OPTS="-Dlog4j.configurationFile=$DATA_PREPPER_HOME/config/log4j2-rolling.properties" +DATA_PREPPER_JAVA_OPTS="-Dfile.encoding=UTF-8 -Dlog4j.configurationFile=$DATA_PREPPER_HOME/config/log4j2-rolling.properties" if [[ $# == 0 ]]; then exec java $DATA_PREPPER_JAVA_OPTS $JAVA_OPTS $DATA_PREPPER_HOME_OPTS -cp "$DATA_PREPPER_CLASSPATH" org.opensearch.dataprepper.DataPrepperExecute From b3b6c650c1b564bec253ea5fa7bc1b34b16b903b Mon Sep 17 00:00:00 2001 From: Mohammed Aghil Puthiyottil <57040494+MohammedAghil@users.noreply.github.com> Date: Fri, 7 Feb 2025 17:16:27 -0800 Subject: [PATCH 18/18] Zero Buffer Implementation and Tests (#5416) * Zero Buffer Implementation and Tests Signed-off-by: Mohammed Aghil Puthiyottil <57040494+MohammedAghil@users.noreply.github.com> * Moved ZeroBuffer Implementation into data-prepper-core and addressed comments Signed-off-by: Mohammed Aghil Puthiyottil <57040494+MohammedAghil@users.noreply.github.com> * Modified ZeroBufferTests to use MockitoExtension and addressed comments Signed-off-by: Mohammed Aghil Puthiyottil <57040494+MohammedAghil@users.noreply.github.com> --------- Signed-off-by: Mohammed Aghil Puthiyottil <57040494+MohammedAghil@users.noreply.github.com> --- .../core/pipeline/PipelineRunner.java | 5 + .../core/pipeline/SupportsPipelineRunner.java | 7 + .../core/pipeline/buffer/ZeroBuffer.java | 110 +++++++ .../core/pipeline/buffer/ZeroBufferTests.java | 276 ++++++++++++++++++ 4 files changed, 398 insertions(+) create mode 100644 data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/PipelineRunner.java create mode 100644 data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/SupportsPipelineRunner.java create mode 100644 data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/buffer/ZeroBuffer.java create mode 100644 data-prepper-core/src/test/java/org/opensearch/dataprepper/core/pipeline/buffer/ZeroBufferTests.java diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/PipelineRunner.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/PipelineRunner.java new file mode 100644 index 0000000000..be7a4595cc --- /dev/null +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/PipelineRunner.java @@ -0,0 +1,5 @@ +package org.opensearch.dataprepper.core.pipeline; + +public interface PipelineRunner { + void runAllProcessorsAndPublishToSinks(); +} diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/SupportsPipelineRunner.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/SupportsPipelineRunner.java new file mode 100644 index 0000000000..2496293f7c --- /dev/null +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/SupportsPipelineRunner.java @@ -0,0 +1,7 @@ +package org.opensearch.dataprepper.core.pipeline; + +public interface SupportsPipelineRunner { + PipelineRunner getPipelineRunner(); + + void setPipelineRunner(PipelineRunner pipelineRunner); +} diff --git a/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/buffer/ZeroBuffer.java b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/buffer/ZeroBuffer.java new file mode 100644 index 0000000000..cebed1b6a3 --- /dev/null +++ b/data-prepper-core/src/main/java/org/opensearch/dataprepper/core/pipeline/buffer/ZeroBuffer.java @@ -0,0 +1,110 @@ +package org.opensearch.dataprepper.core.pipeline.buffer; + +import com.google.common.annotations.VisibleForTesting; +import org.opensearch.dataprepper.core.pipeline.PipelineRunner; +import org.opensearch.dataprepper.core.pipeline.SupportsPipelineRunner; +import org.opensearch.dataprepper.metrics.MetricNames; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.CheckpointState; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.record.Record; +import io.micrometer.core.instrument.Counter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.TimeoutException; + +@DataPrepperPlugin(name = "zero", pluginType = Buffer.class) +public class ZeroBuffer> implements Buffer, SupportsPipelineRunner { + private static final Logger LOG = LoggerFactory.getLogger(ZeroBuffer.class); + private static final String PLUGIN_COMPONENT_ID = "ZeroBuffer"; + private final PluginMetrics pluginMetrics; + private final ThreadLocal> threadLocalStore; + private PipelineRunner pipelineRunner; + @VisibleForTesting + final String pipelineName; + private final Counter writeRecordsCounter; + private final Counter readRecordsCounter; + + @DataPrepperPluginConstructor + public ZeroBuffer(PipelineDescription pipelineDescription) { + this.pluginMetrics = PluginMetrics.fromNames(PLUGIN_COMPONENT_ID, pipelineDescription.getPipelineName()); + this.pipelineName = pipelineDescription.getPipelineName(); + this.threadLocalStore = new ThreadLocal<>(); + this.writeRecordsCounter = pluginMetrics.counter(MetricNames.RECORDS_WRITTEN); + this.readRecordsCounter = pluginMetrics.counter(MetricNames.RECORDS_READ); + } + + @Override + public void write(T record, int timeoutInMillis) throws TimeoutException { + if (record == null) { + throw new NullPointerException("The write record cannot be null"); + } + + if (threadLocalStore.get() == null) { + threadLocalStore.set(new ArrayList<>()); + } + + threadLocalStore.get().add(record); + writeRecordsCounter.increment(); + + getPipelineRunner().runAllProcessorsAndPublishToSinks(); + } + + @Override + public void writeAll(Collection records, int timeoutInMillis) throws Exception { + if (records == null) { + throw new NullPointerException("The write records cannot be null"); + } + + if (threadLocalStore.get() == null) { + threadLocalStore.set(new ArrayList<>(records)); + } else { + // Add the new records to the existing records + threadLocalStore.get().addAll(records); + } + + writeRecordsCounter.increment((double) records.size()); + getPipelineRunner().runAllProcessorsAndPublishToSinks(); + } + + @Override + public Map.Entry, CheckpointState> read(int timeoutInMillis) { + if (threadLocalStore.get() == null) { + threadLocalStore.set(new ArrayList<>()); + } + + Collection storedRecords = threadLocalStore.get(); + CheckpointState checkpointState = new CheckpointState(0); + if (storedRecords!= null && !storedRecords.isEmpty()) { + checkpointState = new CheckpointState(storedRecords.size()); + threadLocalStore.remove(); + readRecordsCounter.increment((double) storedRecords.size()); + } + + return Map.entry(storedRecords, checkpointState); + } + + @Override + public void checkpoint(CheckpointState checkpointState) {} + + @Override + public boolean isEmpty() { + return (this.threadLocalStore.get() == null || this.threadLocalStore.get().isEmpty()); + } + + @Override + public PipelineRunner getPipelineRunner() { + return pipelineRunner; + } + + @Override + public void setPipelineRunner(PipelineRunner pipelineRunner) { + this.pipelineRunner = pipelineRunner; + } +} diff --git a/data-prepper-core/src/test/java/org/opensearch/dataprepper/core/pipeline/buffer/ZeroBufferTests.java b/data-prepper-core/src/test/java/org/opensearch/dataprepper/core/pipeline/buffer/ZeroBufferTests.java new file mode 100644 index 0000000000..653b2b4f8c --- /dev/null +++ b/data-prepper-core/src/test/java/org/opensearch/dataprepper/core/pipeline/buffer/ZeroBufferTests.java @@ -0,0 +1,276 @@ +package org.opensearch.dataprepper.core.pipeline.buffer; + +import io.micrometer.core.instrument.Metrics; +import io.micrometer.core.instrument.simple.SimpleMeterRegistry; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.mockito.Mockito.mock; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.core.pipeline.PipelineRunner; +import org.opensearch.dataprepper.model.CheckpointState; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.ArrayList; +import java.util.Collection; +import java.util.concurrent.TimeoutException; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +@ExtendWith(MockitoExtension.class) +public class ZeroBufferTests { + private static final Logger LOG = LoggerFactory.getLogger(ZeroBufferTests.class); + private static final String MOCK_PIPELINE_NAME = "mock-pipeline"; + private static final int WRITE_TIMEOUT = 100; + private static final int READ_TIMEOUT = 500; + private static final String SINGLE_RECORD_DATA_FORMAT = "{\"message\":\"test\"}"; + private static final String BATCH_RECORDS_DATA_FORMAT = "{\"message\":\"test-%d\"}"; + @Mock + PipelineDescription pipelineDescription; + @Mock + PipelineRunner pipelineRunner; + + + @BeforeEach + public void setup() { + new ArrayList<>(Metrics.globalRegistry.getRegistries()) + .forEach(Metrics.globalRegistry::remove); + new ArrayList<>(Metrics.globalRegistry.getMeters()) + .forEach(Metrics.globalRegistry::remove); + Metrics.addRegistry(new SimpleMeterRegistry()); + } + + @Nested + class WriteTests { + @Test + public void testSingleWriteAndReadReturnsCorrectRecord() throws Exception { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + doNothing().when(pipelineRunner).runAllProcessorsAndPublishToSinks(); + + zeroBuffer.write(generateRecord(SINGLE_RECORD_DATA_FORMAT), WRITE_TIMEOUT); + + Collection> readRecords = zeroBuffer.read(READ_TIMEOUT).getKey(); + assertEquals(1, readRecords.size()); + + assertEquals(SINGLE_RECORD_DATA_FORMAT, readRecords.iterator().next().getData()); + + zeroBuffer.write(generateRecord(SINGLE_RECORD_DATA_FORMAT), WRITE_TIMEOUT); + readRecords = zeroBuffer.read(READ_TIMEOUT).getKey(); + assertEquals(1, readRecords.size()); + verify(pipelineRunner, times(2)).runAllProcessorsAndPublishToSinks(); + } + + @Test + public void testMultipleWriteAndReadReturnsCorrectRecord() throws Exception { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + doNothing().when(pipelineRunner).runAllProcessorsAndPublishToSinks(); + + zeroBuffer.write(generateRecord(SINGLE_RECORD_DATA_FORMAT), WRITE_TIMEOUT); + zeroBuffer.write(generateRecord(SINGLE_RECORD_DATA_FORMAT), WRITE_TIMEOUT); + + Collection> readRecords = zeroBuffer.read(READ_TIMEOUT).getKey(); + assertEquals(2, readRecords.size()); + assertEquals(SINGLE_RECORD_DATA_FORMAT, readRecords.iterator().next().getData()); + assertEquals(SINGLE_RECORD_DATA_FORMAT, readRecords.iterator().next().getData()); + verify(pipelineRunner, times(2)).runAllProcessorsAndPublishToSinks(); + } + + @Test + public void testWriteAllAndReadReturnsAllRecords() throws Exception { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + doNothing().when(pipelineRunner).runAllProcessorsAndPublishToSinks(); + + Collection> writeRecords = generateRecords(IntStream.range(0, 10) + .mapToObj(i -> String.format(BATCH_RECORDS_DATA_FORMAT, i)) + .collect(Collectors.toList())); + zeroBuffer.writeAll(writeRecords, WRITE_TIMEOUT); + + Map.Entry>, CheckpointState> readRecordsMap = zeroBuffer.read(READ_TIMEOUT); + Collection> readRecords = readRecordsMap.getKey(); + for (Record record : readRecords) { + LOG.debug(record.getData()); + } + + // Ensure that the write records are the same as the read records + assertEquals(writeRecords.size(), readRecords.size()); + verify(pipelineRunner).runAllProcessorsAndPublishToSinks(); + } + + @Test + public void testWriteNullRecordThrowsException() { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + + Exception writeException = assertThrows(NullPointerException.class, () -> { + zeroBuffer.write(null, WRITE_TIMEOUT); + }); + + Exception writeAllException = assertThrows(NullPointerException.class, () -> { + zeroBuffer.writeAll(null, WRITE_TIMEOUT); + }); + + assertEquals("The write record cannot be null", writeException.getMessage()); + assertEquals("The write records cannot be null", writeAllException.getMessage()); + verify(pipelineRunner, never()).runAllProcessorsAndPublishToSinks(); + } + + @Test + public void testWriteEmptyRecordDoesNotThrowException() { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + doNothing().when(pipelineRunner).runAllProcessorsAndPublishToSinks(); + + Record emptyRecord = generateRecord(null); + Collection> emptyRecordCollection = generateRecords(new ArrayList<>()); + + assertDoesNotThrow(() -> zeroBuffer.write(emptyRecord, WRITE_TIMEOUT)); + assertDoesNotThrow(() -> zeroBuffer.writeAll(emptyRecordCollection, WRITE_TIMEOUT)); + verify(pipelineRunner, times(2)).runAllProcessorsAndPublishToSinks(); + } + + @Test + public void testThreadReadAndWriteIsolation() throws Exception { + final ZeroBuffer> zeroBuffer = createObjectUnderTestWithPipelineName(); + + Thread workerThread = new Thread(() -> { + try { + PipelineRunner pipelineRunnerMock = mock(PipelineRunner.class); + zeroBuffer.setPipelineRunner(pipelineRunnerMock); + doNothing().when(pipelineRunnerMock).runAllProcessorsAndPublishToSinks(); + zeroBuffer.write(generateRecord(SINGLE_RECORD_DATA_FORMAT), WRITE_TIMEOUT); + verify(pipelineRunnerMock).runAllProcessorsAndPublishToSinks(); + } catch (TimeoutException e) { + fail("Timeout exception occurred"); + } + }); + workerThread.start(); + workerThread.join(); + + // Ensure that main thread does not share the same records store as the worker thread + assertEquals(0, zeroBuffer.read(READ_TIMEOUT).getKey().size()); + assertTrue(zeroBuffer.isEmpty()); + verify(pipelineRunner, never()).runAllProcessorsAndPublishToSinks(); + } + + @Test + public void testWriteAndWriteAllReturnsCorrectRecords() throws Exception { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + doNothing().when(pipelineRunner).runAllProcessorsAndPublishToSinks(); + + zeroBuffer.write(generateRecord(SINGLE_RECORD_DATA_FORMAT), WRITE_TIMEOUT); + zeroBuffer.writeAll(generateRecords(IntStream.range(0, 10) + .mapToObj(i -> String.format(BATCH_RECORDS_DATA_FORMAT, i)) + .collect(Collectors.toList())), WRITE_TIMEOUT); + + Collection> readRecords = zeroBuffer.read(READ_TIMEOUT).getKey(); + for (Record record : readRecords) { + LOG.debug(record.getData()); + } + assertEquals(11, readRecords.size()); + verify(pipelineRunner, times(2)).runAllProcessorsAndPublishToSinks(); + } + } + + @Nested + class ReadTests { + @Test + public void testReadFromNonEmptyBufferReturnsCorrectRecords() throws Exception { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + + zeroBuffer.write(generateRecord(SINGLE_RECORD_DATA_FORMAT), WRITE_TIMEOUT); + + Collection> initialReadRecords = zeroBuffer.read(READ_TIMEOUT).getKey(); + Collection> secondAttemptToReadRecords = zeroBuffer.read(READ_TIMEOUT).getKey(); + + assertEquals(1, initialReadRecords.size()); + assertEquals(SINGLE_RECORD_DATA_FORMAT, initialReadRecords.iterator().next().getData()); + + assertEquals(0, secondAttemptToReadRecords.size()); + verify(pipelineRunner).runAllProcessorsAndPublishToSinks(); + } + + @Test + public void testReadFromEmptyBufferReturnsNoRecords() { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + + Map.Entry>, CheckpointState> readRecordsMap = zeroBuffer.read(READ_TIMEOUT); + assertTrue(readRecordsMap.getKey().isEmpty()); + verify(pipelineRunner, never()).runAllProcessorsAndPublishToSinks(); + } + } + + @Nested + class EmptyBufferTests { + @Test + public void testIsEmptyReturnsTrueWhenBufferIsEmpty() { + ZeroBuffer> zeroBuffer = createObjectUnderTestWithPipelineName(); + assertTrue(zeroBuffer.isEmpty()); + verify(pipelineRunner, never()).runAllProcessorsAndPublishToSinks(); + } + + @Test + public void testIsEmptyReturnsFalseWhenBufferIsNotEmpty() throws Exception { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + + zeroBuffer.write(generateRecord(SINGLE_RECORD_DATA_FORMAT), WRITE_TIMEOUT); + + assertFalse(zeroBuffer.isEmpty()); + verify(pipelineRunner).runAllProcessorsAndPublishToSinks(); + } + } + + @Nested + class CommonTests { + @Test + public void testCreateZeroBufferWithPipelineName() { + ZeroBuffer> zeroBuffer = createObjectUnderTestWithPipelineName(); + assertEquals(MOCK_PIPELINE_NAME, zeroBuffer.pipelineName); + } + + @Test + public void testCheckpointDoesNotThrowException() { + ZeroBuffer> zeroBuffer = createObjectUnderTest(); + assertDoesNotThrow(() -> zeroBuffer.checkpoint(null)); + assertDoesNotThrow(() -> zeroBuffer.checkpoint(new CheckpointState(0))); + } + } + + /*-------------------------Private Helper Methods---------------------------*/ + private Record generateRecord(final T data) { + return new Record<>(data); + } + + private Collection> generateRecords(Collection data) { + Collection> records = new ArrayList<>(); + for (T recordData : data) { + Record record = new Record<>(recordData); + records.add(record); + } + return records; + } + + private ZeroBuffer> createObjectUnderTest() { + ZeroBuffer> zeroBuffer = createObjectUnderTestWithPipelineName(); + zeroBuffer.setPipelineRunner(pipelineRunner); + return zeroBuffer; + } + + private ZeroBuffer> createObjectUnderTestWithPipelineName() { + when(pipelineDescription.getPipelineName()).thenReturn(MOCK_PIPELINE_NAME); + return new ZeroBuffer<>(pipelineDescription); + } +}