From c2dcf76c74aff8f22462d6ad82a85bea2ba60c0b Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Wed, 14 Aug 2024 11:07:18 -0700 Subject: [PATCH 1/8] Kinesis data streams ingestion functionality checkin Signed-off-by: Souvik Bose --- .../kinesis-source/build.gradle | 43 +++ .../plugins/source/kinesis/ClientFactory.java | 48 +++ .../kinesis/KinesisMultiStreamTracker.java | 82 +++++ .../source/kinesis/KinesisService.java | 140 ++++++++ .../plugins/source/kinesis/KinesisSource.java | 56 ++++ .../kinesis/WorkerIdentifierGenerator.java | 28 ++ .../AwsAuthenticationConfig.java | 87 +++++ .../configuration/ConsumerStrategy.java | 23 ++ .../configuration/KinesisSourceConfig.java | 68 ++++ .../configuration/KinesisStreamConfig.java | 32 ++ .../KinesisStreamPollingConfig.java | 17 + .../processor/KinesisRecordProcessor.java | 180 +++++++++++ .../KinesisShardRecordProcessorFactory.java | 43 +++ .../source/kinesis/ClientFactoryTest.java | 47 +++ .../KinesisMultiStreamTrackerTest.java | 143 +++++++++ .../source/kinesis/KinesisServiceTest.java | 287 +++++++++++++++++ .../source/kinesis/KinesisSourceTest.java | 141 ++++++++ .../AwsAuthenticationConfigTest.java | 263 +++++++++++++++ .../KinesisSourceConfigTest.java | 114 +++++++ .../KinesisStreamPollingConfigTest.java | 18 ++ .../processor/KinesisRecordProcessorTest.java | 301 ++++++++++++++++++ ...inesisShardRecordProcessorFactoryTest.java | 84 +++++ .../src/test/resources/simple-pipeline-2.yaml | 21 ++ .../src/test/resources/simple-pipeline.yaml | 16 + settings.gradle | 2 +- 25 files changed, 2283 insertions(+), 1 deletion(-) create mode 100644 data-prepper-plugins/kinesis-source/build.gradle create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactory.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTracker.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisService.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSource.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/WorkerIdentifierGenerator.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfig.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/ConsumerStrategy.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfig.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamConfig.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfig.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessor.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactory.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactoryTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTrackerTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisServiceTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSourceTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfigTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfigTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfigTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessorTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactoryTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline-2.yaml create mode 100644 data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline.yaml diff --git a/data-prepper-plugins/kinesis-source/build.gradle b/data-prepper-plugins/kinesis-source/build.gradle new file mode 100644 index 0000000000..400fe22c00 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/build.gradle @@ -0,0 +1,43 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +plugins { + id 'java' +} + +dependencies { + implementation project(':data-prepper-api') + implementation project(':data-prepper-plugins:common') + implementation project(':data-prepper-plugins:buffer-common') + implementation libs.armeria.core + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation project(':data-prepper-plugins:blocking-buffer') + implementation 'software.amazon.awssdk:kinesis' + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-ion' + implementation 'software.amazon.kinesis:amazon-kinesis-client:2.6.0' + compileOnly 'org.projectlombok:lombok:1.18.20' + annotationProcessor 'org.projectlombok:lombok:1.18.20' + implementation("software.amazon.awssdk:dynamodb") + implementation("com.amazonaws:aws-java-sdk:1.12.394") + implementation project(path: ':data-prepper-plugins:aws-plugin-api') + + testImplementation 'org.yaml:snakeyaml:2.2' + testImplementation project(':data-prepper-test-common') + testImplementation platform('org.junit:junit-bom:5.9.1') + testImplementation 'org.junit.jupiter:junit-jupiter' + testImplementation project(':data-prepper-test-event') +} + +jacocoTestCoverageVerification { + dependsOn jacocoTestReport + violationRules { + rule { //in addition to core projects rule + limit { + minimum = 1.0 + } + } + } +} \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactory.java new file mode 100644 index 0000000000..5abf9185d0 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactory.java @@ -0,0 +1,48 @@ +package org.opensearch.dataprepper.plugins.source.kinesis; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.AwsAuthenticationConfig; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.kinesis.common.KinesisClientUtil; + +public class ClientFactory { + private final AwsCredentialsProvider awsCredentialsProvider; + private final AwsAuthenticationConfig awsAuthenticationConfig; + + public ClientFactory(final AwsCredentialsSupplier awsCredentialsSupplier, + final AwsAuthenticationConfig awsAuthenticationConfig) { + awsCredentialsProvider = awsCredentialsSupplier.getProvider(AwsCredentialsOptions.builder() + .withRegion(awsAuthenticationConfig.getAwsRegion()) + .withStsRoleArn(awsAuthenticationConfig.getAwsStsRoleArn()) + .withStsExternalId(awsAuthenticationConfig.getAwsStsExternalId()) + .withStsHeaderOverrides(awsAuthenticationConfig.getAwsStsHeaderOverrides()) + .build()); + this.awsAuthenticationConfig = awsAuthenticationConfig; + } + + public DynamoDbAsyncClient buildDynamoDBClient() { + return DynamoDbAsyncClient.builder() + .credentialsProvider(awsAuthenticationConfig.authenticateAwsConfiguration()) + .region(awsAuthenticationConfig.getAwsRegion()) + .build(); + } + + public KinesisAsyncClient buildKinesisAsyncClient() { + return KinesisClientUtil.createKinesisAsyncClient( + KinesisAsyncClient.builder() + .credentialsProvider(awsAuthenticationConfig.authenticateAwsConfiguration()) + .region(awsAuthenticationConfig.getAwsRegion()) + ); + } + + public CloudWatchAsyncClient buildCloudWatchAsyncClient() { + return CloudWatchAsyncClient.builder() + .credentialsProvider(awsAuthenticationConfig.authenticateAwsConfiguration()) + .region(awsAuthenticationConfig.getAwsRegion()) + .build(); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTracker.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTracker.java new file mode 100644 index 0000000000..5da7a3d160 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTracker.java @@ -0,0 +1,82 @@ +package org.opensearch.dataprepper.plugins.source.kinesis; + +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse; +import software.amazon.awssdk.services.kinesis.model.StreamDescription; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy; +import software.amazon.kinesis.processor.MultiStreamTracker; + +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; + + +public class KinesisMultiStreamTracker implements MultiStreamTracker { + private static final String COLON = ":"; + + private final KinesisAsyncClient kinesisClient; + private final KinesisSourceConfig sourceConfig; + private final String applicationName; + + public KinesisMultiStreamTracker(KinesisAsyncClient kinesisClient, final KinesisSourceConfig sourceConfig, final String applicationName) { + this.kinesisClient = kinesisClient; + this.sourceConfig = sourceConfig; + this.applicationName = applicationName; + } + + @Override + public List streamConfigList() { + List streamConfigList = new ArrayList<>(); + for (KinesisStreamConfig kinesisStreamConfig : sourceConfig.getStreams()) { + StreamConfig streamConfig; + try { + streamConfig = getStreamConfig(kinesisStreamConfig); + } catch (Exception e) { + throw new RuntimeException(e); + } + streamConfigList.add(streamConfig); + } + return streamConfigList; + } + + private StreamConfig getStreamConfig(KinesisStreamConfig kinesisStreamConfig) throws Exception { + StreamIdentifier sourceStreamIdentifier = getStreamIdentifier(kinesisStreamConfig); + return new StreamConfig(sourceStreamIdentifier, + InitialPositionInStreamExtended.newInitialPosition(kinesisStreamConfig.getInitialPosition())); + } + + private StreamIdentifier getStreamIdentifier(KinesisStreamConfig kinesisStreamConfig) throws Exception { + DescribeStreamRequest describeStreamRequest = DescribeStreamRequest.builder() + .streamName(kinesisStreamConfig.getName()) + .build(); + DescribeStreamResponse describeStreamResponse = kinesisClient.describeStream(describeStreamRequest).get(); + String streamIdentifierString = getStreamIdentifierString(describeStreamResponse.streamDescription()); + return StreamIdentifier.multiStreamInstance(streamIdentifierString); + } + + private String getStreamIdentifierString(StreamDescription streamDescription) { + String accountId = streamDescription.streamARN().split(COLON)[4]; + long creationEpochSecond = streamDescription.streamCreationTimestamp().getEpochSecond(); + return String.join(COLON, accountId, streamDescription.streamName(), String.valueOf(creationEpochSecond)); + } + + /** + * Setting the deletion policy as autodetect and release shard lease with a wait time of 10 sec + */ + @Override + public FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy() { + return new FormerStreamsLeasesDeletionStrategy.AutoDetectionAndDeferredDeletionStrategy() { + @Override + public Duration waitPeriodToDeleteFormerStreams() { + return Duration.ofSeconds(10); + } + }; + + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisService.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisService.java new file mode 100644 index 0000000000..0d48aaf161 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisService.java @@ -0,0 +1,140 @@ +package org.opensearch.dataprepper.plugins.source.kinesis; + +import io.micrometer.core.instrument.util.StringUtils; +import lombok.Setter; +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.configuration.PipelineDescription; +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.kinesis.configuration.ConsumerStrategy; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.processor.KinesisShardRecordProcessorFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.dynamodb.model.BillingMode; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.kinesis.common.ConfigsBuilder; +import software.amazon.kinesis.coordinator.Scheduler; +import software.amazon.kinesis.processor.ShardRecordProcessorFactory; +import software.amazon.kinesis.retrieval.polling.PollingConfig; + +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; +import java.util.concurrent.TimeoutException; + +public class KinesisService { + private static final Logger LOG = LoggerFactory.getLogger(KinesisService.class); + private static final int GRACEFUL_SHUTDOWN_WAIT_INTERVAL_SECONDS = 20; + + private final PluginMetrics pluginMetrics; + private final PluginFactory pluginFactory; + + private final String applicationName; + private final String tableName; + private final String pipelineName; + private final AcknowledgementSetManager acknowledgementSetManager; + private final KinesisSourceConfig sourceConfig; + private final KinesisAsyncClient kinesisClient; + private final DynamoDbAsyncClient dynamoDbClient; + private final CloudWatchAsyncClient cloudWatchClient; + + @Setter + private Scheduler scheduler; + + private final ExecutorService executorService; + + public KinesisService(final KinesisSourceConfig sourceConfig, + final ClientFactory clientFactory, + final PluginMetrics pluginMetrics, + final PluginFactory pluginFactory, + final PipelineDescription pipelineDescription, + final AcknowledgementSetManager acknowledgementSetManager + ){ + this.sourceConfig = sourceConfig; + this.pluginMetrics = pluginMetrics; + this.pluginFactory = pluginFactory; + this.acknowledgementSetManager = acknowledgementSetManager; + this.dynamoDbClient = clientFactory.buildDynamoDBClient(); + this.kinesisClient = clientFactory.buildKinesisAsyncClient(); + this.cloudWatchClient = clientFactory.buildCloudWatchAsyncClient(); + this.pipelineName = pipelineDescription.getPipelineName(); + this.applicationName = pipelineName; + this.tableName = StringUtils.isNotEmpty(sourceConfig.getLeaseCoordinationTable()) ? + sourceConfig.getLeaseCoordinationTable() : applicationName; + this.executorService = Executors.newFixedThreadPool(1); + } + + public void start(final Buffer> buffer) { + if (buffer == null) { + throw new IllegalStateException("Buffer provided is null"); + } + + if (sourceConfig.getStreams() == null || sourceConfig.getStreams().isEmpty()) { + throw new IllegalStateException("Streams are empty!"); + } + + scheduler = getScheduler(buffer); + executorService.execute(scheduler); + } + + public void shutDown() { + LOG.info("Stop request received for Kinesis Source"); + + Future gracefulShutdownFuture = scheduler.startGracefulShutdown(); + LOG.info("Waiting up to {} seconds for shutdown to complete.", GRACEFUL_SHUTDOWN_WAIT_INTERVAL_SECONDS); + try { + gracefulShutdownFuture.get(GRACEFUL_SHUTDOWN_WAIT_INTERVAL_SECONDS, TimeUnit.SECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException ex) { + LOG.error("Exception while executing kinesis consumer graceful shutdown, doing force shutdown", ex); + scheduler.shutdown(); + } + LOG.info("Completed, shutting down now."); + } + + public Scheduler getScheduler(final Buffer> buffer) { + if (scheduler == null) { + return createScheduler(buffer); + } + return scheduler; + } + + public Scheduler createScheduler(final Buffer> buffer) { + final ShardRecordProcessorFactory processorFactory = new KinesisShardRecordProcessorFactory( + buffer, sourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory); + + ConfigsBuilder configsBuilder = + new ConfigsBuilder( + new KinesisMultiStreamTracker(kinesisClient, sourceConfig, applicationName), + applicationName, kinesisClient, dynamoDbClient, cloudWatchClient, + new WorkerIdentifierGenerator().generate(), processorFactory + ).tableName(applicationName); + + ConsumerStrategy consumerStrategy = sourceConfig.getConsumerStrategy(); + if (consumerStrategy == ConsumerStrategy.POLLING) { + configsBuilder.retrievalConfig().retrievalSpecificConfig( + new PollingConfig(kinesisClient) + .maxRecords(sourceConfig.getPollingConfig().getMaxPollingRecords()) + .idleTimeBetweenReadsInMillis( + sourceConfig.getPollingConfig().getIdleTimeBetweenReadsInMillis())); + } + + return new Scheduler( + configsBuilder.checkpointConfig(), + configsBuilder.coordinatorConfig(), + configsBuilder.leaseManagementConfig() + .billingMode(BillingMode.PAY_PER_REQUEST), + configsBuilder.lifecycleConfig(), + configsBuilder.metricsConfig(), + configsBuilder.processorConfig(), + configsBuilder.retrievalConfig() + ); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSource.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSource.java new file mode 100644 index 0000000000..98069b5bb8 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSource.java @@ -0,0 +1,56 @@ +package org.opensearch.dataprepper.plugins.source.kinesis; + +import lombok.Setter; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@DataPrepperPlugin(name = "kinesis", pluginType = Source.class, pluginConfigurationType = KinesisSourceConfig.class) +public class KinesisSource implements Source> { + private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class); + private final KinesisSourceConfig kinesisSourceConfig; + + @Setter + private KinesisService kinesisService; + + @DataPrepperPluginConstructor + public KinesisSource(final KinesisSourceConfig kinesisSourceConfig, + final PluginMetrics pluginMetrics, + final PluginFactory pluginFactory, + final PipelineDescription pipelineDescription, + final AwsCredentialsSupplier awsCredentialsSupplier, + final AcknowledgementSetManager acknowledgementSetManager) { + this.kinesisSourceConfig = kinesisSourceConfig; + ClientFactory clientFactory = new ClientFactory(awsCredentialsSupplier, kinesisSourceConfig.getAwsAuthenticationConfig()); + this.kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager); + } + @Override + public void start(final Buffer> buffer) { + if (buffer == null) { + throw new IllegalStateException("Buffer provided is null"); + } + + kinesisService.start(buffer); + } + + @Override + public void stop() { + kinesisService.shutDown(); + } + + @Override + public boolean areAcknowledgementsEnabled() { + return kinesisSourceConfig.isAcknowledgments(); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/WorkerIdentifierGenerator.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/WorkerIdentifierGenerator.java new file mode 100644 index 0000000000..9390e25920 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/WorkerIdentifierGenerator.java @@ -0,0 +1,28 @@ +package org.opensearch.dataprepper.plugins.source.kinesis; + +import java.net.InetAddress; +import java.net.UnknownHostException; + +/** + * Generate a unique ID to represent a consumer application instance. + */ +public class WorkerIdentifierGenerator { + + private static final String hostName; + + static { + try { + hostName = InetAddress.getLocalHost().getHostName(); + } catch (final UnknownHostException e) { + throw new RuntimeException(e); + } + } + + + /** + * @return Default to use host name. + */ + public String generate() { + return hostName; + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfig.java new file mode 100644 index 0000000000..d51a7ee6a4 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfig.java @@ -0,0 +1,87 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.configuration; + +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; +import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; + +import java.util.Map; +import java.util.UUID; + +public class AwsAuthenticationConfig { + private static final String AWS_IAM_ROLE = "role"; + private static final String AWS_IAM = "iam"; + + @JsonProperty("region") + @Size(min = 1, message = "Region cannot be empty string") + private String awsRegion; + + @JsonProperty("sts_role_arn") + @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") + private String awsStsRoleArn; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } + + public AwsCredentialsProvider authenticateAwsConfiguration() { + + final AwsCredentialsProvider awsCredentialsProvider; + if (awsStsRoleArn != null && !awsStsRoleArn.isEmpty()) { + try { + Arn.fromString(awsStsRoleArn); + } catch (final Exception e) { + throw new IllegalArgumentException("Invalid ARN format for awsStsRoleArn"); + } + + final StsClient stsClient = StsClient.builder().region(getAwsRegion()).build(); + + AssumeRoleRequest.Builder assumeRoleRequestBuilder = AssumeRoleRequest.builder() + .roleSessionName("GeoIP-Processor-" + UUID.randomUUID()).roleArn(awsStsRoleArn); + + if (awsStsHeaderOverrides != null && !awsStsHeaderOverrides.isEmpty()) { + assumeRoleRequestBuilder = assumeRoleRequestBuilder.overrideConfiguration( + configuration -> awsStsHeaderOverrides.forEach(configuration::putHeader)); + } + + awsCredentialsProvider = StsAssumeRoleCredentialsProvider.builder() + .stsClient(stsClient) + .refreshRequest(assumeRoleRequestBuilder.build()) + .build(); + + } else { + awsCredentialsProvider = DefaultCredentialsProvider.create(); + } + return awsCredentialsProvider; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/ConsumerStrategy.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/ConsumerStrategy.java new file mode 100644 index 0000000000..7b7a7ddf9d --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/ConsumerStrategy.java @@ -0,0 +1,23 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.configuration; + +import com.fasterxml.jackson.annotation.JsonValue; + +// Reference: https://docs.aws.amazon.com/streams/latest/dev/enhanced-consumers.html + +public enum ConsumerStrategy { + + POLLING("Polling"), + + ENHANCED_FAN_OUT("Fan-Out"); + + private final String value; + + ConsumerStrategy(String value) { + this.value = value; + } + + @JsonValue + public String getValue() { + return value; + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfig.java new file mode 100644 index 0000000000..3aefe7bece --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfig.java @@ -0,0 +1,68 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import jakarta.validation.constraints.Size; +import lombok.Getter; +import org.opensearch.dataprepper.model.configuration.PluginModel; + +import java.time.Duration; +import java.util.List; + +public class KinesisSourceConfig { + static final Duration DEFAULT_TIME_OUT_IN_MILLIS = Duration.ofMillis(1000); + static final int DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE = 100; + static final Duration DEFAULT_SHARD_ACKNOWLEDGEMENT_TIMEOUT = Duration.ofMinutes(10); + + @Getter + @JsonProperty("streams") + @NotNull + @Valid + @Size(min = 1, max = 4, message = "Only support a maximum of 4 streams") + private List streams; + + @Getter + @JsonProperty("aws") + @NotNull + @Valid + private AwsAuthenticationConfig awsAuthenticationConfig; + + @Getter + @JsonProperty("buffer_timeout") + private Duration bufferTimeout = DEFAULT_TIME_OUT_IN_MILLIS; + + @Getter + @JsonProperty("records_to_accumulate") + private int numberOfRecordsToAccumulate = DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE; + + @JsonProperty("acknowledgments") + @Getter + private boolean acknowledgments = false; + + @Getter + @JsonProperty("consumer_strategy") + private ConsumerStrategy consumerStrategy = ConsumerStrategy.ENHANCED_FAN_OUT; + + @Getter + @JsonProperty("polling") + private KinesisStreamPollingConfig pollingConfig; + + @JsonProperty("lease_coordination_table") + @Getter + private String leaseCoordinationTable; + + @Getter + @JsonProperty("codec") + private PluginModel codec; + + @JsonProperty("shard_acknowledgment_timeout") + private Duration shardAcknowledgmentTimeout = DEFAULT_SHARD_ACKNOWLEDGEMENT_TIMEOUT; + + public Duration getShardAcknowledgmentTimeout() { + return shardAcknowledgmentTimeout; + } +} + + + diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamConfig.java new file mode 100644 index 0000000000..49e3a5743f --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamConfig.java @@ -0,0 +1,32 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import lombok.Getter; +import software.amazon.kinesis.common.InitialPositionInStream; + +@Getter +public class KinesisStreamConfig { + // Checkpointing interval + private static final int MINIMAL_CHECKPOINT_INTERVAL_MILLIS = 2 * 60 * 1000; // 2 minute + private static final boolean DEFAULT_ENABLE_CHECKPOINT = false; + + @JsonProperty("stream_name") + @NotNull + @Valid + private String name; + + @JsonProperty("stream_arn") + private String arn; + + @JsonProperty("initial_position") + private InitialPositionInStream initialPosition = InitialPositionInStream.LATEST; + + @JsonProperty("checkpoint_interval") + private int checkPointIntervalInMilliseconds = MINIMAL_CHECKPOINT_INTERVAL_MILLIS; + + @Getter + @JsonProperty("enableCheckpoint") + private boolean enableCheckPoint = DEFAULT_ENABLE_CHECKPOINT; +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfig.java new file mode 100644 index 0000000000..e48908d7f8 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfig.java @@ -0,0 +1,17 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.Getter; + +public class KinesisStreamPollingConfig { + private static final int DEFAULT_MAX_RECORDS = 10000; + private static final int IDLE_TIME_BETWEEN_READS_IN_MILLIS = 250; + @Getter + @JsonProperty("maxPollingRecords") + private int maxPollingRecords = DEFAULT_MAX_RECORDS; + + @Getter + @JsonProperty("idleTimeBetweenReadsInMillis") + private int idleTimeBetweenReadsInMillis = IDLE_TIME_BETWEEN_READS_IN_MILLIS; + +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessor.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessor.java new file mode 100644 index 0000000000..0a7c7c0ebe --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessor.java @@ -0,0 +1,180 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.processor; + +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.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.kinesis.KinesisSource; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.exceptions.InvalidStateException; +import software.amazon.kinesis.exceptions.ShutdownException; +import software.amazon.kinesis.exceptions.ThrottlingException; +import software.amazon.kinesis.lifecycle.events.InitializationInput; +import software.amazon.kinesis.lifecycle.events.LeaseLostInput; +import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; +import software.amazon.kinesis.lifecycle.events.ShardEndedInput; +import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; +import software.amazon.kinesis.processor.RecordProcessorCheckpointer; +import software.amazon.kinesis.processor.ShardRecordProcessor; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +public class KinesisRecordProcessor implements ShardRecordProcessor { + private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class); + private final StreamIdentifier streamIdentifier; + private final KinesisStreamConfig kinesisStreamConfig; + private final int checkpointIntervalMilliSeconds; + private final KinesisSourceConfig kinesisSourceConfig; + private final Buffer> buffer; + private String kinesisShardId; + private final InputCodec codec; + private long lastCheckpointTimeInMillis; + private final int bufferTimeoutMillis; + private final AcknowledgementSetManager acknowledgementSetManager; + private final Counter acknowledgementSetCallbackCounter; + private final Counter recordProcessingErrors; + private final Counter checkpointFailures; + private static final Duration ACKNOWLEDGEMENT_SET_TIMEOUT = Duration.ofSeconds(20); + private static final String ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME = "acknowledgementSetCallbackCounter"; + public static final String KINESIS_RECORD_PROCESSING_ERRORS = "recordProcessingErrors"; + public static final String KINESIS_CHECKPOINT_FAILURES = "checkpointFailures"; + public static final String KINESIS_STREAM_TAG_KEY = "stream"; + + public KinesisRecordProcessor(Buffer> buffer, + final KinesisSourceConfig kinesisSourceConfig, + final AcknowledgementSetManager acknowledgementSetManager, + final PluginMetrics pluginMetrics, + final PluginFactory pluginFactory, + final StreamIdentifier streamIdentifier) { + this.bufferTimeoutMillis = (int) kinesisSourceConfig.getBufferTimeout().toMillis(); + this.streamIdentifier = streamIdentifier; + this.kinesisSourceConfig = kinesisSourceConfig; + this.kinesisStreamConfig = getStreamConfig(kinesisSourceConfig); + final PluginModel codecConfiguration = kinesisSourceConfig.getCodec(); + final PluginSetting codecPluginSettings = new PluginSetting(codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); + this.codec = pluginFactory.loadPlugin(InputCodec.class, codecPluginSettings); + this.acknowledgementSetManager = acknowledgementSetManager; + this.acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME); + this.recordProcessingErrors = pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); + this.checkpointFailures = pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); + this.checkpointIntervalMilliSeconds = kinesisStreamConfig.getCheckPointIntervalInMilliseconds(); + this.buffer = buffer; + } + + private KinesisStreamConfig getStreamConfig(final KinesisSourceConfig kinesisSourceConfig) { + return kinesisSourceConfig.getStreams().stream().filter(streamConfig -> streamConfig.getName().equals(streamIdentifier.streamName())).findAny().get(); + } + + @Override + public void initialize(InitializationInput initializationInput) { + // Called once when the processor is initialized. + kinesisShardId = initializationInput.shardId(); + LOG.info("Initialize Processor for shard: " + kinesisShardId); + lastCheckpointTimeInMillis = System.currentTimeMillis(); + } + + private AcknowledgementSet createAcknowledgmentSet(final ProcessRecordsInput processRecordsInput) { + return acknowledgementSetManager.create((result) -> { + acknowledgementSetCallbackCounter.increment(); + if (result) { + LOG.info("acknowledgements received"); + checkpoint(processRecordsInput.checkpointer()); + } else { + LOG.info("acknowledgements received with false"); + } + + }, ACKNOWLEDGEMENT_SET_TIMEOUT); + } + + @Override + public void processRecords(ProcessRecordsInput processRecordsInput) { + List> records = new ArrayList<>(); + + try { + AcknowledgementSet acknowledgementSet; + boolean acknowledgementsEnabled = kinesisSourceConfig.isAcknowledgments(); + if (acknowledgementsEnabled) { + acknowledgementSet = createAcknowledgmentSet(processRecordsInput); + } else { + acknowledgementSet = null; + } + + for (KinesisClientRecord record : processRecordsInput.records()) { + processRecord(record, records::add); + } + + if (acknowledgementSet != null) { + records.forEach(record -> { + acknowledgementSet.add(record.getData()); + }); + } + + buffer.writeAll(records, bufferTimeoutMillis); + + if (acknowledgementSet != null) { + acknowledgementSet.complete(); + } + + // Checkpoint for shard + if (kinesisStreamConfig.isEnableCheckPoint() && System.currentTimeMillis() - lastCheckpointTimeInMillis > checkpointIntervalMilliSeconds) { + LOG.info("Regular checkpointing for shard " + kinesisShardId); + checkpoint(processRecordsInput.checkpointer()); + lastCheckpointTimeInMillis = System.currentTimeMillis(); + } + } catch (Exception ex) { + recordProcessingErrors.increment(); + LOG.error("Failed writing shard data to buffer: ", ex); + } + } + + private void processRecord(KinesisClientRecord record, Consumer> eventConsumer) throws IOException { + // Read bytebuffer + byte[] arr = new byte[record.data().remaining()]; + record.data().get(arr); + ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(arr); + codec.parse(byteArrayInputStream, eventConsumer); + } + + @Override + public void leaseLost(LeaseLostInput leaseLostInput) { + LOG.debug("Lease Lost"); + } + + @Override + public void shardEnded(ShardEndedInput shardEndedInput) { + LOG.info("Reached shard end, checkpointing shard: {}", kinesisShardId); + checkpoint(shardEndedInput.checkpointer()); + } + + @Override + public void shutdownRequested(ShutdownRequestedInput shutdownRequestedInput) { + LOG.info("Scheduler is shutting down, checkpointing shard: {}", kinesisShardId); + checkpoint(shutdownRequestedInput.checkpointer()); + } + + private void checkpoint(RecordProcessorCheckpointer checkpointer) { + try { + checkpointer.checkpoint(); + } catch (ShutdownException | ThrottlingException | InvalidStateException ex) { + LOG.info("Caught exception at checkpoint, skipping checkpoint.", ex); + checkpointFailures.increment(); + } + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactory.java new file mode 100644 index 0000000000..91294857b4 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactory.java @@ -0,0 +1,43 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.processor; + +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.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.processor.ShardRecordProcessor; +import software.amazon.kinesis.processor.ShardRecordProcessorFactory; + +public class KinesisShardRecordProcessorFactory implements ShardRecordProcessorFactory { + + private final Buffer> buffer; + private final KinesisSourceConfig kinesisSourceConfig; + private final AcknowledgementSetManager acknowledgementSetManager; + private final PluginMetrics pluginMetrics; + private final PluginFactory pluginFactory; + + public KinesisShardRecordProcessorFactory(Buffer> buffer, + KinesisSourceConfig kinesisSourceConfig, + final AcknowledgementSetManager acknowledgementSetManager, + final PluginMetrics pluginMetrics, + final PluginFactory pluginFactory) { + this.kinesisSourceConfig = kinesisSourceConfig; + this.buffer = buffer; + this.acknowledgementSetManager = acknowledgementSetManager; + this.pluginMetrics = pluginMetrics; + this.pluginFactory = pluginFactory; + } + + @Override + public ShardRecordProcessor shardRecordProcessor() { + throw new UnsupportedOperationException("Use the method with stream details!"); + } + + @Override + public ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) { + return new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactoryTest.java new file mode 100644 index 0000000000..3365a1e690 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactoryTest.java @@ -0,0 +1,47 @@ +package org.opensearch.dataprepper.plugins.source.kinesis; + +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; + +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.mock; + +public class ClientFactoryTest { + private Region region = Region.US_EAST_1; + private String roleArn; + private Map stsHeader; + private AwsCredentialsSupplier awsCredentialsSupplier; + + @Test + void testCreateClient() throws NoSuchFieldException, IllegalAccessException { + roleArn = "arn:aws:iam::278936200144:role/test-role"; + stsHeader= new HashMap<>(); + stsHeader.put(UUID.randomUUID().toString(),UUID.randomUUID().toString()); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + + AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", roleArn); + + ClientFactory clientFactory = new ClientFactory(awsCredentialsSupplier, awsAuthenticationOptionsConfig); + + final DynamoDbAsyncClient dynamoDbAsyncClient = clientFactory.buildDynamoDBClient(); + assertNotNull(dynamoDbAsyncClient); + + final KinesisAsyncClient kinesisAsyncClient = clientFactory.buildKinesisAsyncClient(); + assertNotNull(kinesisAsyncClient); + + final CloudWatchAsyncClient cloudWatchAsyncClient = clientFactory.buildCloudWatchAsyncClient(); + assertNotNull(cloudWatchAsyncClient); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTrackerTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTrackerTest.java new file mode 100644 index 0000000000..637cb16375 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTrackerTest.java @@ -0,0 +1,143 @@ +package org.opensearch.dataprepper.plugins.source.kinesis; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse; +import software.amazon.awssdk.services.kinesis.model.StreamDescription; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.common.InitialPositionInStreamExtended; +import software.amazon.kinesis.common.StreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.processor.FormerStreamsLeasesDeletionStrategy; + +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class KinesisMultiStreamTrackerTest { + private static final String APPLICATION_NAME = "multi-stream-application"; + private static final String awsAccountId = "1234"; + private static final String streamArnFormat = "arn:aws:kinesis:us-east-1:%s:stream/%s"; + private static final Instant streamCreationTime = Instant.now(); + private static final List STREAMS_LIST = ImmutableList.of("stream-1", "stream-2", "stream-3"); + + private KinesisMultiStreamTracker kinesisMultiStreamTracker; + @Mock + private KinesisAsyncClient kinesisClient; + private List streamConfigList; + + private Map streamConfigMap; + + @Mock + KinesisSourceConfig kinesisSourceConfig; + + @BeforeEach + public void setUp() { + MockitoAnnotations.openMocks(this); + List kinesisStreamConfigs = new ArrayList<>(); + streamConfigMap = new HashMap<>(); + STREAMS_LIST.forEach(stream -> { + KinesisStreamConfig kinesisStreamConfig = mock(KinesisStreamConfig.class); + when(kinesisStreamConfig.getName()).thenReturn(stream); + when(kinesisStreamConfig.getInitialPosition()).thenReturn(InitialPositionInStream.LATEST); + + StreamDescription streamDescription = StreamDescription.builder() + .streamARN(String.format(streamArnFormat, awsAccountId, stream)) + .streamCreationTimestamp(streamCreationTime) + .streamName(stream) + .build(); + + DescribeStreamRequest describeStreamRequest = DescribeStreamRequest.builder() + .streamName(stream) + .build(); + + DescribeStreamResponse describeStreamResponse = DescribeStreamResponse.builder() + .streamDescription(streamDescription) + .build(); + + when(kinesisClient.describeStream(describeStreamRequest)).thenReturn(CompletableFuture.completedFuture(describeStreamResponse)); + kinesisStreamConfigs.add(kinesisStreamConfig); + + streamConfigMap.put(stream, kinesisStreamConfig); + }); + + when(kinesisSourceConfig.getStreams()).thenReturn(kinesisStreamConfigs); + kinesisMultiStreamTracker = new KinesisMultiStreamTracker(kinesisClient, kinesisSourceConfig, APPLICATION_NAME); + } + + @Test + public void testStreamConfigList() { + streamConfigList = kinesisMultiStreamTracker.streamConfigList(); + assertEquals(kinesisSourceConfig.getStreams().size(), streamConfigList.size()); + + int totalStreams = streamConfigList.size(); + for (int i=0; i kinesisStreamConfigs = new ArrayList<>(); + streamConfigMap = new HashMap<>(); + STREAMS_LIST.forEach(stream -> { + KinesisStreamConfig kinesisStreamConfig = mock(KinesisStreamConfig.class); + when(kinesisStreamConfig.getName()).thenReturn(stream); + when(kinesisStreamConfig.getInitialPosition()).thenReturn(InitialPositionInStream.LATEST); + + DescribeStreamRequest describeStreamRequest = DescribeStreamRequest.builder() + .streamName(stream) + .build(); + + when(kinesisClient.describeStream(describeStreamRequest)).thenThrow(new RuntimeException()); + kinesisStreamConfigs.add(kinesisStreamConfig); + + streamConfigMap.put(stream, kinesisStreamConfig); + }); + + when(kinesisSourceConfig.getStreams()).thenReturn(kinesisStreamConfigs); + kinesisMultiStreamTracker = new KinesisMultiStreamTracker(kinesisClient, kinesisSourceConfig, APPLICATION_NAME); + + assertThrows(RuntimeException.class, () -> kinesisMultiStreamTracker.streamConfigList()); + } + + @Test + public void formerStreamsLeasesDeletionStrategy() { + + FormerStreamsLeasesDeletionStrategy formerStreamsLeasesDeletionStrategy = + kinesisMultiStreamTracker.formerStreamsLeasesDeletionStrategy(); + + Duration duration = formerStreamsLeasesDeletionStrategy.waitPeriodToDeleteFormerStreams(); + + Assertions.assertTrue(formerStreamsLeasesDeletionStrategy instanceof + FormerStreamsLeasesDeletionStrategy.AutoDetectionAndDeferredDeletionStrategy); + assertEquals(10, duration.getSeconds()); + } + + private StreamIdentifier getStreamIdentifier(final String streamName) { + return StreamIdentifier.multiStreamInstance(String.join(":", awsAccountId, streamName, String.valueOf(streamCreationTime.getEpochSecond()))); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisServiceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisServiceTest.java new file mode 100644 index 0000000000..3a631dae97 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisServiceTest.java @@ -0,0 +1,287 @@ +package org.opensearch.dataprepper.plugins.source.kinesis; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +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.configuration.PipelineDescription; +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.kinesis.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.ConsumerStrategy; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamPollingConfig; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; +import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; +import software.amazon.awssdk.services.kinesis.KinesisServiceClientConfiguration; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; +import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse; +import software.amazon.awssdk.services.kinesis.model.StreamDescription; +import software.amazon.kinesis.common.InitialPositionInStream; +import software.amazon.kinesis.coordinator.Scheduler; +import software.amazon.kinesis.metrics.MetricsLevel; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class KinesisServiceTest { + private final String PIPELINE_NAME = "kinesis-pipeline-test"; + private final String streamId = "stream-1"; + + private static final int CHECKPOINT_INTERVAL_MS = 0; + private static final int NUMBER_OF_RECORDS_TO_ACCUMULATE = 10; + private static final int DEFAULT_MAX_RECORDS = 10000; + private static final int IDLE_TIME_BETWEEN_READS_IN_MILLIS = 250; + private static final String awsAccountId = "123456789012"; + private static final String streamArnFormat = "arn:aws:kinesis:us-east-1:%s:stream/%s"; + private static final Instant streamCreationTime = Instant.now(); + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private PluginFactory pluginFactory; + + @Mock + private KinesisSourceConfig kinesisSourceConfig; + + @Mock + private KinesisStreamConfig kinesisStreamConfig; + + @Mock + private KinesisStreamPollingConfig kinesisStreamPollingConfig; + + @Mock + private AwsAuthenticationConfig awsAuthenticationConfig; + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + @Mock + private PipelineDescription pipelineDescription; + + @Mock + private ClientFactory clientFactory; + + @Mock + private KinesisAsyncClient kinesisClient; + + @Mock + private DynamoDbAsyncClient dynamoDbClient; + + @Mock + private CloudWatchAsyncClient cloudWatchClient; + + @Mock + Buffer> buffer; + + @Mock + private Scheduler scheduler; + + @BeforeEach + void setup() { + awsAuthenticationConfig = mock(AwsAuthenticationConfig.class); + kinesisSourceConfig = mock(KinesisSourceConfig.class); + kinesisStreamConfig = mock(KinesisStreamConfig.class); + kinesisStreamPollingConfig = mock(KinesisStreamPollingConfig.class); + kinesisClient = mock(KinesisAsyncClient.class); + dynamoDbClient = mock(DynamoDbAsyncClient.class); + cloudWatchClient = mock(CloudWatchAsyncClient.class); + clientFactory = mock(ClientFactory.class); + scheduler = mock(Scheduler.class); + pipelineDescription = mock(PipelineDescription.class); + buffer = mock(Buffer.class); + + when(awsAuthenticationConfig.getAwsRegion()).thenReturn(Region.of("us-west-2")); + when(awsAuthenticationConfig.getAwsStsRoleArn()).thenReturn(UUID.randomUUID().toString()); + when(awsAuthenticationConfig.getAwsStsExternalId()).thenReturn(UUID.randomUUID().toString()); + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + when(awsAuthenticationConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); + StreamDescription streamDescription = StreamDescription.builder() + .streamARN(String.format(streamArnFormat, awsAccountId, streamId)) + .streamCreationTimestamp(streamCreationTime) + .streamName(streamId) + .build(); + + DescribeStreamRequest describeStreamRequest = DescribeStreamRequest.builder() + .streamName(streamId) + .build(); + + DescribeStreamResponse describeStreamResponse = DescribeStreamResponse.builder() + .streamDescription(streamDescription) + .build(); + + when(kinesisClient.describeStream(describeStreamRequest)).thenReturn(CompletableFuture.completedFuture(describeStreamResponse)); + + when(kinesisSourceConfig.getAwsAuthenticationConfig()).thenReturn(awsAuthenticationConfig); + when(kinesisStreamConfig.getName()).thenReturn(streamId); + when(kinesisStreamConfig.getCheckPointIntervalInMilliseconds()).thenReturn(CHECKPOINT_INTERVAL_MS); + when(kinesisStreamConfig.getInitialPosition()).thenReturn(InitialPositionInStream.LATEST); + when(kinesisSourceConfig.getConsumerStrategy()).thenReturn(ConsumerStrategy.POLLING); + when(kinesisSourceConfig.getPollingConfig()).thenReturn(kinesisStreamPollingConfig); + when(kinesisStreamPollingConfig.getMaxPollingRecords()).thenReturn(DEFAULT_MAX_RECORDS); + when(kinesisStreamPollingConfig.getIdleTimeBetweenReadsInMillis()).thenReturn(IDLE_TIME_BETWEEN_READS_IN_MILLIS); + + List streamConfigs = new ArrayList<>(); + streamConfigs.add(kinesisStreamConfig); + when(kinesisSourceConfig.getStreams()).thenReturn(streamConfigs); + when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); + + when(clientFactory.buildDynamoDBClient()).thenReturn(dynamoDbClient); + when(clientFactory.buildKinesisAsyncClient()).thenReturn(kinesisClient); + when(clientFactory.buildCloudWatchAsyncClient()).thenReturn(cloudWatchClient); + when(kinesisClient.serviceClientConfiguration()).thenReturn(KinesisServiceClientConfiguration.builder().region(Region.US_EAST_1).build()); + when(scheduler.startGracefulShutdown()).thenReturn(CompletableFuture.completedFuture(true)); + when(pipelineDescription.getPipelineName()).thenReturn(PIPELINE_NAME); + } + + public KinesisService createObjectUnderTest() { + return new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager); + } + + @Test + void testServiceStart() { + KinesisService kinesisService = createObjectUnderTest(); + kinesisService.start(buffer); + assertNotNull(kinesisService.getScheduler(buffer)); + } + + @Test + void testCreateScheduler() { + KinesisService kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager); + Scheduler schedulerObjectUnderTest = kinesisService.createScheduler(buffer); + + assertNotNull(schedulerObjectUnderTest); + assertNotNull(schedulerObjectUnderTest.checkpointConfig()); + assertNotNull(schedulerObjectUnderTest.leaseManagementConfig()); + assertSame(schedulerObjectUnderTest.leaseManagementConfig().initialPositionInStream().getInitialPositionInStream(), InitialPositionInStream.TRIM_HORIZON); + assertNotNull(schedulerObjectUnderTest.lifecycleConfig()); + assertNotNull(schedulerObjectUnderTest.metricsConfig()); + assertSame(schedulerObjectUnderTest.metricsConfig().metricsLevel(), MetricsLevel.DETAILED); + assertNotNull(schedulerObjectUnderTest.processorConfig()); + assertNotNull(schedulerObjectUnderTest.retrievalConfig()); + } + + @Test + void testCreateSchedulerWithPollingStrategy() { + when(kinesisSourceConfig.getConsumerStrategy()).thenReturn(ConsumerStrategy.POLLING); + KinesisService kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager); + Scheduler schedulerObjectUnderTest = kinesisService.createScheduler(buffer); + + assertNotNull(schedulerObjectUnderTest); + assertNotNull(schedulerObjectUnderTest.checkpointConfig()); + assertNotNull(schedulerObjectUnderTest.leaseManagementConfig()); + assertSame(schedulerObjectUnderTest.leaseManagementConfig().initialPositionInStream().getInitialPositionInStream(), InitialPositionInStream.TRIM_HORIZON); + assertNotNull(schedulerObjectUnderTest.lifecycleConfig()); + assertNotNull(schedulerObjectUnderTest.metricsConfig()); + assertSame(schedulerObjectUnderTest.metricsConfig().metricsLevel(), MetricsLevel.DETAILED); + assertNotNull(schedulerObjectUnderTest.processorConfig()); + assertNotNull(schedulerObjectUnderTest.retrievalConfig()); + } + + + @Test + void testServiceStartNullBufferThrows() { + KinesisService kinesisService = createObjectUnderTest(); + assertThrows(IllegalStateException.class, () -> kinesisService.start(null)); + + verify(scheduler, times(0)).run(); + } + + @Test + void testServiceStartNullStreams() { + when(kinesisSourceConfig.getStreams()).thenReturn(null); + + KinesisService kinesisService = createObjectUnderTest(); + assertThrows(IllegalStateException.class, () -> kinesisService.start(buffer)); + + verify(scheduler, times(0)).run(); + } + + @Test + void testServiceStartEmptyStreams() { + when(kinesisSourceConfig.getStreams()).thenReturn(new ArrayList<>()); + + KinesisService kinesisService = createObjectUnderTest(); + assertThrows(IllegalStateException.class, () -> kinesisService.start(buffer)); + + verify(scheduler, times(0)).run(); + } + + @Test + public void testShutdownGraceful() { + KinesisService kinesisService = createObjectUnderTest(); + kinesisService.setScheduler(scheduler); + kinesisService.shutDown(); + + verify(scheduler).startGracefulShutdown(); + verify(scheduler, times(0)).shutdown(); + } + + @Test + public void testShutdownGracefulThrowInterruptedException() { + KinesisService kinesisService = createObjectUnderTest(); + + when(scheduler.startGracefulShutdown()).thenReturn(CompletableFuture.failedFuture(new InterruptedException())); + kinesisService.setScheduler(scheduler); + assertDoesNotThrow(kinesisService::shutDown); + + verify(scheduler).startGracefulShutdown(); + verify(scheduler, times(1)).shutdown(); + } + + @Test + public void testShutdownGracefulThrowTimeoutException() { + KinesisService kinesisService = createObjectUnderTest(); + kinesisService.setScheduler(scheduler); + when(scheduler.startGracefulShutdown()).thenReturn(CompletableFuture.failedFuture(new TimeoutException())); + assertDoesNotThrow(kinesisService::shutDown); + + verify(scheduler).startGracefulShutdown(); + verify(scheduler, times(1)).shutdown(); + } + + @Test + public void testShutdownGracefulThrowExecutionException() { + KinesisService kinesisService = createObjectUnderTest(); + kinesisService.setScheduler(scheduler); + when(scheduler.startGracefulShutdown()).thenReturn(CompletableFuture.failedFuture(new ExecutionException(new Throwable()))); + assertDoesNotThrow(kinesisService::shutDown); + + verify(scheduler).startGracefulShutdown(); + verify(scheduler, times(1)).shutdown(); + } + + @Test + public void testShutdownExecutorServiceInterruptedException() { + when(scheduler.startGracefulShutdown()).thenReturn(CompletableFuture.failedFuture(new InterruptedException())); + + KinesisService kinesisService = createObjectUnderTest(); + kinesisService.setScheduler(scheduler); + kinesisService.shutDown(); + + verify(scheduler).startGracefulShutdown(); + verify(scheduler).shutdown(); + } + +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSourceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSourceTest.java new file mode 100644 index 0000000000..d367780571 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSourceTest.java @@ -0,0 +1,141 @@ +package org.opensearch.dataprepper.plugins.source.kinesis; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.configuration.PipelineDescription; +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.kinesis.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import software.amazon.awssdk.regions.Region; + +import java.util.List; +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class KinesisSourceTest { + private final String PIPELINE_NAME = "kinesis-pipeline-test"; + private final String streamId = "stream-1"; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private PluginFactory pluginFactory; + + @Mock + private KinesisSourceConfig kinesisSourceConfig; + + @Mock + private AwsAuthenticationConfig awsAuthenticationConfig; + + private KinesisSource source; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + @Mock + private AwsCredentialsSupplier awsCredentialsSupplier; + + @Mock + private PipelineDescription pipelineDescription; + + @Mock KinesisService kinesisService; + + @BeforeEach + void setup() { + pluginMetrics = mock(PluginMetrics.class); + pluginFactory = mock(PluginFactory.class); + kinesisSourceConfig = mock(KinesisSourceConfig.class); + this.pipelineDescription = mock(PipelineDescription.class); + awsCredentialsSupplier = mock(AwsCredentialsSupplier.class); + awsAuthenticationConfig = mock(AwsAuthenticationConfig.class); + acknowledgementSetManager = mock(AcknowledgementSetManager.class); + kinesisService = mock(KinesisService.class); + + when(awsAuthenticationConfig.getAwsRegion()).thenReturn(Region.of("us-west-2")); + when(awsAuthenticationConfig.getAwsStsRoleArn()).thenReturn(UUID.randomUUID().toString()); + when(awsAuthenticationConfig.getAwsStsExternalId()).thenReturn(UUID.randomUUID().toString()); + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + when(awsAuthenticationConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); + when(kinesisSourceConfig.getAwsAuthenticationConfig()).thenReturn(awsAuthenticationConfig); + when(pipelineDescription.getPipelineName()).thenReturn(PIPELINE_NAME); + } + + public KinesisSource createObjectUnderTest() { + return new KinesisSource(kinesisSourceConfig, pluginMetrics, pluginFactory, pipelineDescription, awsCredentialsSupplier, acknowledgementSetManager); + } + + @Test + public void testSourceWithoutAcknowledgements() { + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); + source = createObjectUnderTest(); + assertThat(source.areAcknowledgementsEnabled(), equalTo(false)); + } + + @Test + public void testSourceWithAcknowledgements() { + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(true); + source = createObjectUnderTest(); + assertThat(source.areAcknowledgementsEnabled(), equalTo(true)); + } + + @Test + public void testSourceStart() { + + source = createObjectUnderTest(); + + Buffer> buffer = mock(Buffer.class); + when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(100); + KinesisStreamConfig kinesisStreamConfig = mock(KinesisStreamConfig.class); + when(kinesisStreamConfig.getName()).thenReturn(streamId); + when(kinesisSourceConfig.getStreams()).thenReturn(List.of(kinesisStreamConfig)); + source.setKinesisService(kinesisService); + + source.start(buffer); + + verify(kinesisService, times(1)).start(any(Buffer.class)); + + } + + @Test + public void testSourceStartBufferNull() { + + source = createObjectUnderTest(); + + assertThrows(IllegalStateException.class, () -> source.start(null)); + + verify(kinesisService, times(0)).start(any(Buffer.class)); + + } + + @Test + public void testSourceStop() { + + source = createObjectUnderTest(); + + source.setKinesisService(kinesisService); + + source.stop(); + + verify(kinesisService, times(1)).shutDown(); + + } + +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfigTest.java new file mode 100644 index 0000000000..bfb6debc6f --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfigTest.java @@ -0,0 +1,263 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.hamcrest.CoreMatchers; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; +import org.mockito.MockedStatic; +import org.opensearch.dataprepper.test.helper.ReflectivelySetField; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.awscore.AwsRequestOverrideConfiguration; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.StsClientBuilder; +import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; + +import java.util.Collections; +import java.util.Map; +import java.util.UUID; +import java.util.function.Consumer; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +public class AwsAuthenticationConfigTest { + private ObjectMapper objectMapper = new ObjectMapper(); + private final String TEST_ROLE = "arn:aws:iam::123456789012:role/test-role"; + + @ParameterizedTest + @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) + void getAwsRegionReturnsRegion(final String regionString) { + final Region expectedRegionObject = Region.of(regionString); + final Map jsonMap = Map.of("region", regionString); + final AwsAuthenticationConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationConfig.class); + assertThat(objectUnderTest.getAwsRegion(), equalTo(expectedRegionObject)); + } + + @Test + void getAwsRegionReturnsNullWhenRegionIsNull() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationConfig.class); + assertThat(objectUnderTest.getAwsRegion(), nullValue()); + } + + @Test + void getAwsStsRoleArnReturnsValueFromDeserializedJSON() { + final String stsRoleArn = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_role_arn", stsRoleArn); + final AwsAuthenticationConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationConfig.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), equalTo(stsRoleArn)); + } + + @Test + void getAwsStsRoleArnReturnsNullIfNotInJSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationConfig.class); + assertThat(objectUnderTest.getAwsStsRoleArn(), nullValue()); + } + + @Test + void getAwsStsExternalIdReturnsValueFromDeserializedJSON() { + final String stsExternalId = UUID.randomUUID().toString(); + final Map jsonMap = Map.of("sts_external_id", stsExternalId); + final AwsAuthenticationConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationConfig.class); + assertThat(objectUnderTest.getAwsStsExternalId(), equalTo(stsExternalId)); + } + + @Test + void getAwsStsExternalIdReturnsNullIfNotInJSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationConfig.class); + assertThat(objectUnderTest.getAwsStsExternalId(), nullValue()); + } + + @Test + void getAwsStsHeaderOverridesReturnsValueFromDeserializedJSON() { + final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + final Map jsonMap = Map.of("sts_header_overrides", stsHeaderOverrides); + final AwsAuthenticationConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationConfig.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), equalTo(stsHeaderOverrides)); + } + + @Test + void getAwsStsHeaderOverridesReturnsNullIfNotInJSON() { + final Map jsonMap = Collections.emptyMap(); + final AwsAuthenticationConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationConfig.class); + assertThat(objectUnderTest.getAwsStsHeaderOverrides(), nullValue()); + } + + @Test + void authenticateAWSConfigurationShouldReturnWithoutStsRoleArn() throws NoSuchFieldException, IllegalAccessException { + AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", null); + + final DefaultCredentialsProvider mockedCredentialsProvider = mock(DefaultCredentialsProvider.class); + final AwsCredentialsProvider actualCredentialsProvider; + try (final MockedStatic defaultCredentialsProviderMockedStatic = mockStatic(DefaultCredentialsProvider.class)) { + defaultCredentialsProviderMockedStatic.when(DefaultCredentialsProvider::create) + .thenReturn(mockedCredentialsProvider); + actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); + } + + assertThat(actualCredentialsProvider, sameInstance(mockedCredentialsProvider)); + } + + + @Nested + class WithSts { + private StsClient stsClient; + private StsClientBuilder stsClientBuilder; + + @BeforeEach + void setUp() { + stsClient = mock(StsClient.class); + stsClientBuilder = mock(StsClientBuilder.class); + + when(stsClientBuilder.build()).thenReturn(stsClient); + } + + @Test + void authenticateAWSConfigurationShouldReturnWithStsRoleArn() throws NoSuchFieldException, IllegalAccessException { + AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", TEST_ROLE); + + when(stsClientBuilder.region(Region.US_EAST_1)).thenReturn(stsClientBuilder); + final AssumeRoleRequest.Builder assumeRoleRequestBuilder = mock(AssumeRoleRequest.Builder.class); + when(assumeRoleRequestBuilder.roleSessionName(anyString())) + .thenReturn(assumeRoleRequestBuilder); + when(assumeRoleRequestBuilder.roleArn(anyString())) + .thenReturn(assumeRoleRequestBuilder); + + final AwsCredentialsProvider actualCredentialsProvider; + try (final MockedStatic stsClientMockedStatic = mockStatic(StsClient.class); + final MockedStatic assumeRoleRequestMockedStatic = mockStatic(AssumeRoleRequest.class)) { + stsClientMockedStatic.when(StsClient::builder).thenReturn(stsClientBuilder); + assumeRoleRequestMockedStatic.when(AssumeRoleRequest::builder).thenReturn(assumeRoleRequestBuilder); + actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); + } + + assertThat(actualCredentialsProvider, instanceOf(AwsCredentialsProvider.class)); + + verify(assumeRoleRequestBuilder).roleArn(TEST_ROLE); + verify(assumeRoleRequestBuilder).roleSessionName(anyString()); + verify(assumeRoleRequestBuilder).build(); + verifyNoMoreInteractions(assumeRoleRequestBuilder); + } + + @Test + void authenticateAWSConfigurationShouldReturnWithStsRoleArnWhenNoRegion() throws NoSuchFieldException, IllegalAccessException { + AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", null); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", TEST_ROLE); + assertThat(awsAuthenticationOptionsConfig.getAwsRegion(), CoreMatchers.equalTo(null)); + + when(stsClientBuilder.region(null)).thenReturn(stsClientBuilder); + + final AwsCredentialsProvider actualCredentialsProvider; + try (final MockedStatic stsClientMockedStatic = mockStatic(StsClient.class)) { + stsClientMockedStatic.when(StsClient::builder).thenReturn(stsClientBuilder); + actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); + } + + assertThat(actualCredentialsProvider, instanceOf(AwsCredentialsProvider.class)); + } + + @Test + void authenticateAWSConfigurationShouldOverrideSTSHeadersWhenHeaderOverridesSet() throws NoSuchFieldException, IllegalAccessException { + final String headerName1 = UUID.randomUUID().toString(); + final String headerValue1 = UUID.randomUUID().toString(); + final String headerName2 = UUID.randomUUID().toString(); + final String headerValue2 = UUID.randomUUID().toString(); + final Map overrideHeaders = Map.of(headerName1, headerValue1, headerName2, headerValue2); + + AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", TEST_ROLE); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsHeaderOverrides", overrideHeaders); + + when(stsClientBuilder.region(Region.US_EAST_1)).thenReturn(stsClientBuilder); + + final AssumeRoleRequest.Builder assumeRoleRequestBuilder = mock(AssumeRoleRequest.Builder.class); + when(assumeRoleRequestBuilder.roleSessionName(anyString())) + .thenReturn(assumeRoleRequestBuilder); + when(assumeRoleRequestBuilder.roleArn(anyString())) + .thenReturn(assumeRoleRequestBuilder); + when(assumeRoleRequestBuilder.overrideConfiguration(any(Consumer.class))) + .thenReturn(assumeRoleRequestBuilder); + + final AwsCredentialsProvider actualCredentialsProvider; + try (final MockedStatic stsClientMockedStatic = mockStatic(StsClient.class); + final MockedStatic assumeRoleRequestMockedStatic = mockStatic(AssumeRoleRequest.class)) { + stsClientMockedStatic.when(StsClient::builder).thenReturn(stsClientBuilder); + assumeRoleRequestMockedStatic.when(AssumeRoleRequest::builder).thenReturn(assumeRoleRequestBuilder); + actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); + } + + assertThat(actualCredentialsProvider, instanceOf(AwsCredentialsProvider.class)); + + final ArgumentCaptor> configurationCaptor = ArgumentCaptor.forClass(Consumer.class); + + verify(assumeRoleRequestBuilder).roleArn(TEST_ROLE); + verify(assumeRoleRequestBuilder).roleSessionName(anyString()); + verify(assumeRoleRequestBuilder).overrideConfiguration(configurationCaptor.capture()); + verify(assumeRoleRequestBuilder).build(); + verifyNoMoreInteractions(assumeRoleRequestBuilder); + + final Consumer actualOverride = configurationCaptor.getValue(); + + final AwsRequestOverrideConfiguration.Builder configurationBuilder = mock(AwsRequestOverrideConfiguration.Builder.class); + actualOverride.accept(configurationBuilder); + verify(configurationBuilder).putHeader(headerName1, headerValue1); + verify(configurationBuilder).putHeader(headerName2, headerValue2); + verifyNoMoreInteractions(configurationBuilder); + } + + @Test + void authenticateAWSConfigurationShouldNotOverrideSTSHeadersWhenHeaderOverridesAreEmpty() throws NoSuchFieldException, IllegalAccessException { + + AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", TEST_ROLE); + ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsHeaderOverrides", Collections.emptyMap()); + + when(stsClientBuilder.region(Region.US_EAST_1)).thenReturn(stsClientBuilder); + final AssumeRoleRequest.Builder assumeRoleRequestBuilder = mock(AssumeRoleRequest.Builder.class); + when(assumeRoleRequestBuilder.roleSessionName(anyString())) + .thenReturn(assumeRoleRequestBuilder); + when(assumeRoleRequestBuilder.roleArn(anyString())) + .thenReturn(assumeRoleRequestBuilder); + + final AwsCredentialsProvider actualCredentialsProvider; + try (final MockedStatic stsClientMockedStatic = mockStatic(StsClient.class); + final MockedStatic assumeRoleRequestMockedStatic = mockStatic(AssumeRoleRequest.class)) { + stsClientMockedStatic.when(StsClient::builder).thenReturn(stsClientBuilder); + assumeRoleRequestMockedStatic.when(AssumeRoleRequest::builder).thenReturn(assumeRoleRequestBuilder); + actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); + } + + assertThat(actualCredentialsProvider, instanceOf(AwsCredentialsProvider.class)); + + verify(assumeRoleRequestBuilder).roleArn(TEST_ROLE); + verify(assumeRoleRequestBuilder).roleSessionName(anyString()); + verify(assumeRoleRequestBuilder).build(); + verifyNoMoreInteractions(assumeRoleRequestBuilder); + } + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfigTest.java new file mode 100644 index 0000000000..1f2b723919 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfigTest.java @@ -0,0 +1,114 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.configuration; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInfo; +import org.yaml.snakeyaml.Yaml; +import software.amazon.awssdk.regions.Region; +import software.amazon.kinesis.common.InitialPositionInStream; + +import java.io.FileReader; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.util.List; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class KinesisSourceConfigTest { + private static final String SIMPLE_PIPELINE_CONFIG = "simple-pipeline.yaml"; + private static final String SIMPLE_PIPELINE_CONFIG_2 = "simple-pipeline-2.yaml"; + private static final int MINIMAL_CHECKPOINT_INTERVAL_MILLIS = 2 * 60 * 1000; // 2 minute + private static final int DEFAULT_MAX_RECORDS = 10000; + private static final int IDLE_TIME_BETWEEN_READS_IN_MILLIS = 250; + + KinesisSourceConfig kinesisSourceConfig; + + @BeforeEach + void setUp(TestInfo testInfo) throws IOException { + String fileName = testInfo.getTags().stream().findFirst().orElse(""); + Yaml yaml = new Yaml(); + FileReader fileReader = new FileReader(getClass().getClassLoader().getResource(fileName).getFile()); + Object data = yaml.load(fileReader); + ObjectMapper mapper = new ObjectMapper(); + if (data instanceof Map) { + Map propertyMap = (Map) data; + Map logPipelineMap = (Map) propertyMap.get("kinesis-pipeline"); + Map sourceMap = (Map) logPipelineMap.get("source"); + Map kinesisConfigMap = (Map) sourceMap.get("kinesis"); + mapper.registerModule(new JavaTimeModule()); + String json = mapper.writeValueAsString(kinesisConfigMap); + Reader reader = new StringReader(json); + kinesisSourceConfig = mapper.readValue(reader, KinesisSourceConfig.class); + } + } + + @Test + @Tag(SIMPLE_PIPELINE_CONFIG) + void testSourceConfig() { + + assertThat(kinesisSourceConfig, notNullValue()); + assertEquals(KinesisSourceConfig.DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE, kinesisSourceConfig.getNumberOfRecordsToAccumulate()); + assertEquals(KinesisSourceConfig.DEFAULT_TIME_OUT_IN_MILLIS, kinesisSourceConfig.getBufferTimeout()); + assertTrue(kinesisSourceConfig.isAcknowledgments()); + assertEquals(KinesisSourceConfig.DEFAULT_SHARD_ACKNOWLEDGEMENT_TIMEOUT, kinesisSourceConfig.getShardAcknowledgmentTimeout()); + assertThat(kinesisSourceConfig.getAwsAuthenticationConfig(), notNullValue()); + assertEquals(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsRegion(), Region.US_EAST_1); + assertEquals(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsRoleArn(), "arn:aws:iam::123456789012:role/OSI-PipelineRole"); + assertNull(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsExternalId()); + assertNull(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsHeaderOverrides()); + + List streamConfigs = kinesisSourceConfig.getStreams(); + assertNull(kinesisSourceConfig.getCodec()); + assertEquals(kinesisSourceConfig.getConsumerStrategy(), ConsumerStrategy.ENHANCED_FAN_OUT); + assertNull(kinesisSourceConfig.getPollingConfig()); + + for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { + assertTrue(kinesisStreamConfig.getName().contains("stream")); + assertTrue(kinesisStreamConfig.getArn().contains("123456789012:stream/stream")); + assertFalse(kinesisStreamConfig.isEnableCheckPoint()); + assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.LATEST); + assertEquals(kinesisStreamConfig.getCheckPointIntervalInMilliseconds(), MINIMAL_CHECKPOINT_INTERVAL_MILLIS); + } + } + + @Test + @Tag(SIMPLE_PIPELINE_CONFIG_2) + void testSourceConfigWithStreamCodec() { + + assertThat(kinesisSourceConfig, notNullValue()); + assertEquals(KinesisSourceConfig.DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE, kinesisSourceConfig.getNumberOfRecordsToAccumulate()); + assertEquals(KinesisSourceConfig.DEFAULT_TIME_OUT_IN_MILLIS, kinesisSourceConfig.getBufferTimeout()); + assertFalse(kinesisSourceConfig.isAcknowledgments()); + assertEquals(KinesisSourceConfig.DEFAULT_SHARD_ACKNOWLEDGEMENT_TIMEOUT, kinesisSourceConfig.getShardAcknowledgmentTimeout()); + assertThat(kinesisSourceConfig.getAwsAuthenticationConfig(), notNullValue()); + assertEquals(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsRegion(), Region.US_EAST_1); + assertEquals(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsRoleArn(), "arn:aws:iam::123456789012:role/OSI-PipelineRole"); + assertNull(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsExternalId()); + assertNull(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsHeaderOverrides()); + assertNotNull(kinesisSourceConfig.getCodec()); + List streamConfigs = kinesisSourceConfig.getStreams(); + assertEquals(kinesisSourceConfig.getConsumerStrategy(), ConsumerStrategy.POLLING); + assertNotNull(kinesisSourceConfig.getPollingConfig()); + assertEquals(kinesisSourceConfig.getPollingConfig().getMaxPollingRecords(), 10); + assertEquals(kinesisSourceConfig.getPollingConfig().getIdleTimeBetweenReadsInMillis(), 10); + + for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { + assertTrue(kinesisStreamConfig.getName().contains("stream")); + assertTrue(kinesisStreamConfig.getArn().contains("123456789012:stream/stream")); + assertFalse(kinesisStreamConfig.isEnableCheckPoint()); + assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.LATEST); + assertEquals(kinesisStreamConfig.getCheckPointIntervalInMilliseconds(), MINIMAL_CHECKPOINT_INTERVAL_MILLIS); + } + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfigTest.java new file mode 100644 index 0000000000..342e9d70f6 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfigTest.java @@ -0,0 +1,18 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.configuration; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class KinesisStreamPollingConfigTest { + private static final int DEFAULT_MAX_RECORDS = 10000; + private static final int IDLE_TIME_BETWEEN_READS_IN_MILLIS = 250; + + @Test + void testConfig() { + KinesisStreamPollingConfig kinesisStreamPollingConfig = new KinesisStreamPollingConfig(); + assertEquals(kinesisStreamPollingConfig.getMaxPollingRecords(), DEFAULT_MAX_RECORDS); + assertEquals(kinesisStreamPollingConfig.getIdleTimeBetweenReadsInMillis(), IDLE_TIME_BETWEEN_READS_IN_MILLIS); + } + +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessorTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessorTest.java new file mode 100644 index 0000000000..8ccfbd235a --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessorTest.java @@ -0,0 +1,301 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.processor; + +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; +import software.amazon.kinesis.exceptions.InvalidStateException; +import software.amazon.kinesis.exceptions.ShutdownException; +import software.amazon.kinesis.exceptions.ThrottlingException; +import software.amazon.kinesis.lifecycle.events.InitializationInput; +import software.amazon.kinesis.lifecycle.events.ProcessRecordsInput; +import software.amazon.kinesis.lifecycle.events.ShardEndedInput; +import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; +import software.amazon.kinesis.processor.RecordProcessorCheckpointer; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.nio.ByteBuffer; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.kinesis.processor.KinesisRecordProcessor.KINESIS_CHECKPOINT_FAILURES; +import static org.opensearch.dataprepper.plugins.source.kinesis.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSING_ERRORS; +import static org.opensearch.dataprepper.plugins.source.kinesis.processor.KinesisRecordProcessor.KINESIS_STREAM_TAG_KEY; + +public class KinesisRecordProcessorTest { + private KinesisRecordProcessor kinesisRecordProcessor; + private static final String shardId = "123"; + private static final String streamId = "stream-1"; + private static final String codec_plugin_name = "json"; + + private static final int CHECKPOINT_INTERVAL_MS = 1000; + private static final int NUMBER_OF_RECORDS_TO_ACCUMULATE = 10; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private PluginFactory pluginFactory; + + @Mock + private KinesisSourceConfig kinesisSourceConfig; + + @Mock + private KinesisStreamConfig kinesisStreamConfig; + + @Mock + private InitializationInput initializationInput; + + @Mock + private ProcessRecordsInput processRecordsInput; + + @Mock + private RecordProcessorCheckpointer checkpointer; + + @Mock + private Buffer> buffer; + + @Mock + StreamIdentifier streamIdentifier; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + @Mock + private AcknowledgementSet acknowledgementSet; + + @Mock + private Counter recordProcessingErrors; + + @Mock + private Counter checkpointFailures; + + @BeforeEach + public void setup() { + MockitoAnnotations.initMocks(this); + pluginMetrics = mock(PluginMetrics.class); + pluginFactory = mock(PluginFactory.class); + acknowledgementSet = mock(AcknowledgementSet.class); + + when(initializationInput.shardId()).thenReturn(shardId); + when(streamIdentifier.streamName()).thenReturn(streamId); + when(kinesisStreamConfig.getName()).thenReturn(streamId); + PluginModel pluginModel = mock(PluginModel.class); + when(pluginModel.getPluginName()).thenReturn(codec_plugin_name); + when(pluginModel.getPluginSettings()).thenReturn(Collections.emptyMap()); + when(kinesisSourceConfig.getCodec()).thenReturn(pluginModel); + + InputCodec codec = mock(InputCodec.class); + when(pluginFactory.loadPlugin(eq(InputCodec.class), any())).thenReturn(codec); + when(kinesisStreamConfig.getCheckPointIntervalInMilliseconds()).thenReturn(CHECKPOINT_INTERVAL_MS); + when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); + when(kinesisSourceConfig.getStreams()).thenReturn(List.of(kinesisStreamConfig)); + when(processRecordsInput.checkpointer()).thenReturn(checkpointer); + } + + @Test + void testProcessRecordsWithoutAcknowledgementsCheckpointsEnabled() + throws Exception { + when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); + when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(true); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + + kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor.initialize(initializationInput); + + Thread.sleep(2000); + + kinesisRecordProcessor.processRecords(processRecordsInput); + + verify(checkpointer).checkpoint(); + verify(buffer).writeAll(anyCollection(), anyInt()); + verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); + } + + @Test + void testProcessRecordsWithNDJsonInputCodec() + throws Exception { + when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); + when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(true); + + PluginModel pluginModel = mock(PluginModel.class); + when(pluginModel.getPluginName()).thenReturn("ndjson"); + when(pluginModel.getPluginSettings()).thenReturn(Collections.emptyMap()); + when(kinesisSourceConfig.getCodec()).thenReturn(pluginModel); + + InputCodec codec = mock(InputCodec.class); + when(pluginFactory.loadPlugin(eq(InputCodec.class), any())).thenReturn(codec); + + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor.initialize(initializationInput); + + Thread.sleep(2000); + kinesisRecordProcessor.processRecords(processRecordsInput); + + verify(checkpointer).checkpoint(); + verify(buffer).writeAll(anyCollection(), anyInt()); + verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); + } + + + @Test + void testProcessRecordsNoThrowException() + throws Exception { + when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); + when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); + + final Throwable exception = mock(RuntimeException.class); + doThrow(exception).when(buffer).writeAll(any(), anyInt()); + + recordProcessingErrors = mock(Counter.class); + when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(recordProcessingErrors); + + kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor.initialize(initializationInput); + + assertDoesNotThrow(() -> kinesisRecordProcessor.processRecords(processRecordsInput)); + verify(recordProcessingErrors, times(1)).increment(); + } + + @Test + void testProcessRecordsWithoutAcknowledgementsAndCheckpoints() + throws Exception { + when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); + when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + + kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor.initialize(initializationInput); + + kinesisRecordProcessor.processRecords(processRecordsInput); + + verifyNoInteractions(checkpointer); + verify(buffer).writeAll(anyCollection(), anyInt()); + verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); + } + + @Test + void testProcessRecordsWithAcknowledgements() + throws Exception { + when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(true); + when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + + kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor.initialize(initializationInput); + + kinesisRecordProcessor.processRecords(processRecordsInput); + + verifyNoInteractions(checkpointer); + verify(buffer).writeAll(anyCollection(), anyInt()); + verify(acknowledgementSetManager, times(1)).create(any(), any(Duration.class)); + } + + @Test + void testShardEndedFlushCalled() throws Exception { + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + ShardEndedInput shardEndedInput = mock(ShardEndedInput.class); + when(shardEndedInput.checkpointer()).thenReturn(checkpointer); + + mockKinesisRecordProcessor.shardEnded(shardEndedInput); + + verify(shardEndedInput).checkpointer(); + } + + @ParameterizedTest + @ValueSource(classes = {ShutdownException.class, ThrottlingException.class, InvalidStateException.class}) + void testShardEndedCheckpointerThrowsNoThrowException(final Class exceptionType) throws Exception { + checkpointFailures = mock(Counter.class); + when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(checkpointFailures); + + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + ShardEndedInput shardEndedInput = mock(ShardEndedInput.class); + when(shardEndedInput.checkpointer()).thenReturn(checkpointer); + doThrow(exceptionType).when(checkpointer).checkpoint(); + + assertDoesNotThrow(() -> mockKinesisRecordProcessor.shardEnded(shardEndedInput)); + + verify(checkpointer).checkpoint(); + verify(shardEndedInput, times(1)).checkpointer(); + verify(checkpointFailures, times(1)).increment(); + } + + @Test + void testShutdownRequested() { + checkpointFailures = mock(Counter.class); + when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(checkpointFailures); + + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + ShutdownRequestedInput shutdownRequestedInput = mock(ShutdownRequestedInput.class); + when(shutdownRequestedInput.checkpointer()).thenReturn(checkpointer); + + mockKinesisRecordProcessor.shutdownRequested(shutdownRequestedInput); + + verify(shutdownRequestedInput).checkpointer(); + verify(checkpointFailures, times(0)).increment(); + } + + @ParameterizedTest + @ValueSource(classes = {ShutdownException.class, ThrottlingException.class, InvalidStateException.class}) + void testShutdownRequestedCheckpointerThrowsNoThrowException(final Class exceptionType) throws Exception { + checkpointFailures = mock(Counter.class); + when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(checkpointFailures); + + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + ShutdownRequestedInput shutdownRequestedInput = mock(ShutdownRequestedInput.class); + when(shutdownRequestedInput.checkpointer()).thenReturn(checkpointer); + doThrow(exceptionType).when(checkpointer).checkpoint(); + + assertDoesNotThrow(() -> mockKinesisRecordProcessor.shutdownRequested(shutdownRequestedInput)); + + verify(checkpointer).checkpoint(); + verify(shutdownRequestedInput, times(1)).checkpointer(); + verify(checkpointFailures, times(1)).increment(); + } + + private List createInputKinesisClientRecords() { + List kinesisClientRecords = new ArrayList<>(); + for (int i = 0; i< KinesisRecordProcessorTest.NUMBER_OF_RECORDS_TO_ACCUMULATE; i++) { + Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder().data(ByteBuffer.wrap(event.toJsonString().getBytes())).build(); + kinesisClientRecords.add(kinesisClientRecord); + } + return kinesisClientRecords; + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactoryTest.java new file mode 100644 index 0000000000..1bfe08ad6a --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactoryTest.java @@ -0,0 +1,84 @@ +package org.opensearch.dataprepper.plugins.source.kinesis.processor; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.configuration.PluginModel; +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.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import software.amazon.kinesis.common.StreamIdentifier; + +import java.util.Collections; +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertInstanceOf; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class KinesisShardRecordProcessorFactoryTest { + private KinesisShardRecordProcessorFactory kinesisShardRecordProcessorFactory; + + private static final String streamId = "stream-1"; + private static final String codec_plugin_name = "json"; + + @Mock + private Buffer> buffer; + + @Mock + StreamIdentifier streamIdentifier; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private PluginFactory pluginFactory; + + @Mock + private KinesisSourceConfig kinesisSourceConfig; + + @Mock + private KinesisStreamConfig kinesisStreamConfig; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + @BeforeEach + void setup() { + MockitoAnnotations.initMocks(this); + + PluginModel pluginModel = mock(PluginModel.class); + when(pluginModel.getPluginName()).thenReturn(codec_plugin_name); + when(pluginModel.getPluginSettings()).thenReturn(Collections.emptyMap()); + when(kinesisSourceConfig.getCodec()).thenReturn(pluginModel); + + InputCodec codec = mock(InputCodec.class); + when(pluginFactory.loadPlugin(eq(InputCodec.class), any())).thenReturn(codec); + + when(streamIdentifier.streamName()).thenReturn(streamId); + when(kinesisStreamConfig.getName()).thenReturn(streamId); + when(kinesisSourceConfig.getStreams()).thenReturn(List.of(kinesisStreamConfig)); + } + + @Test + void testKinesisRecordProcessFactoryReturnsKinesisRecordProcessor() { + kinesisShardRecordProcessorFactory = new KinesisShardRecordProcessorFactory(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory); + assertInstanceOf(KinesisRecordProcessor.class, kinesisShardRecordProcessorFactory.shardRecordProcessor(streamIdentifier)); + } + + @Test + void testKinesisRecordProcessFactoryDefaultUnsupported() { + kinesisShardRecordProcessorFactory = new KinesisShardRecordProcessorFactory(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory); + assertThrows(UnsupportedOperationException.class, () -> kinesisShardRecordProcessorFactory.shardRecordProcessor()); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline-2.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline-2.yaml new file mode 100644 index 0000000000..46234d50fb --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline-2.yaml @@ -0,0 +1,21 @@ +kinesis-pipeline: + source: + kinesis: + streams: + - stream_name: "stream-1" + stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-1" + - stream_name: "stream-2" + stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-2" + - stream_name: "stream-3" + stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-3" + codec: + ndjson: + aws: + sts_role_arn: "arn:aws:iam::123456789012:role/OSI-PipelineRole" + region: "us-east-1" + consumer_strategy: "Polling" + polling: + maxPollingRecords: 10 + idleTimeBetweenReadsInMillis: 10 + sink: + - stdout: \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline.yaml new file mode 100644 index 0000000000..07caf50d02 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline.yaml @@ -0,0 +1,16 @@ +kinesis-pipeline: + source: + kinesis: + streams: + - stream_name: "stream-1" + stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-1" + - stream_name: "stream-2" + stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-2" + - stream_name: "stream-3" + stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-3" + aws: + sts_role_arn: "arn:aws:iam::123456789012:role/OSI-PipelineRole" + region: "us-east-1" + acknowledgments: true + sink: + - stdout: \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index 97742e8576..4328fa9aac 100644 --- a/settings.gradle +++ b/settings.gradle @@ -183,4 +183,4 @@ include 'data-prepper-plugins:http-common' include 'data-prepper-plugins:aws-lambda' //include 'data-prepper-plugins:dummy-plugin' include 'data-prepper-plugin-schema' - +include 'data-prepper-plugins:kinesis-source' \ No newline at end of file From 1523b8a7554785e783fb8cee220f10f9013c04c5 Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Fri, 16 Aug 2024 09:03:42 -0700 Subject: [PATCH 2/8] Add support for extensions to configure lease coordination table. Signed-off-by: Souvik Bose --- .../kinesis-source/build.gradle | 4 ++ .../DefaultKinesisLeaseConfigSupplier.java | 17 ++++++ .../kinesis/extension/KinesisLeaseConfig.java | 10 ++++ .../KinesisLeaseConfigExtension.java | 21 ++++++++ .../extension/KinesisLeaseConfigProvider.java | 23 ++++++++ .../extension/KinesisLeaseConfigSupplier.java | 10 ++++ .../source}/ClientFactory.java | 4 +- .../source}/KinesisMultiStreamTracker.java | 6 +-- .../source}/KinesisService.java | 23 ++++---- .../source}/KinesisSource.java | 13 +++-- .../source}/WorkerIdentifierGenerator.java | 2 +- .../AwsAuthenticationConfig.java | 2 +- .../configuration/ConsumerStrategy.java | 2 +- .../configuration/KinesisSourceConfig.java | 6 +-- .../configuration/KinesisStreamConfig.java | 2 +- .../KinesisStreamPollingConfig.java | 2 +- .../metrics/MicrometerFilterMetricsScope.java | 38 +++++++++++++ .../metrics/MicrometerMetricFactory.java | 18 +++++++ .../processor/KinesisRecordProcessor.java | 8 +-- .../KinesisShardRecordProcessorFactory.java | 4 +- ...DefaultKinesisLeaseConfigSupplierTest.java | 44 +++++++++++++++ .../KinesisLeaseConfigExtensionTest.java | 38 +++++++++++++ .../KinesisLeaseConfigProviderTest.java | 47 ++++++++++++++++ .../extension/KinesisLeaseConfigTest.java | 53 +++++++++++++++++++ .../source}/ClientFactoryTest.java | 4 +- .../KinesisMultiStreamTrackerTest.java | 6 +-- .../source}/KinesisServiceTest.java | 34 ++++++++---- .../source}/KinesisSourceTest.java | 24 ++++++--- .../AwsAuthenticationConfigTest.java | 2 +- .../KinesisSourceConfigTest.java | 2 +- .../KinesisStreamPollingConfigTest.java | 2 +- .../processor/KinesisRecordProcessorTest.java | 12 ++--- ...inesisShardRecordProcessorFactoryTest.java | 6 +-- .../simple_pipeline_with_extensions.yaml | 3 ++ 34 files changed, 425 insertions(+), 67 deletions(-) create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplier.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProvider.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/ClientFactory.java (94%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/KinesisMultiStreamTracker.java (95%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/KinesisService.java (88%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/KinesisSource.java (79%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/WorkerIdentifierGenerator.java (90%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/configuration/AwsAuthenticationConfig.java (98%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/configuration/ConsumerStrategy.java (86%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/configuration/KinesisSourceConfig.java (91%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/configuration/KinesisStreamConfig.java (93%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/configuration/KinesisStreamPollingConfig.java (88%) create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerFilterMetricsScope.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerMetricFactory.java rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/processor/KinesisRecordProcessor.java (97%) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/processor/KinesisShardRecordProcessorFactory.java (94%) create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtensionTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProviderTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/ClientFactoryTest.java (94%) rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/KinesisMultiStreamTrackerTest.java (97%) rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/KinesisServiceTest.java (89%) rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/KinesisSourceTest.java (83%) rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/configuration/AwsAuthenticationConfigTest.java (99%) rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/configuration/KinesisSourceConfigTest.java (99%) rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/configuration/KinesisStreamPollingConfigTest.java (90%) rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/processor/KinesisRecordProcessorTest.java (97%) rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/{source/kinesis => kinesis/source}/processor/KinesisShardRecordProcessorFactoryTest.java (94%) create mode 100644 data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml diff --git a/data-prepper-plugins/kinesis-source/build.gradle b/data-prepper-plugins/kinesis-source/build.gradle index 400fe22c00..f5bb41385a 100644 --- a/data-prepper-plugins/kinesis-source/build.gradle +++ b/data-prepper-plugins/kinesis-source/build.gradle @@ -29,6 +29,10 @@ dependencies { testImplementation platform('org.junit:junit-bom:5.9.1') testImplementation 'org.junit.jupiter:junit-jupiter' testImplementation project(':data-prepper-test-event') + testImplementation project(':data-prepper-core') + testImplementation project(':data-prepper-plugin-framework') + testImplementation project(':data-prepper-pipeline-parser') + testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplier.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplier.java new file mode 100644 index 0000000000..f7e4bd9e87 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplier.java @@ -0,0 +1,17 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import java.util.Optional; + +public class DefaultKinesisLeaseConfigSupplier implements KinesisLeaseConfigSupplier { + + private KinesisLeaseConfig kinesisLeaseConfig; + + public DefaultKinesisLeaseConfigSupplier(final KinesisLeaseConfig kinesisLeaseConfig) { + this.kinesisLeaseConfig = kinesisLeaseConfig; + } + + @Override + public Optional getKinesisExtensionLeaseConfig() { + return kinesisLeaseConfig != null ? Optional.of(kinesisLeaseConfig) : Optional.empty(); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java new file mode 100644 index 0000000000..fbafce9d7c --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java @@ -0,0 +1,10 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.Getter; + +@Getter +public class KinesisLeaseConfig { + @JsonProperty("lease_coordination_table") + private String leaseCoordinationTable; +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java new file mode 100644 index 0000000000..5e6ad46efe --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java @@ -0,0 +1,21 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import org.opensearch.dataprepper.model.annotations.DataPrepperExtensionPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.plugin.ExtensionPlugin; +import org.opensearch.dataprepper.model.plugin.ExtensionPoints; + +@DataPrepperExtensionPlugin(modelType = KinesisLeaseConfig.class, rootKeyJsonPath = "/kinesis_lease_config", allowInPipelineConfigurations = true) +public class KinesisLeaseConfigExtension implements ExtensionPlugin { + + private KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; + @DataPrepperPluginConstructor + public KinesisLeaseConfigExtension(final KinesisLeaseConfig kinesisLeaseConfig) { + this.kinesisLeaseConfigSupplier = new DefaultKinesisLeaseConfigSupplier(kinesisLeaseConfig); + } + + @Override + public void apply(final ExtensionPoints extensionPoints) { + extensionPoints.addExtensionProvider(new KinesisLeaseConfigProvider(this.kinesisLeaseConfigSupplier)); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProvider.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProvider.java new file mode 100644 index 0000000000..30165c001c --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProvider.java @@ -0,0 +1,23 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import org.opensearch.dataprepper.model.plugin.ExtensionProvider; + +import java.util.Optional; + +class KinesisLeaseConfigProvider implements ExtensionProvider { + private final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; + + public KinesisLeaseConfigProvider(final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier) { + this.kinesisLeaseConfigSupplier = kinesisLeaseConfigSupplier; + } + + @Override + public Optional provideInstance(Context context) { + return Optional.of(this.kinesisLeaseConfigSupplier); + } + + @Override + public Class supportedClass() { + return KinesisLeaseConfigSupplier.class; + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java new file mode 100644 index 0000000000..3db85d3ca5 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java @@ -0,0 +1,10 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import java.util.Optional; + +public interface KinesisLeaseConfigSupplier { + + default Optional getKinesisExtensionLeaseConfig() { + return Optional.empty(); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java similarity index 94% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactory.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java index 5abf9185d0..dd12324f1e 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactory.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java @@ -1,8 +1,8 @@ -package org.opensearch.dataprepper.plugins.source.kinesis; +package org.opensearch.dataprepper.plugins.kinesis.source; import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTracker.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java similarity index 95% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTracker.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java index 5da7a3d160..166be331ea 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTracker.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java @@ -1,7 +1,7 @@ -package org.opensearch.dataprepper.plugins.source.kinesis; +package org.opensearch.dataprepper.plugins.kinesis.source; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisService.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java similarity index 88% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisService.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java index 0d48aaf161..dd0ad61249 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisService.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java @@ -1,6 +1,5 @@ -package org.opensearch.dataprepper.plugins.source.kinesis; +package org.opensearch.dataprepper.plugins.kinesis.source; -import io.micrometer.core.instrument.util.StringUtils; import lombok.Setter; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; @@ -9,9 +8,10 @@ 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.kinesis.configuration.ConsumerStrategy; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.processor.KinesisShardRecordProcessorFactory; +import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfigSupplier; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.ConsumerStrategy; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisShardRecordProcessorFactory; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; @@ -56,8 +56,9 @@ public KinesisService(final KinesisSourceConfig sourceConfig, final PluginMetrics pluginMetrics, final PluginFactory pluginFactory, final PipelineDescription pipelineDescription, - final AcknowledgementSetManager acknowledgementSetManager - ){ + final AcknowledgementSetManager acknowledgementSetManager, + final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier + ){ this.sourceConfig = sourceConfig; this.pluginMetrics = pluginMetrics; this.pluginFactory = pluginFactory; @@ -67,8 +68,10 @@ public KinesisService(final KinesisSourceConfig sourceConfig, this.cloudWatchClient = clientFactory.buildCloudWatchAsyncClient(); this.pipelineName = pipelineDescription.getPipelineName(); this.applicationName = pipelineName; - this.tableName = StringUtils.isNotEmpty(sourceConfig.getLeaseCoordinationTable()) ? - sourceConfig.getLeaseCoordinationTable() : applicationName; + if (kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().isEmpty()) { + throw new IllegalStateException("Lease Coordination table should be provided!"); + } + this.tableName = kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable(); this.executorService = Executors.newFixedThreadPool(1); } @@ -115,7 +118,7 @@ public Scheduler createScheduler(final Buffer> buffer) { new KinesisMultiStreamTracker(kinesisClient, sourceConfig, applicationName), applicationName, kinesisClient, dynamoDbClient, cloudWatchClient, new WorkerIdentifierGenerator().generate(), processorFactory - ).tableName(applicationName); + ).tableName(tableName); ConsumerStrategy consumerStrategy = sourceConfig.getConsumerStrategy(); if (consumerStrategy == ConsumerStrategy.POLLING) { diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSource.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java similarity index 79% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSource.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java index 98069b5bb8..a88cff0de0 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSource.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis; +package org.opensearch.dataprepper.plugins.kinesis.source; import lombok.Setter; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; @@ -12,7 +12,8 @@ import org.opensearch.dataprepper.model.source.Source; import org.opensearch.dataprepper.model.configuration.PipelineDescription; import org.opensearch.dataprepper.model.plugin.PluginFactory; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfigSupplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -20,6 +21,7 @@ public class KinesisSource implements Source> { private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class); private final KinesisSourceConfig kinesisSourceConfig; + private final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; @Setter private KinesisService kinesisService; @@ -30,10 +32,13 @@ public KinesisSource(final KinesisSourceConfig kinesisSourceConfig, final PluginFactory pluginFactory, final PipelineDescription pipelineDescription, final AwsCredentialsSupplier awsCredentialsSupplier, - final AcknowledgementSetManager acknowledgementSetManager) { + final AcknowledgementSetManager acknowledgementSetManager, + final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier) { this.kinesisSourceConfig = kinesisSourceConfig; + this.kinesisLeaseConfigSupplier = kinesisLeaseConfigSupplier; ClientFactory clientFactory = new ClientFactory(awsCredentialsSupplier, kinesisSourceConfig.getAwsAuthenticationConfig()); - this.kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager); + this.kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); } @Override public void start(final Buffer> buffer) { diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/WorkerIdentifierGenerator.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java similarity index 90% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/WorkerIdentifierGenerator.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java index 9390e25920..7686861a87 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/WorkerIdentifierGenerator.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis; +package org.opensearch.dataprepper.plugins.kinesis.source; import java.net.InetAddress; import java.net.UnknownHostException; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java similarity index 98% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfig.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java index d51a7ee6a4..27774828ef 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.configuration; +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; /* * Copyright OpenSearch Contributors diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/ConsumerStrategy.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java similarity index 86% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/ConsumerStrategy.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java index 7b7a7ddf9d..fa507f55d5 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/ConsumerStrategy.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.configuration; +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; import com.fasterxml.jackson.annotation.JsonValue; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java similarity index 91% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfig.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java index 3aefe7bece..4be5bb7f95 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.configuration; +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.Valid; @@ -48,10 +48,6 @@ public class KinesisSourceConfig { @JsonProperty("polling") private KinesisStreamPollingConfig pollingConfig; - @JsonProperty("lease_coordination_table") - @Getter - private String leaseCoordinationTable; - @Getter @JsonProperty("codec") private PluginModel codec; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java similarity index 93% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamConfig.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java index 49e3a5743f..3faf55469c 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.configuration; +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.Valid; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java similarity index 88% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfig.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java index e48908d7f8..36dbedbdd6 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.configuration; +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; import com.fasterxml.jackson.annotation.JsonProperty; import lombok.Getter; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerFilterMetricsScope.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerFilterMetricsScope.java new file mode 100644 index 0000000000..bfab36170f --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerFilterMetricsScope.java @@ -0,0 +1,38 @@ +//package org.opensearch.dataprepper.plugins.kinesis.source.metrics; +// +//import com.google.common.collect.ImmutableSet; +//import org.opensearch.dataprepper.metrics.PluginMetrics; +//import org.slf4j.Logger; +//import org.slf4j.LoggerFactory; +//import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; +//import software.amazon.kinesis.metrics.FilteringMetricsScope; +//import software.amazon.kinesis.metrics.MetricsLevel; +// +//public class MicrometerFilterMetricsScope extends FilteringMetricsScope { +// +// private static final Logger LOG = LoggerFactory.getLogger(MicrometerFilterMetricsScope.class); +// +// private final PluginMetrics pluginMetrics; +// +// public MicrometerFilterMetricsScope(final PluginMetrics pluginMetrics) { +// super(MetricsLevel.SUMMARY, ImmutableSet.of(METRICS_DIMENSIONS_ALL)); +// this.pluginMetrics = pluginMetrics; +// } +// +// @Override +// public void addData(String name, double value, StandardUnit unit) { +// // TODO: report metrics. +// LOG.info("{} {} {}", name, value, unit); +// } +// +// @Override +// public void addData(String name, double value, StandardUnit unit, MetricsLevel level) { +// // TODO: report metrics. +// LOG.info("{} {} {} {}", name, value, unit, level); +// } +// +// @Override +// public void end() { +// // TODO: Update this +// } +//} \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerMetricFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerMetricFactory.java new file mode 100644 index 0000000000..114e1ca4fa --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerMetricFactory.java @@ -0,0 +1,18 @@ +//package org.opensearch.dataprepper.plugins.kinesis.source.metrics; +// +//import org.opensearch.dataprepper.metrics.PluginMetrics; +//import software.amazon.kinesis.metrics.MetricsFactory; +//import software.amazon.kinesis.metrics.MetricsScope; +// +//public class MicrometerMetricFactory implements MetricsFactory { +// +// final PluginMetrics pluginMetrics; +// +// public MicrometerMetricFactory(PluginMetrics pluginMetrics) { +// this.pluginMetrics = pluginMetrics; +// } +// +// public MetricsScope createMetrics() { +// return new MicrometerFilterMetricsScope(pluginMetrics); +// } +//} \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessor.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java similarity index 97% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessor.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java index 0a7c7c0ebe..bfa7c2c023 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessor.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.processor; +package org.opensearch.dataprepper.plugins.kinesis.source.processor; import io.micrometer.core.instrument.Counter; import org.opensearch.dataprepper.metrics.PluginMetrics; @@ -11,9 +11,9 @@ 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.kinesis.KinesisSource; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.KinesisSource; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.kinesis.common.StreamIdentifier; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java similarity index 94% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactory.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java index 91294857b4..f551c503e5 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactory.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.processor; +package org.opensearch.dataprepper.plugins.kinesis.source.processor; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; @@ -6,7 +6,7 @@ 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.kinesis.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java new file mode 100644 index 0000000000..9914c30fbf --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java @@ -0,0 +1,44 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.util.Optional; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class DefaultKinesisLeaseConfigSupplierTest { + private static final String LEASE_COORDINATION_TABLE = "lease-table"; + @Mock + KinesisLeaseConfig kinesisLeaseConfig; + + private DefaultKinesisLeaseConfigSupplier createObjectUnderTest() { + return new DefaultKinesisLeaseConfigSupplier(kinesisLeaseConfig); + } + + @Test + void testGetters() { + when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(LEASE_COORDINATION_TABLE); + KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier = createObjectUnderTest(); + assertThat(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable(), equalTo(LEASE_COORDINATION_TABLE)); + } + + @Test + void testGettersWithNullTableConfig() { + when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(null); + DefaultKinesisLeaseConfigSupplier defaultKinesisLeaseConfigSupplier = createObjectUnderTest(); + assertThat(defaultKinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable(), equalTo(null)); + + } + + @Test + void testGettersWithNullConfig() { + KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier = new DefaultKinesisLeaseConfigSupplier(null); + assertThat(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig(), equalTo(Optional.empty())); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtensionTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtensionTest.java new file mode 100644 index 0000000000..4476d28def --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtensionTest.java @@ -0,0 +1,38 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import org.junit.jupiter.api.Test; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.opensearch.dataprepper.model.plugin.ExtensionPoints; +import org.opensearch.dataprepper.model.plugin.ExtensionProvider; + +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; + +public class KinesisLeaseConfigExtensionTest { + @Mock + private ExtensionPoints extensionPoints; + + @Mock + private KinesisLeaseConfig kinesisLeaseConfig; + + private KinesisLeaseConfigExtension createObjectUnderTest() { + return new KinesisLeaseConfigExtension(kinesisLeaseConfig); + } + + @Test + void applyShouldAddExtensionProvider() { + extensionPoints = mock(ExtensionPoints.class); + createObjectUnderTest().apply(extensionPoints); + final ArgumentCaptor extensionProviderArgumentCaptor = + ArgumentCaptor.forClass(ExtensionProvider.class); + + verify(extensionPoints).addExtensionProvider(extensionProviderArgumentCaptor.capture()); + + final ExtensionProvider actualExtensionProvider = extensionProviderArgumentCaptor.getValue(); + + assertThat(actualExtensionProvider, instanceOf(KinesisLeaseConfigProvider.class)); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProviderTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProviderTest.java new file mode 100644 index 0000000000..50bc02d628 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProviderTest.java @@ -0,0 +1,47 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.plugin.ExtensionProvider; + +import java.util.Optional; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.CoreMatchers.sameInstance; +import static org.hamcrest.MatcherAssert.assertThat; + +@ExtendWith(MockitoExtension.class) +public class KinesisLeaseConfigProviderTest { + @Mock + private KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; + + @Mock + private ExtensionProvider.Context context; + + private KinesisLeaseConfigProvider createObjectUnderTest() { + return new KinesisLeaseConfigProvider(kinesisLeaseConfigSupplier); + } + + @Test + void supportedClassReturnsKinesisSourceConfigSupplier() { + assertThat(createObjectUnderTest().supportedClass(), equalTo(KinesisLeaseConfigSupplier.class)); + } + + @Test + void provideInstanceReturnsKinesisSourceConfigSupplierFromConstructor() { + final KinesisLeaseConfigProvider objectUnderTest = createObjectUnderTest(); + + final Optional optionalKinesisSourceConfigSupplier = objectUnderTest.provideInstance(context); + assertThat(optionalKinesisSourceConfigSupplier, notNullValue()); + assertThat(optionalKinesisSourceConfigSupplier.isPresent(), equalTo(true)); + assertThat(optionalKinesisSourceConfigSupplier.get(), equalTo(kinesisLeaseConfigSupplier)); + + final Optional anotherOptionalKinesisSourceConfigSupplier = objectUnderTest.provideInstance(context); + assertThat(anotherOptionalKinesisSourceConfigSupplier, notNullValue()); + assertThat(anotherOptionalKinesisSourceConfigSupplier.isPresent(), equalTo(true)); + assertThat(anotherOptionalKinesisSourceConfigSupplier.get(), sameInstance(optionalKinesisSourceConfigSupplier.get())); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java new file mode 100644 index 0000000000..9731a7fda9 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java @@ -0,0 +1,53 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.model.types.ByteCount; +import org.opensearch.dataprepper.parser.model.DataPrepperConfiguration; +import org.opensearch.dataprepper.pipeline.parser.ByteCountDeserializer; +import org.opensearch.dataprepper.pipeline.parser.DataPrepperDurationDeserializer; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; + +import java.io.File; +import java.io.IOException; +import java.io.Reader; +import java.io.StringReader; +import java.time.Duration; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.notNullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +public class KinesisLeaseConfigTest { + private static SimpleModule simpleModule = new SimpleModule() + .addDeserializer(Duration.class, new DataPrepperDurationDeserializer()) + .addDeserializer(ByteCount.class, new ByteCountDeserializer()); + private static ObjectMapper OBJECT_MAPPER = new ObjectMapper(new YAMLFactory()).registerModule(simpleModule); + + private KinesisLeaseConfig makeConfig(String filePath) throws IOException { + final File configurationFile = new File(filePath); + final DataPrepperConfiguration dataPrepperConfiguration = OBJECT_MAPPER.readValue(configurationFile, DataPrepperConfiguration.class); + assertThat(dataPrepperConfiguration, notNullValue()); + assertThat(dataPrepperConfiguration.getPipelineExtensions(), notNullValue()); + final Map kinesisLeaseConfigMap = + (Map) dataPrepperConfiguration.getPipelineExtensions().getExtensionMap().get("kinesis_lease_config"); + String json = OBJECT_MAPPER.writeValueAsString(kinesisLeaseConfigMap); + Reader reader = new StringReader(json); + return OBJECT_MAPPER.readValue(reader, KinesisLeaseConfig.class); + } + + + @Test + void testConfigWithTestExtension() throws IOException { + final KinesisLeaseConfig kinesisLeaseConfig = makeConfig( + "src/test/resources/simple_pipeline_with_extensions.yaml"); + + assertNotNull(kinesisLeaseConfig.getLeaseCoordinationTable()); + assertEquals(kinesisLeaseConfig.getLeaseCoordinationTable(), "kinesis-pipeline-kcl"); + + } + +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java similarity index 94% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactoryTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java index 3365a1e690..f2e4ed7797 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/ClientFactoryTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java @@ -1,8 +1,8 @@ -package org.opensearch.dataprepper.plugins.source.kinesis; +package org.opensearch.dataprepper.plugins.kinesis.source; import org.junit.jupiter.api.Test; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; import org.opensearch.dataprepper.test.helper.ReflectivelySetField; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTrackerTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTrackerTest.java similarity index 97% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTrackerTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTrackerTest.java index 637cb16375..4f90243939 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisMultiStreamTrackerTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTrackerTest.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis; +package org.opensearch.dataprepper.plugins.kinesis.source; import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Assertions; @@ -6,8 +6,8 @@ import org.junit.jupiter.api.Test; import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.awssdk.services.kinesis.model.DescribeStreamRequest; import software.amazon.awssdk.services.kinesis.model.DescribeStreamResponse; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisServiceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java similarity index 89% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisServiceTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java index 3a631dae97..d523a1c768 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisServiceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis; +package org.opensearch.dataprepper.plugins.kinesis.source; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -10,11 +10,13 @@ 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.kinesis.configuration.AwsAuthenticationConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.ConsumerStrategy; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamPollingConfig; +import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfig; +import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfigSupplier; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.ConsumerStrategy; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamPollingConfig; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; @@ -31,6 +33,7 @@ import java.util.ArrayList; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -98,6 +101,12 @@ public class KinesisServiceTest { @Mock private Scheduler scheduler; + @Mock + KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; + + @Mock + KinesisLeaseConfig kinesisLeaseConfig; + @BeforeEach void setup() { awsAuthenticationConfig = mock(AwsAuthenticationConfig.class); @@ -111,6 +120,10 @@ void setup() { scheduler = mock(Scheduler.class); pipelineDescription = mock(PipelineDescription.class); buffer = mock(Buffer.class); + kinesisLeaseConfigSupplier = mock(KinesisLeaseConfigSupplier.class); + kinesisLeaseConfig = mock(KinesisLeaseConfig.class); + when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn("kinesis-lease-table"); + when(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig()).thenReturn(Optional.ofNullable(kinesisLeaseConfig)); when(awsAuthenticationConfig.getAwsRegion()).thenReturn(Region.of("us-west-2")); when(awsAuthenticationConfig.getAwsStsRoleArn()).thenReturn(UUID.randomUUID().toString()); @@ -156,7 +169,8 @@ void setup() { } public KinesisService createObjectUnderTest() { - return new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager); + return new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); } @Test @@ -168,7 +182,8 @@ void testServiceStart() { @Test void testCreateScheduler() { - KinesisService kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager); + KinesisService kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); Scheduler schedulerObjectUnderTest = kinesisService.createScheduler(buffer); assertNotNull(schedulerObjectUnderTest); @@ -185,7 +200,8 @@ void testCreateScheduler() { @Test void testCreateSchedulerWithPollingStrategy() { when(kinesisSourceConfig.getConsumerStrategy()).thenReturn(ConsumerStrategy.POLLING); - KinesisService kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager); + KinesisService kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); Scheduler schedulerObjectUnderTest = kinesisService.createScheduler(buffer); assertNotNull(schedulerObjectUnderTest); diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSourceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java similarity index 83% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSourceTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java index d367780571..69249b82e4 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/KinesisSourceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis; +package org.opensearch.dataprepper.plugins.kinesis.source; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -11,13 +11,16 @@ 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.kinesis.configuration.AwsAuthenticationConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfig; +import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfigSupplier; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; import software.amazon.awssdk.regions.Region; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; import static org.hamcrest.MatcherAssert.assertThat; @@ -58,6 +61,12 @@ public class KinesisSourceTest { @Mock KinesisService kinesisService; + @Mock + KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; + + @Mock + KinesisLeaseConfig kinesisLeaseConfig; + @BeforeEach void setup() { pluginMetrics = mock(PluginMetrics.class); @@ -68,7 +77,10 @@ void setup() { awsAuthenticationConfig = mock(AwsAuthenticationConfig.class); acknowledgementSetManager = mock(AcknowledgementSetManager.class); kinesisService = mock(KinesisService.class); - + kinesisLeaseConfigSupplier = mock(KinesisLeaseConfigSupplier.class); + kinesisLeaseConfig = mock(KinesisLeaseConfig.class); + when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn("kinesis-lease-table"); + when(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig()).thenReturn(Optional.ofNullable(kinesisLeaseConfig)); when(awsAuthenticationConfig.getAwsRegion()).thenReturn(Region.of("us-west-2")); when(awsAuthenticationConfig.getAwsStsRoleArn()).thenReturn(UUID.randomUUID().toString()); when(awsAuthenticationConfig.getAwsStsExternalId()).thenReturn(UUID.randomUUID().toString()); @@ -79,7 +91,7 @@ void setup() { } public KinesisSource createObjectUnderTest() { - return new KinesisSource(kinesisSourceConfig, pluginMetrics, pluginFactory, pipelineDescription, awsCredentialsSupplier, acknowledgementSetManager); + return new KinesisSource(kinesisSourceConfig, pluginMetrics, pluginFactory, pipelineDescription, awsCredentialsSupplier, acknowledgementSetManager, kinesisLeaseConfigSupplier); } @Test diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java similarity index 99% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfigTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java index bfb6debc6f..09c64966f7 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/AwsAuthenticationConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.configuration; +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; import com.fasterxml.jackson.databind.ObjectMapper; import org.hamcrest.CoreMatchers; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java similarity index 99% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfigTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java index 1f2b723919..863e7fb4b2 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisSourceConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.configuration; +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java similarity index 90% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfigTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java index 342e9d70f6..21e83e4e39 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/configuration/KinesisStreamPollingConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.configuration; +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessorTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java similarity index 97% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessorTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java index 8ccfbd235a..bb851837ac 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisRecordProcessorTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.processor; +package org.opensearch.dataprepper.plugins.kinesis.source.processor; import io.micrometer.core.instrument.Counter; import org.junit.jupiter.api.BeforeEach; @@ -17,8 +17,8 @@ import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisSourceConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.InvalidStateException; import software.amazon.kinesis.exceptions.ShutdownException; @@ -48,9 +48,9 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; -import static org.opensearch.dataprepper.plugins.source.kinesis.processor.KinesisRecordProcessor.KINESIS_CHECKPOINT_FAILURES; -import static org.opensearch.dataprepper.plugins.source.kinesis.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSING_ERRORS; -import static org.opensearch.dataprepper.plugins.source.kinesis.processor.KinesisRecordProcessor.KINESIS_STREAM_TAG_KEY; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_CHECKPOINT_FAILURES; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSING_ERRORS; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_STREAM_TAG_KEY; public class KinesisRecordProcessorTest { private KinesisRecordProcessor kinesisRecordProcessor; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java similarity index 94% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactoryTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java index 1bfe08ad6a..a0cdf07ea8 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/source/kinesis/processor/KinesisShardRecordProcessorFactoryTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java @@ -1,4 +1,4 @@ -package org.opensearch.dataprepper.plugins.source.kinesis.processor; +package org.opensearch.dataprepper.plugins.kinesis.source.processor; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -12,8 +12,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.kinesis.configuration.KinesisSourceConfig; -import org.opensearch.dataprepper.plugins.source.kinesis.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; import software.amazon.kinesis.common.StreamIdentifier; import java.util.Collections; diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml new file mode 100644 index 0000000000..a1c9be1e9d --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml @@ -0,0 +1,3 @@ +extensions: + kinesis_lease_config: + lease_coordination_table: "kinesis-pipeline-kcl" \ No newline at end of file From f8f89fb3a738e9a7af3e79c45f6c3ff91759e5e3 Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Fri, 16 Aug 2024 17:06:33 -0700 Subject: [PATCH 3/8] Cleanup unused code for metrics Signed-off-by: Souvik Bose --- .../kinesis/extension/KinesisLeaseConfig.java | 4 +- .../KinesisLeaseConfigExtension.java | 2 +- .../KinesisLeaseCoordinationTableConfig.java | 22 +++++++++++ .../plugins/kinesis/source/ClientFactory.java | 14 ++++--- .../kinesis/source/KinesisService.java | 21 ++++++---- .../metrics/MicrometerFilterMetricsScope.java | 38 ------------------- .../metrics/MicrometerMetricFactory.java | 18 --------- ...DefaultKinesisLeaseConfigSupplierTest.java | 10 ++++- .../extension/KinesisLeaseConfigTest.java | 5 ++- .../kinesis/source/ClientFactoryTest.java | 4 +- .../kinesis/source/KinesisServiceTest.java | 14 +++++-- .../kinesis/source/KinesisSourceTest.java | 8 +++- .../simple_pipeline_with_extensions.yaml | 6 ++- 13 files changed, 82 insertions(+), 84 deletions(-) create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseCoordinationTableConfig.java delete mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerFilterMetricsScope.java delete mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerMetricFactory.java diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java index fbafce9d7c..7690c05b52 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java @@ -5,6 +5,6 @@ @Getter public class KinesisLeaseConfig { - @JsonProperty("lease_coordination_table") - private String leaseCoordinationTable; + @JsonProperty("lease_coordination") + private KinesisLeaseCoordinationTableConfig leaseCoordinationTable; } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java index 5e6ad46efe..c5b4b03299 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java @@ -5,7 +5,7 @@ import org.opensearch.dataprepper.model.plugin.ExtensionPlugin; import org.opensearch.dataprepper.model.plugin.ExtensionPoints; -@DataPrepperExtensionPlugin(modelType = KinesisLeaseConfig.class, rootKeyJsonPath = "/kinesis_lease_config", allowInPipelineConfigurations = true) +@DataPrepperExtensionPlugin(modelType = KinesisLeaseConfig.class, rootKeyJsonPath = "/kinesis", allowInPipelineConfigurations = true) public class KinesisLeaseConfigExtension implements ExtensionPlugin { private KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseCoordinationTableConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseCoordinationTableConfig.java new file mode 100644 index 0000000000..5369b325aa --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseCoordinationTableConfig.java @@ -0,0 +1,22 @@ +package org.opensearch.dataprepper.plugins.kinesis.extension; + +import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.Getter; +import lombok.NonNull; +import software.amazon.awssdk.regions.Region; + +@Getter +public class KinesisLeaseCoordinationTableConfig { + + @JsonProperty("table_name") + @NonNull + private String tableName; + + @JsonProperty("region") + @NonNull + private String region; + + public Region getAwsRegion() { + return Region.of(region); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java index dd12324f1e..64378854a4 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java @@ -4,6 +4,8 @@ import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; @@ -24,10 +26,10 @@ public ClientFactory(final AwsCredentialsSupplier awsCredentialsSupplier, this.awsAuthenticationConfig = awsAuthenticationConfig; } - public DynamoDbAsyncClient buildDynamoDBClient() { + public DynamoDbAsyncClient buildDynamoDBClient(Region region) { return DynamoDbAsyncClient.builder() - .credentialsProvider(awsAuthenticationConfig.authenticateAwsConfiguration()) - .region(awsAuthenticationConfig.getAwsRegion()) + .credentialsProvider(DefaultCredentialsProvider.create()) + .region(region) .build(); } @@ -39,10 +41,10 @@ public KinesisAsyncClient buildKinesisAsyncClient() { ); } - public CloudWatchAsyncClient buildCloudWatchAsyncClient() { + public CloudWatchAsyncClient buildCloudWatchAsyncClient(Region region) { return CloudWatchAsyncClient.builder() - .credentialsProvider(awsAuthenticationConfig.authenticateAwsConfiguration()) - .region(awsAuthenticationConfig.getAwsRegion()) + .credentialsProvider(DefaultCredentialsProvider.create()) + .region(region) .build(); } } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java index dd0ad61249..fa36259a3d 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java @@ -8,6 +8,7 @@ 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.kinesis.extension.KinesisLeaseConfig; import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfigSupplier; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.ConsumerStrategy; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; @@ -39,6 +40,7 @@ public class KinesisService { private final String applicationName; private final String tableName; + private final String kclMetricsNamespaceName; private final String pipelineName; private final AcknowledgementSetManager acknowledgementSetManager; private final KinesisSourceConfig sourceConfig; @@ -63,15 +65,18 @@ public KinesisService(final KinesisSourceConfig sourceConfig, this.pluginMetrics = pluginMetrics; this.pluginFactory = pluginFactory; this.acknowledgementSetManager = acknowledgementSetManager; - this.dynamoDbClient = clientFactory.buildDynamoDBClient(); - this.kinesisClient = clientFactory.buildKinesisAsyncClient(); - this.cloudWatchClient = clientFactory.buildCloudWatchAsyncClient(); - this.pipelineName = pipelineDescription.getPipelineName(); - this.applicationName = pipelineName; if (kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().isEmpty()) { throw new IllegalStateException("Lease Coordination table should be provided!"); } - this.tableName = kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable(); + KinesisLeaseConfig kinesisLeaseConfig = + kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get(); + this.tableName = kinesisLeaseConfig.getLeaseCoordinationTable().getTableName(); + this.kclMetricsNamespaceName = this.tableName; + this.dynamoDbClient = clientFactory.buildDynamoDBClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); + this.kinesisClient = clientFactory.buildKinesisAsyncClient(); + this.cloudWatchClient = clientFactory.buildCloudWatchAsyncClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); + this.pipelineName = pipelineDescription.getPipelineName(); + this.applicationName = pipelineName; this.executorService = Executors.newFixedThreadPool(1); } @@ -118,7 +123,9 @@ public Scheduler createScheduler(final Buffer> buffer) { new KinesisMultiStreamTracker(kinesisClient, sourceConfig, applicationName), applicationName, kinesisClient, dynamoDbClient, cloudWatchClient, new WorkerIdentifierGenerator().generate(), processorFactory - ).tableName(tableName); + ) + .tableName(tableName) + .namespace(kclMetricsNamespaceName); ConsumerStrategy consumerStrategy = sourceConfig.getConsumerStrategy(); if (consumerStrategy == ConsumerStrategy.POLLING) { diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerFilterMetricsScope.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerFilterMetricsScope.java deleted file mode 100644 index bfab36170f..0000000000 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerFilterMetricsScope.java +++ /dev/null @@ -1,38 +0,0 @@ -//package org.opensearch.dataprepper.plugins.kinesis.source.metrics; -// -//import com.google.common.collect.ImmutableSet; -//import org.opensearch.dataprepper.metrics.PluginMetrics; -//import org.slf4j.Logger; -//import org.slf4j.LoggerFactory; -//import software.amazon.awssdk.services.cloudwatch.model.StandardUnit; -//import software.amazon.kinesis.metrics.FilteringMetricsScope; -//import software.amazon.kinesis.metrics.MetricsLevel; -// -//public class MicrometerFilterMetricsScope extends FilteringMetricsScope { -// -// private static final Logger LOG = LoggerFactory.getLogger(MicrometerFilterMetricsScope.class); -// -// private final PluginMetrics pluginMetrics; -// -// public MicrometerFilterMetricsScope(final PluginMetrics pluginMetrics) { -// super(MetricsLevel.SUMMARY, ImmutableSet.of(METRICS_DIMENSIONS_ALL)); -// this.pluginMetrics = pluginMetrics; -// } -// -// @Override -// public void addData(String name, double value, StandardUnit unit) { -// // TODO: report metrics. -// LOG.info("{} {} {}", name, value, unit); -// } -// -// @Override -// public void addData(String name, double value, StandardUnit unit, MetricsLevel level) { -// // TODO: report metrics. -// LOG.info("{} {} {} {}", name, value, unit, level); -// } -// -// @Override -// public void end() { -// // TODO: Update this -// } -//} \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerMetricFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerMetricFactory.java deleted file mode 100644 index 114e1ca4fa..0000000000 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/metrics/MicrometerMetricFactory.java +++ /dev/null @@ -1,18 +0,0 @@ -//package org.opensearch.dataprepper.plugins.kinesis.source.metrics; -// -//import org.opensearch.dataprepper.metrics.PluginMetrics; -//import software.amazon.kinesis.metrics.MetricsFactory; -//import software.amazon.kinesis.metrics.MetricsScope; -// -//public class MicrometerMetricFactory implements MetricsFactory { -// -// final PluginMetrics pluginMetrics; -// -// public MicrometerMetricFactory(PluginMetrics pluginMetrics) { -// this.pluginMetrics = pluginMetrics; -// } -// -// public MetricsScope createMetrics() { -// return new MicrometerFilterMetricsScope(pluginMetrics); -// } -//} \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java index 9914c30fbf..dc9bbf3fbe 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java @@ -17,15 +17,21 @@ public class DefaultKinesisLeaseConfigSupplierTest { @Mock KinesisLeaseConfig kinesisLeaseConfig; + @Mock + KinesisLeaseCoordinationTableConfig kinesisLeaseCoordinationTableConfig; + private DefaultKinesisLeaseConfigSupplier createObjectUnderTest() { return new DefaultKinesisLeaseConfigSupplier(kinesisLeaseConfig); } @Test void testGetters() { - when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(LEASE_COORDINATION_TABLE); + when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(kinesisLeaseCoordinationTableConfig); + when(kinesisLeaseCoordinationTableConfig.getTableName()).thenReturn(LEASE_COORDINATION_TABLE); + when(kinesisLeaseCoordinationTableConfig.getRegion()).thenReturn("us-east-1"); KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier = createObjectUnderTest(); - assertThat(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable(), equalTo(LEASE_COORDINATION_TABLE)); + assertThat(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable().getTableName(), equalTo(LEASE_COORDINATION_TABLE)); + assertThat(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable().getRegion(), equalTo("us-east-1")); } @Test diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java index 9731a7fda9..1f7b8fbfc4 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java @@ -33,7 +33,7 @@ private KinesisLeaseConfig makeConfig(String filePath) throws IOException { assertThat(dataPrepperConfiguration, notNullValue()); assertThat(dataPrepperConfiguration.getPipelineExtensions(), notNullValue()); final Map kinesisLeaseConfigMap = - (Map) dataPrepperConfiguration.getPipelineExtensions().getExtensionMap().get("kinesis_lease_config"); + (Map) dataPrepperConfiguration.getPipelineExtensions().getExtensionMap().get("kinesis"); String json = OBJECT_MAPPER.writeValueAsString(kinesisLeaseConfigMap); Reader reader = new StringReader(json); return OBJECT_MAPPER.readValue(reader, KinesisLeaseConfig.class); @@ -46,7 +46,8 @@ void testConfigWithTestExtension() throws IOException { "src/test/resources/simple_pipeline_with_extensions.yaml"); assertNotNull(kinesisLeaseConfig.getLeaseCoordinationTable()); - assertEquals(kinesisLeaseConfig.getLeaseCoordinationTable(), "kinesis-pipeline-kcl"); + assertEquals(kinesisLeaseConfig.getLeaseCoordinationTable().getTableName(), "kinesis-pipeline-kcl"); + assertEquals(kinesisLeaseConfig.getLeaseCoordinationTable().getRegion(), "us-east-1"); } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java index f2e4ed7797..78faf5382d 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java @@ -35,13 +35,13 @@ void testCreateClient() throws NoSuchFieldException, IllegalAccessException { ClientFactory clientFactory = new ClientFactory(awsCredentialsSupplier, awsAuthenticationOptionsConfig); - final DynamoDbAsyncClient dynamoDbAsyncClient = clientFactory.buildDynamoDBClient(); + final DynamoDbAsyncClient dynamoDbAsyncClient = clientFactory.buildDynamoDBClient(Region.US_EAST_1); assertNotNull(dynamoDbAsyncClient); final KinesisAsyncClient kinesisAsyncClient = clientFactory.buildKinesisAsyncClient(); assertNotNull(kinesisAsyncClient); - final CloudWatchAsyncClient cloudWatchAsyncClient = clientFactory.buildCloudWatchAsyncClient(); + final CloudWatchAsyncClient cloudWatchAsyncClient = clientFactory.buildCloudWatchAsyncClient(Region.US_EAST_1); assertNotNull(cloudWatchAsyncClient); } } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java index d523a1c768..dfc4455d26 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java @@ -12,6 +12,7 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfig; import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfigSupplier; +import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseCoordinationTableConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.ConsumerStrategy; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; @@ -107,6 +108,9 @@ public class KinesisServiceTest { @Mock KinesisLeaseConfig kinesisLeaseConfig; + @Mock + KinesisLeaseCoordinationTableConfig kinesisLeaseCoordinationTableConfig; + @BeforeEach void setup() { awsAuthenticationConfig = mock(AwsAuthenticationConfig.class); @@ -122,7 +126,11 @@ void setup() { buffer = mock(Buffer.class); kinesisLeaseConfigSupplier = mock(KinesisLeaseConfigSupplier.class); kinesisLeaseConfig = mock(KinesisLeaseConfig.class); - when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn("kinesis-lease-table"); + kinesisLeaseCoordinationTableConfig = mock(KinesisLeaseCoordinationTableConfig.class); + when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(kinesisLeaseCoordinationTableConfig); + when(kinesisLeaseCoordinationTableConfig.getTableName()).thenReturn("kinesis-lease-table"); + when(kinesisLeaseCoordinationTableConfig.getRegion()).thenReturn("us-east-1"); + when(kinesisLeaseCoordinationTableConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); when(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig()).thenReturn(Optional.ofNullable(kinesisLeaseConfig)); when(awsAuthenticationConfig.getAwsRegion()).thenReturn(Region.of("us-west-2")); @@ -160,9 +168,9 @@ void setup() { when(kinesisSourceConfig.getStreams()).thenReturn(streamConfigs); when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); - when(clientFactory.buildDynamoDBClient()).thenReturn(dynamoDbClient); + when(clientFactory.buildDynamoDBClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(dynamoDbClient); when(clientFactory.buildKinesisAsyncClient()).thenReturn(kinesisClient); - when(clientFactory.buildCloudWatchAsyncClient()).thenReturn(cloudWatchClient); + when(clientFactory.buildCloudWatchAsyncClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(cloudWatchClient); when(kinesisClient.serviceClientConfiguration()).thenReturn(KinesisServiceClientConfiguration.builder().region(Region.US_EAST_1).build()); when(scheduler.startGracefulShutdown()).thenReturn(CompletableFuture.completedFuture(true)); when(pipelineDescription.getPipelineName()).thenReturn(PIPELINE_NAME); diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java index 69249b82e4..14ac812138 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java @@ -13,6 +13,7 @@ import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfig; import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfigSupplier; +import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseCoordinationTableConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; @@ -67,6 +68,9 @@ public class KinesisSourceTest { @Mock KinesisLeaseConfig kinesisLeaseConfig; + @Mock + KinesisLeaseCoordinationTableConfig kinesisLeaseCoordinationTableConfig; + @BeforeEach void setup() { pluginMetrics = mock(PluginMetrics.class); @@ -79,7 +83,9 @@ void setup() { kinesisService = mock(KinesisService.class); kinesisLeaseConfigSupplier = mock(KinesisLeaseConfigSupplier.class); kinesisLeaseConfig = mock(KinesisLeaseConfig.class); - when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn("kinesis-lease-table"); + kinesisLeaseCoordinationTableConfig = mock(KinesisLeaseCoordinationTableConfig.class); + when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(kinesisLeaseCoordinationTableConfig); + when(kinesisLeaseCoordinationTableConfig.getTableName()).thenReturn("us-east-1"); when(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig()).thenReturn(Optional.ofNullable(kinesisLeaseConfig)); when(awsAuthenticationConfig.getAwsRegion()).thenReturn(Region.of("us-west-2")); when(awsAuthenticationConfig.getAwsStsRoleArn()).thenReturn(UUID.randomUUID().toString()); diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml index a1c9be1e9d..4f964cae7f 100644 --- a/data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml +++ b/data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml @@ -1,3 +1,5 @@ extensions: - kinesis_lease_config: - lease_coordination_table: "kinesis-pipeline-kcl" \ No newline at end of file + kinesis: + lease_coordination: + table_name: "kinesis-pipeline-kcl" + region: "us-east-1" \ No newline at end of file From 0567858c6e04c8c80ec8f2ee1121f65511f8759a Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Tue, 27 Aug 2024 20:25:57 -0700 Subject: [PATCH 4/8] Update tests and changes per review comments Signed-off-by: Souvik Bose --- ...Factory.java => KinesisClientFactory.java} | 6 +- .../kinesis/source/KinesisService.java | 8 +-- .../plugins/kinesis/source/KinesisSource.java | 4 +- .../AwsAuthenticationConfig.java | 18 ++--- .../processor/KinesisRecordProcessor.java | 21 ++---- .../extension/KinesisLeaseConfigTest.java | 3 +- ...est.java => KinesisClientFactoryTest.java} | 4 +- .../kinesis/source/KinesisServiceTest.java | 23 +++--- .../processor/KinesisRecordProcessorTest.java | 70 +++++++++++++++++++ 9 files changed, 110 insertions(+), 47 deletions(-) rename data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/{ClientFactory.java => KinesisClientFactory.java} (91%) rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/{ClientFactoryTest.java => KinesisClientFactoryTest.java} (92%) diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java similarity index 91% rename from data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java rename to data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java index 64378854a4..e85b66aa37 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactory.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java @@ -11,12 +11,12 @@ import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; import software.amazon.kinesis.common.KinesisClientUtil; -public class ClientFactory { +public class KinesisClientFactory { private final AwsCredentialsProvider awsCredentialsProvider; private final AwsAuthenticationConfig awsAuthenticationConfig; - public ClientFactory(final AwsCredentialsSupplier awsCredentialsSupplier, - final AwsAuthenticationConfig awsAuthenticationConfig) { + public KinesisClientFactory(final AwsCredentialsSupplier awsCredentialsSupplier, + final AwsAuthenticationConfig awsAuthenticationConfig) { awsCredentialsProvider = awsCredentialsSupplier.getProvider(AwsCredentialsOptions.builder() .withRegion(awsAuthenticationConfig.getAwsRegion()) .withStsRoleArn(awsAuthenticationConfig.getAwsStsRoleArn()) diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java index fa36259a3d..cf2f96d42f 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java @@ -54,7 +54,7 @@ public class KinesisService { private final ExecutorService executorService; public KinesisService(final KinesisSourceConfig sourceConfig, - final ClientFactory clientFactory, + final KinesisClientFactory kinesisClientFactory, final PluginMetrics pluginMetrics, final PluginFactory pluginFactory, final PipelineDescription pipelineDescription, @@ -72,9 +72,9 @@ public KinesisService(final KinesisSourceConfig sourceConfig, kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get(); this.tableName = kinesisLeaseConfig.getLeaseCoordinationTable().getTableName(); this.kclMetricsNamespaceName = this.tableName; - this.dynamoDbClient = clientFactory.buildDynamoDBClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); - this.kinesisClient = clientFactory.buildKinesisAsyncClient(); - this.cloudWatchClient = clientFactory.buildCloudWatchAsyncClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); + this.dynamoDbClient = kinesisClientFactory.buildDynamoDBClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); + this.kinesisClient = kinesisClientFactory.buildKinesisAsyncClient(); + this.cloudWatchClient = kinesisClientFactory.buildCloudWatchAsyncClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); this.pipelineName = pipelineDescription.getPipelineName(); this.applicationName = pipelineName; this.executorService = Executors.newFixedThreadPool(1); diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java index a88cff0de0..16b25d072b 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java @@ -36,8 +36,8 @@ public KinesisSource(final KinesisSourceConfig kinesisSourceConfig, final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier) { this.kinesisSourceConfig = kinesisSourceConfig; this.kinesisLeaseConfigSupplier = kinesisLeaseConfigSupplier; - ClientFactory clientFactory = new ClientFactory(awsCredentialsSupplier, kinesisSourceConfig.getAwsAuthenticationConfig()); - this.kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, + KinesisClientFactory kinesisClientFactory = new KinesisClientFactory(awsCredentialsSupplier, kinesisSourceConfig.getAwsAuthenticationConfig()); + this.kinesisService = new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); } @Override diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java index 27774828ef..e70ec337e7 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java @@ -7,6 +7,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.Size; +import lombok.Getter; import software.amazon.awssdk.arns.Arn; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; @@ -26,14 +27,17 @@ public class AwsAuthenticationConfig { @Size(min = 1, message = "Region cannot be empty string") private String awsRegion; + @Getter @JsonProperty("sts_role_arn") @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") private String awsStsRoleArn; + @Getter @JsonProperty("sts_external_id") @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") private String awsStsExternalId; + @Getter @JsonProperty("sts_header_overrides") @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") private Map awsStsHeaderOverrides; @@ -42,18 +46,6 @@ public Region getAwsRegion() { return awsRegion != null ? Region.of(awsRegion) : null; } - public String getAwsStsRoleArn() { - return awsStsRoleArn; - } - - public String getAwsStsExternalId() { - return awsStsExternalId; - } - - public Map getAwsStsHeaderOverrides() { - return awsStsHeaderOverrides; - } - public AwsCredentialsProvider authenticateAwsConfiguration() { final AwsCredentialsProvider awsCredentialsProvider; @@ -67,7 +59,7 @@ public AwsCredentialsProvider authenticateAwsConfiguration() { final StsClient stsClient = StsClient.builder().region(getAwsRegion()).build(); AssumeRoleRequest.Builder assumeRoleRequestBuilder = AssumeRoleRequest.builder() - .roleSessionName("GeoIP-Processor-" + UUID.randomUUID()).roleArn(awsStsRoleArn); + .roleSessionName("Kinesis-source-" + UUID.randomUUID()).roleArn(awsStsRoleArn); if (awsStsHeaderOverrides != null && !awsStsHeaderOverrides.isEmpty()) { assumeRoleRequestBuilder = assumeRoleRequestBuilder.overrideConfiguration( diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java index bfa7c2c023..ada8094908 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java @@ -34,6 +34,7 @@ import java.time.Duration; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.function.Consumer; public class KinesisRecordProcessor implements ShardRecordProcessor { @@ -52,7 +53,7 @@ public class KinesisRecordProcessor implements ShardRecordProcessor { private final Counter recordProcessingErrors; private final Counter checkpointFailures; private static final Duration ACKNOWLEDGEMENT_SET_TIMEOUT = Duration.ofSeconds(20); - private static final String ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME = "acknowledgementSetCallbackCounter"; + public static final String ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME = "acknowledgementSetCallbackCounter"; public static final String KINESIS_RECORD_PROCESSING_ERRORS = "recordProcessingErrors"; public static final String KINESIS_CHECKPOINT_FAILURES = "checkpointFailures"; public static final String KINESIS_STREAM_TAG_KEY = "stream"; @@ -71,7 +72,7 @@ public KinesisRecordProcessor(Buffer> buffer, final PluginSetting codecPluginSettings = new PluginSetting(codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); this.codec = pluginFactory.loadPlugin(InputCodec.class, codecPluginSettings); this.acknowledgementSetManager = acknowledgementSetManager; - this.acknowledgementSetCallbackCounter = pluginMetrics.counter(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME); + this.acknowledgementSetCallbackCounter = pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.recordProcessingErrors = pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.checkpointFailures = pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.checkpointIntervalMilliSeconds = kinesisStreamConfig.getCheckPointIntervalInMilliseconds(); @@ -108,29 +109,21 @@ public void processRecords(ProcessRecordsInput processRecordsInput) { List> records = new ArrayList<>(); try { - AcknowledgementSet acknowledgementSet; + Optional acknowledgementSetOpt = Optional.empty(); boolean acknowledgementsEnabled = kinesisSourceConfig.isAcknowledgments(); if (acknowledgementsEnabled) { - acknowledgementSet = createAcknowledgmentSet(processRecordsInput); - } else { - acknowledgementSet = null; + acknowledgementSetOpt = Optional.of(createAcknowledgmentSet(processRecordsInput)); } for (KinesisClientRecord record : processRecordsInput.records()) { processRecord(record, records::add); } - if (acknowledgementSet != null) { - records.forEach(record -> { - acknowledgementSet.add(record.getData()); - }); - } + acknowledgementSetOpt.ifPresent(acknowledgementSet -> records.forEach(record -> acknowledgementSet.add(record.getData()))); buffer.writeAll(records, bufferTimeoutMillis); - if (acknowledgementSet != null) { - acknowledgementSet.complete(); - } + acknowledgementSetOpt.ifPresent(AcknowledgementSet::complete); // Checkpoint for shard if (kinesisStreamConfig.isEnableCheckPoint() && System.currentTimeMillis() - lastCheckpointTimeInMillis > checkpointIntervalMilliSeconds) { diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java index 1f7b8fbfc4..582bf68f69 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java @@ -8,6 +8,7 @@ import org.opensearch.dataprepper.pipeline.parser.ByteCountDeserializer; import org.opensearch.dataprepper.pipeline.parser.DataPrepperDurationDeserializer; import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import software.amazon.awssdk.regions.Region; import java.io.File; import java.io.IOException; @@ -48,7 +49,7 @@ void testConfigWithTestExtension() throws IOException { assertNotNull(kinesisLeaseConfig.getLeaseCoordinationTable()); assertEquals(kinesisLeaseConfig.getLeaseCoordinationTable().getTableName(), "kinesis-pipeline-kcl"); assertEquals(kinesisLeaseConfig.getLeaseCoordinationTable().getRegion(), "us-east-1"); - + assertEquals(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion(), Region.US_EAST_1); } } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java similarity index 92% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java index 78faf5382d..5c6e298760 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/ClientFactoryTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java @@ -16,7 +16,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.mockito.Mockito.mock; -public class ClientFactoryTest { +public class KinesisClientFactoryTest { private Region region = Region.US_EAST_1; private String roleArn; private Map stsHeader; @@ -33,7 +33,7 @@ void testCreateClient() throws NoSuchFieldException, IllegalAccessException { ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", roleArn); - ClientFactory clientFactory = new ClientFactory(awsCredentialsSupplier, awsAuthenticationOptionsConfig); + KinesisClientFactory clientFactory = new KinesisClientFactory(awsCredentialsSupplier, awsAuthenticationOptionsConfig); final DynamoDbAsyncClient dynamoDbAsyncClient = clientFactory.buildDynamoDBClient(Region.US_EAST_1); assertNotNull(dynamoDbAsyncClient); diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java index dfc4455d26..d5d8a93c4f 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java @@ -85,7 +85,7 @@ public class KinesisServiceTest { private PipelineDescription pipelineDescription; @Mock - private ClientFactory clientFactory; + private KinesisClientFactory kinesisClientFactory; @Mock private KinesisAsyncClient kinesisClient; @@ -120,7 +120,7 @@ void setup() { kinesisClient = mock(KinesisAsyncClient.class); dynamoDbClient = mock(DynamoDbAsyncClient.class); cloudWatchClient = mock(CloudWatchAsyncClient.class); - clientFactory = mock(ClientFactory.class); + kinesisClientFactory = mock(KinesisClientFactory.class); scheduler = mock(Scheduler.class); pipelineDescription = mock(PipelineDescription.class); buffer = mock(Buffer.class); @@ -168,16 +168,16 @@ void setup() { when(kinesisSourceConfig.getStreams()).thenReturn(streamConfigs); when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); - when(clientFactory.buildDynamoDBClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(dynamoDbClient); - when(clientFactory.buildKinesisAsyncClient()).thenReturn(kinesisClient); - when(clientFactory.buildCloudWatchAsyncClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(cloudWatchClient); + when(kinesisClientFactory.buildDynamoDBClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(dynamoDbClient); + when(kinesisClientFactory.buildKinesisAsyncClient()).thenReturn(kinesisClient); + when(kinesisClientFactory.buildCloudWatchAsyncClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(cloudWatchClient); when(kinesisClient.serviceClientConfiguration()).thenReturn(KinesisServiceClientConfiguration.builder().region(Region.US_EAST_1).build()); when(scheduler.startGracefulShutdown()).thenReturn(CompletableFuture.completedFuture(true)); when(pipelineDescription.getPipelineName()).thenReturn(PIPELINE_NAME); } public KinesisService createObjectUnderTest() { - return new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, + return new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); } @@ -188,9 +188,16 @@ void testServiceStart() { assertNotNull(kinesisService.getScheduler(buffer)); } + @Test + void testServiceThrowsWhenLeaseConfigIsInvalid() { + when(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig()).thenReturn(Optional.empty()); + assertThrows(IllegalStateException.class, () -> new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier)); + } + @Test void testCreateScheduler() { - KinesisService kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, + KinesisService kinesisService = new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); Scheduler schedulerObjectUnderTest = kinesisService.createScheduler(buffer); @@ -208,7 +215,7 @@ void testCreateScheduler() { @Test void testCreateSchedulerWithPollingStrategy() { when(kinesisSourceConfig.getConsumerStrategy()).thenReturn(ConsumerStrategy.POLLING); - KinesisService kinesisService = new KinesisService(kinesisSourceConfig, clientFactory, pluginMetrics, pluginFactory, + KinesisService kinesisService = new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); Scheduler schedulerObjectUnderTest = kinesisService.createScheduler(buffer); diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java index bb851837ac..e226596849 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java @@ -36,18 +36,22 @@ import java.util.Collections; import java.util.List; import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.anyCollection; import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_CHECKPOINT_FAILURES; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSING_ERRORS; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_STREAM_TAG_KEY; @@ -100,6 +104,9 @@ public class KinesisRecordProcessorTest { @Mock private Counter checkpointFailures; + @Mock + private Counter acknowledgementSetCallbackCounter; + @BeforeEach public void setup() { MockitoAnnotations.initMocks(this); @@ -121,6 +128,7 @@ public void setup() { when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); when(kinesisSourceConfig.getStreams()).thenReturn(List.of(kinesisStreamConfig)); when(processRecordsInput.checkpointer()).thenReturn(checkpointer); + when(pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(acknowledgementSetCallbackCounter); } @Test @@ -143,6 +151,68 @@ void testProcessRecordsWithoutAcknowledgementsCheckpointsEnabled() verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); } + @Test + void testProcessRecordsWithAcknowledgementsCheckpointsEnabled() + throws Exception { + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(true); + when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); + AtomicReference numEventsAdded = new AtomicReference<>(0); + doAnswer(a -> { + numEventsAdded.getAndSet(numEventsAdded.get() + 1); + return null; + }).when(acknowledgementSet).add(any()); + + doAnswer(invocation -> { + Consumer consumer = invocation.getArgument(0); + consumer.accept(true); + return acknowledgementSet; + }).when(acknowledgementSetManager).create(any(Consumer.class), any(Duration.class)); + + kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor.initialize(initializationInput); + + Thread.sleep(2000); + + kinesisRecordProcessor.processRecords(processRecordsInput); + + verify(checkpointer).checkpoint(); + verify(buffer).writeAll(anyCollection(), anyInt()); + verify(acknowledgementSetManager, times(1)).create(any(), any(Duration.class)); + } + + @Test + void testProcessRecordsWithAcknowledgementsEnabledAndAcksReturnFalse() + throws Exception { + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(true); + when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); + AtomicReference numEventsAdded = new AtomicReference<>(0); + doAnswer(a -> { + numEventsAdded.getAndSet(numEventsAdded.get() + 1); + return null; + }).when(acknowledgementSet).add(any()); + + doAnswer(invocation -> { + Consumer consumer = invocation.getArgument(0); + consumer.accept(false); + return acknowledgementSet; + }).when(acknowledgementSetManager).create(any(Consumer.class), any(Duration.class)); + + kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor.initialize(initializationInput); + + Thread.sleep(2000); + + kinesisRecordProcessor.processRecords(processRecordsInput); + + verify(checkpointer, times(0)).checkpoint(); + verify(buffer).writeAll(anyCollection(), anyInt()); + verify(acknowledgementSetManager, times(1)).create(any(), any(Duration.class)); + } + @Test void testProcessRecordsWithNDJsonInputCodec() throws Exception { From 63a2f91cd84af55d16c432698d2514e1371fdf8b Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Wed, 28 Aug 2024 21:45:23 -0700 Subject: [PATCH 5/8] Address review comments Signed-off-by: Souvik Bose --- .../kinesis-source/build.gradle | 14 +- .../DefaultKinesisLeaseConfigSupplier.java | 17 -- .../KinesisLeaseConfigExtension.java | 2 +- .../extension/KinesisLeaseConfigSupplier.java | 12 +- .../HostNameWorkerIdentifierGenerator.java | 29 +++ .../kinesis/source/KinesisClientFactory.java | 13 +- .../kinesis/source/KinesisService.java | 11 +- .../plugins/kinesis/source/KinesisSource.java | 2 +- .../source/WorkerIdentifierGenerator.java | 26 +-- .../AwsAuthenticationConfig.java | 38 ---- .../configuration/ConsumerStrategy.java | 8 +- .../InitialPositionInStreamConfig.java | 37 ++++ .../configuration/KinesisSourceConfig.java | 3 +- .../configuration/KinesisStreamConfig.java | 17 +- .../KinesisStreamPollingConfig.java | 10 +- .../processor/KinesisRecordProcessor.java | 10 +- ...va => KinesisLeaseConfigSupplierTest.java} | 10 +- .../source/KinesisClientFactoryTest.java | 9 +- .../kinesis/source/KinesisServiceTest.java | 27 ++- .../kinesis/source/KinesisSourceTest.java | 10 +- .../AwsAuthenticationConfigTest.java | 183 ------------------ .../InitialPositionInStreamConfigTest.java | 28 +++ .../KinesisSourceConfigTest.java | 92 ++++++--- .../KinesisStreamPollingConfigTest.java | 4 +- .../processor/KinesisRecordProcessorTest.java | 4 +- .../resources/pipeline_with_acks_enabled.yaml | 12 ++ .../pipeline_with_checkpoint_enabled.yaml | 20 ++ .../pipeline_with_polling_config_enabled.yaml | 15 ++ .../src/test/resources/simple-pipeline-2.yaml | 21 -- .../src/test/resources/simple-pipeline.yaml | 16 -- 30 files changed, 306 insertions(+), 394 deletions(-) delete mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplier.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/HostNameWorkerIdentifierGenerator.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfig.java rename data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/{DefaultKinesisLeaseConfigSupplierTest.java => KinesisLeaseConfigSupplierTest.java} (84%) create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfigTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_acks_enabled.yaml create mode 100644 data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_checkpoint_enabled.yaml create mode 100644 data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_polling_config_enabled.yaml delete mode 100644 data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline-2.yaml delete mode 100644 data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline.yaml diff --git a/data-prepper-plugins/kinesis-source/build.gradle b/data-prepper-plugins/kinesis-source/build.gradle index f5bb41385a..ae380ba0e1 100644 --- a/data-prepper-plugins/kinesis-source/build.gradle +++ b/data-prepper-plugins/kinesis-source/build.gradle @@ -10,24 +10,16 @@ plugins { dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:common') - implementation project(':data-prepper-plugins:buffer-common') - implementation libs.armeria.core implementation 'com.fasterxml.jackson.core:jackson-core' - implementation 'com.fasterxml.jackson.core:jackson-databind' - implementation project(':data-prepper-plugins:blocking-buffer') - implementation 'software.amazon.awssdk:kinesis' - implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-ion' + implementation 'io.micrometer:micrometer-core' implementation 'software.amazon.kinesis:amazon-kinesis-client:2.6.0' compileOnly 'org.projectlombok:lombok:1.18.20' annotationProcessor 'org.projectlombok:lombok:1.18.20' - implementation("software.amazon.awssdk:dynamodb") - implementation("com.amazonaws:aws-java-sdk:1.12.394") implementation project(path: ':data-prepper-plugins:aws-plugin-api') - testImplementation 'org.yaml:snakeyaml:2.2' + testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' testImplementation project(':data-prepper-test-common') - testImplementation platform('org.junit:junit-bom:5.9.1') - testImplementation 'org.junit.jupiter:junit-jupiter' testImplementation project(':data-prepper-test-event') testImplementation project(':data-prepper-core') testImplementation project(':data-prepper-plugin-framework') diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplier.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplier.java deleted file mode 100644 index f7e4bd9e87..0000000000 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplier.java +++ /dev/null @@ -1,17 +0,0 @@ -package org.opensearch.dataprepper.plugins.kinesis.extension; - -import java.util.Optional; - -public class DefaultKinesisLeaseConfigSupplier implements KinesisLeaseConfigSupplier { - - private KinesisLeaseConfig kinesisLeaseConfig; - - public DefaultKinesisLeaseConfigSupplier(final KinesisLeaseConfig kinesisLeaseConfig) { - this.kinesisLeaseConfig = kinesisLeaseConfig; - } - - @Override - public Optional getKinesisExtensionLeaseConfig() { - return kinesisLeaseConfig != null ? Optional.of(kinesisLeaseConfig) : Optional.empty(); - } -} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java index c5b4b03299..e2cf1ad475 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java @@ -11,7 +11,7 @@ public class KinesisLeaseConfigExtension implements ExtensionPlugin { private KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; @DataPrepperPluginConstructor public KinesisLeaseConfigExtension(final KinesisLeaseConfig kinesisLeaseConfig) { - this.kinesisLeaseConfigSupplier = new DefaultKinesisLeaseConfigSupplier(kinesisLeaseConfig); + this.kinesisLeaseConfigSupplier = new KinesisLeaseConfigSupplier(kinesisLeaseConfig); } @Override diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java index 3db85d3ca5..6d61e3dd78 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java @@ -2,9 +2,15 @@ import java.util.Optional; -public interface KinesisLeaseConfigSupplier { +public class KinesisLeaseConfigSupplier { - default Optional getKinesisExtensionLeaseConfig() { - return Optional.empty(); + private KinesisLeaseConfig kinesisLeaseConfig; + + public KinesisLeaseConfigSupplier(final KinesisLeaseConfig kinesisLeaseConfig) { + this.kinesisLeaseConfig = kinesisLeaseConfig; + } + + public Optional getKinesisExtensionLeaseConfig() { + return Optional.ofNullable(kinesisLeaseConfig); } } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/HostNameWorkerIdentifierGenerator.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/HostNameWorkerIdentifierGenerator.java new file mode 100644 index 0000000000..962b9f5d82 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/HostNameWorkerIdentifierGenerator.java @@ -0,0 +1,29 @@ +package org.opensearch.dataprepper.plugins.kinesis.source; + +import java.net.InetAddress; +import java.net.UnknownHostException; + +/** + * Generate a unique ID to represent a consumer application instance. + */ +public class HostNameWorkerIdentifierGenerator implements WorkerIdentifierGenerator { + + private static final String hostName; + + static { + try { + hostName = InetAddress.getLocalHost().getHostName(); + } catch (final UnknownHostException e) { + throw new RuntimeException(e); + } + } + + + /** + * @return Default to use host name. + */ + @Override + public String generate() { + return hostName; + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java index e85b66aa37..ed699bae20 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java @@ -4,7 +4,6 @@ import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; @@ -13,6 +12,7 @@ public class KinesisClientFactory { private final AwsCredentialsProvider awsCredentialsProvider; + private final AwsCredentialsProvider defaultCredentialsProvider; private final AwsAuthenticationConfig awsAuthenticationConfig; public KinesisClientFactory(final AwsCredentialsSupplier awsCredentialsSupplier, @@ -23,27 +23,28 @@ public KinesisClientFactory(final AwsCredentialsSupplier awsCredentialsSupplier, .withStsExternalId(awsAuthenticationConfig.getAwsStsExternalId()) .withStsHeaderOverrides(awsAuthenticationConfig.getAwsStsHeaderOverrides()) .build()); + defaultCredentialsProvider = awsCredentialsSupplier.getProvider(AwsCredentialsOptions.defaultOptions()); this.awsAuthenticationConfig = awsAuthenticationConfig; } public DynamoDbAsyncClient buildDynamoDBClient(Region region) { return DynamoDbAsyncClient.builder() - .credentialsProvider(DefaultCredentialsProvider.create()) + .credentialsProvider(defaultCredentialsProvider) .region(region) .build(); } - public KinesisAsyncClient buildKinesisAsyncClient() { + public KinesisAsyncClient buildKinesisAsyncClient(Region region) { return KinesisClientUtil.createKinesisAsyncClient( KinesisAsyncClient.builder() - .credentialsProvider(awsAuthenticationConfig.authenticateAwsConfiguration()) - .region(awsAuthenticationConfig.getAwsRegion()) + .credentialsProvider(awsCredentialsProvider) + .region(region) ); } public CloudWatchAsyncClient buildCloudWatchAsyncClient(Region region) { return CloudWatchAsyncClient.builder() - .credentialsProvider(DefaultCredentialsProvider.create()) + .credentialsProvider(defaultCredentialsProvider) .region(region) .build(); } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java index cf2f96d42f..33cf786884 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java @@ -47,6 +47,7 @@ public class KinesisService { private final KinesisAsyncClient kinesisClient; private final DynamoDbAsyncClient dynamoDbClient; private final CloudWatchAsyncClient cloudWatchClient; + private final WorkerIdentifierGenerator workerIdentifierGenerator; @Setter private Scheduler scheduler; @@ -59,7 +60,8 @@ public KinesisService(final KinesisSourceConfig sourceConfig, final PluginFactory pluginFactory, final PipelineDescription pipelineDescription, final AcknowledgementSetManager acknowledgementSetManager, - final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier + final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier, + final WorkerIdentifierGenerator workerIdentifierGenerator ){ this.sourceConfig = sourceConfig; this.pluginMetrics = pluginMetrics; @@ -73,10 +75,11 @@ public KinesisService(final KinesisSourceConfig sourceConfig, this.tableName = kinesisLeaseConfig.getLeaseCoordinationTable().getTableName(); this.kclMetricsNamespaceName = this.tableName; this.dynamoDbClient = kinesisClientFactory.buildDynamoDBClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); - this.kinesisClient = kinesisClientFactory.buildKinesisAsyncClient(); + this.kinesisClient = kinesisClientFactory.buildKinesisAsyncClient(sourceConfig.getAwsAuthenticationConfig().getAwsRegion()); this.cloudWatchClient = kinesisClientFactory.buildCloudWatchAsyncClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); this.pipelineName = pipelineDescription.getPipelineName(); this.applicationName = pipelineName; + this.workerIdentifierGenerator = workerIdentifierGenerator; this.executorService = Executors.newFixedThreadPool(1); } @@ -122,7 +125,7 @@ public Scheduler createScheduler(final Buffer> buffer) { new ConfigsBuilder( new KinesisMultiStreamTracker(kinesisClient, sourceConfig, applicationName), applicationName, kinesisClient, dynamoDbClient, cloudWatchClient, - new WorkerIdentifierGenerator().generate(), processorFactory + workerIdentifierGenerator.generate(), processorFactory ) .tableName(tableName) .namespace(kclMetricsNamespaceName); @@ -133,7 +136,7 @@ public Scheduler createScheduler(final Buffer> buffer) { new PollingConfig(kinesisClient) .maxRecords(sourceConfig.getPollingConfig().getMaxPollingRecords()) .idleTimeBetweenReadsInMillis( - sourceConfig.getPollingConfig().getIdleTimeBetweenReadsInMillis())); + sourceConfig.getPollingConfig().getIdleTimeBetweenReads().toMillis())); } return new Scheduler( diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java index 16b25d072b..afca88ba3d 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java @@ -38,7 +38,7 @@ public KinesisSource(final KinesisSourceConfig kinesisSourceConfig, this.kinesisLeaseConfigSupplier = kinesisLeaseConfigSupplier; KinesisClientFactory kinesisClientFactory = new KinesisClientFactory(awsCredentialsSupplier, kinesisSourceConfig.getAwsAuthenticationConfig()); this.kinesisService = new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, - pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier, new HostNameWorkerIdentifierGenerator()); } @Override public void start(final Buffer> buffer) { diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java index 7686861a87..91baa9a831 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java @@ -1,28 +1,6 @@ package org.opensearch.dataprepper.plugins.kinesis.source; -import java.net.InetAddress; -import java.net.UnknownHostException; +public interface WorkerIdentifierGenerator { -/** - * Generate a unique ID to represent a consumer application instance. - */ -public class WorkerIdentifierGenerator { - - private static final String hostName; - - static { - try { - hostName = InetAddress.getLocalHost().getHostName(); - } catch (final UnknownHostException e) { - throw new RuntimeException(e); - } - } - - - /** - * @return Default to use host name. - */ - public String generate() { - return hostName; - } + String generate(); } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java index e70ec337e7..25a1db6fc5 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java @@ -8,16 +8,9 @@ import com.fasterxml.jackson.annotation.JsonProperty; import jakarta.validation.constraints.Size; import lombok.Getter; -import software.amazon.awssdk.arns.Arn; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.sts.StsClient; -import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; -import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; import java.util.Map; -import java.util.UUID; public class AwsAuthenticationConfig { private static final String AWS_IAM_ROLE = "role"; @@ -45,35 +38,4 @@ public class AwsAuthenticationConfig { public Region getAwsRegion() { return awsRegion != null ? Region.of(awsRegion) : null; } - - public AwsCredentialsProvider authenticateAwsConfiguration() { - - final AwsCredentialsProvider awsCredentialsProvider; - if (awsStsRoleArn != null && !awsStsRoleArn.isEmpty()) { - try { - Arn.fromString(awsStsRoleArn); - } catch (final Exception e) { - throw new IllegalArgumentException("Invalid ARN format for awsStsRoleArn"); - } - - final StsClient stsClient = StsClient.builder().region(getAwsRegion()).build(); - - AssumeRoleRequest.Builder assumeRoleRequestBuilder = AssumeRoleRequest.builder() - .roleSessionName("Kinesis-source-" + UUID.randomUUID()).roleArn(awsStsRoleArn); - - if (awsStsHeaderOverrides != null && !awsStsHeaderOverrides.isEmpty()) { - assumeRoleRequestBuilder = assumeRoleRequestBuilder.overrideConfiguration( - configuration -> awsStsHeaderOverrides.forEach(configuration::putHeader)); - } - - awsCredentialsProvider = StsAssumeRoleCredentialsProvider.builder() - .stsClient(stsClient) - .refreshRequest(assumeRoleRequestBuilder.build()) - .build(); - - } else { - awsCredentialsProvider = DefaultCredentialsProvider.create(); - } - return awsCredentialsProvider; - } } \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java index fa507f55d5..59a2fd2522 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java @@ -2,13 +2,15 @@ import com.fasterxml.jackson.annotation.JsonValue; -// Reference: https://docs.aws.amazon.com/streams/latest/dev/enhanced-consumers.html +/** + * @see Enhanced Consumers + */ public enum ConsumerStrategy { - POLLING("Polling"), + POLLING("polling"), - ENHANCED_FAN_OUT("Fan-Out"); + ENHANCED_FAN_OUT("fan-out"); private final String value; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfig.java new file mode 100644 index 0000000000..058be8109c --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfig.java @@ -0,0 +1,37 @@ +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; + +import lombok.Getter; +import software.amazon.kinesis.common.InitialPositionInStream; + +import java.util.Arrays; +import java.util.Map; +import java.util.stream.Collectors; + +@Getter +public enum InitialPositionInStreamConfig { + LATEST("latest", InitialPositionInStream.LATEST), + EARLIEST("earliest", InitialPositionInStream.TRIM_HORIZON); + + private final String position; + + private final InitialPositionInStream positionInStream; + + InitialPositionInStreamConfig(final String position, final InitialPositionInStream positionInStream) { + this.position = position; + this.positionInStream = positionInStream; + } + + private static final Map POSITIONS_MAP = Arrays.stream(InitialPositionInStreamConfig.values()) + .collect(Collectors.toMap( + value -> value.position, + value -> value + )); + + public static InitialPositionInStreamConfig fromPositionValue(final String position) { + return POSITIONS_MAP.get(position.toLowerCase()); + } + + public String toString() { + return this.position; + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java index 4be5bb7f95..c8aeb01bfd 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java @@ -19,7 +19,7 @@ public class KinesisSourceConfig { @JsonProperty("streams") @NotNull @Valid - @Size(min = 1, max = 4, message = "Only support a maximum of 4 streams") + @Size(min = 1, max = 4, message = "Provide 1-4 streams to read from.") private List streams; @Getter @@ -49,6 +49,7 @@ public class KinesisSourceConfig { private KinesisStreamPollingConfig pollingConfig; @Getter + @NotNull @JsonProperty("codec") private PluginModel codec; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java index 3faf55469c..a336acece3 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java @@ -6,10 +6,12 @@ import lombok.Getter; import software.amazon.kinesis.common.InitialPositionInStream; +import java.time.Duration; + @Getter public class KinesisStreamConfig { // Checkpointing interval - private static final int MINIMAL_CHECKPOINT_INTERVAL_MILLIS = 2 * 60 * 1000; // 2 minute + private static final Duration MINIMAL_CHECKPOINT_INTERVAL = Duration.ofMillis(2 * 60 * 1000); // 2 minute private static final boolean DEFAULT_ENABLE_CHECKPOINT = false; @JsonProperty("stream_name") @@ -17,16 +19,17 @@ public class KinesisStreamConfig { @Valid private String name; - @JsonProperty("stream_arn") - private String arn; - @JsonProperty("initial_position") - private InitialPositionInStream initialPosition = InitialPositionInStream.LATEST; + private InitialPositionInStreamConfig initialPosition = InitialPositionInStreamConfig.LATEST; @JsonProperty("checkpoint_interval") - private int checkPointIntervalInMilliseconds = MINIMAL_CHECKPOINT_INTERVAL_MILLIS; + private Duration checkPointInterval = MINIMAL_CHECKPOINT_INTERVAL; @Getter - @JsonProperty("enableCheckpoint") + @JsonProperty("enable_checkpoint") private boolean enableCheckPoint = DEFAULT_ENABLE_CHECKPOINT; + + public InitialPositionInStream getInitialPosition() { + return initialPosition.getPositionInStream(); + } } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java index 36dbedbdd6..b205df2b2f 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java @@ -3,15 +3,17 @@ import com.fasterxml.jackson.annotation.JsonProperty; import lombok.Getter; +import java.time.Duration; + public class KinesisStreamPollingConfig { private static final int DEFAULT_MAX_RECORDS = 10000; - private static final int IDLE_TIME_BETWEEN_READS_IN_MILLIS = 250; + private static final Duration IDLE_TIME_BETWEEN_READS = Duration.ofMillis(250); @Getter - @JsonProperty("maxPollingRecords") + @JsonProperty("max_polling_records") private int maxPollingRecords = DEFAULT_MAX_RECORDS; @Getter - @JsonProperty("idleTimeBetweenReadsInMillis") - private int idleTimeBetweenReadsInMillis = IDLE_TIME_BETWEEN_READS_IN_MILLIS; + @JsonProperty("idle_time_between_reads") + private Duration idleTimeBetweenReads = IDLE_TIME_BETWEEN_READS; } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java index ada8094908..956ba88f7a 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java @@ -41,7 +41,7 @@ public class KinesisRecordProcessor implements ShardRecordProcessor { private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class); private final StreamIdentifier streamIdentifier; private final KinesisStreamConfig kinesisStreamConfig; - private final int checkpointIntervalMilliSeconds; + private final Duration checkpointInterval; private final KinesisSourceConfig kinesisSourceConfig; private final Buffer> buffer; private String kinesisShardId; @@ -75,7 +75,7 @@ public KinesisRecordProcessor(Buffer> buffer, this.acknowledgementSetCallbackCounter = pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.recordProcessingErrors = pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.checkpointFailures = pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); - this.checkpointIntervalMilliSeconds = kinesisStreamConfig.getCheckPointIntervalInMilliseconds(); + this.checkpointInterval = kinesisStreamConfig.getCheckPointInterval(); this.buffer = buffer; } @@ -87,7 +87,7 @@ private KinesisStreamConfig getStreamConfig(final KinesisSourceConfig kinesisSou public void initialize(InitializationInput initializationInput) { // Called once when the processor is initialized. kinesisShardId = initializationInput.shardId(); - LOG.info("Initialize Processor for shard: " + kinesisShardId); + LOG.info("Initialize Processor for shard: {}", kinesisShardId); lastCheckpointTimeInMillis = System.currentTimeMillis(); } @@ -126,7 +126,7 @@ public void processRecords(ProcessRecordsInput processRecordsInput) { acknowledgementSetOpt.ifPresent(AcknowledgementSet::complete); // Checkpoint for shard - if (kinesisStreamConfig.isEnableCheckPoint() && System.currentTimeMillis() - lastCheckpointTimeInMillis > checkpointIntervalMilliSeconds) { + if (kinesisStreamConfig.isEnableCheckPoint() && System.currentTimeMillis() - lastCheckpointTimeInMillis > checkpointInterval.toMillis()) { LOG.info("Regular checkpointing for shard " + kinesisShardId); checkpoint(processRecordsInput.checkpointer()); lastCheckpointTimeInMillis = System.currentTimeMillis(); @@ -152,7 +152,7 @@ public void leaseLost(LeaseLostInput leaseLostInput) { @Override public void shardEnded(ShardEndedInput shardEndedInput) { - LOG.info("Reached shard end, checkpointing shard: {}", kinesisShardId); + LOG.debug("Reached shard end, checkpointing shard: {}", kinesisShardId); checkpoint(shardEndedInput.checkpointer()); } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplierTest.java similarity index 84% rename from data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java rename to data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplierTest.java index dc9bbf3fbe..c3eeb28d40 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/DefaultKinesisLeaseConfigSupplierTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplierTest.java @@ -12,7 +12,7 @@ import static org.mockito.Mockito.when; @ExtendWith(MockitoExtension.class) -public class DefaultKinesisLeaseConfigSupplierTest { +public class KinesisLeaseConfigSupplierTest { private static final String LEASE_COORDINATION_TABLE = "lease-table"; @Mock KinesisLeaseConfig kinesisLeaseConfig; @@ -20,8 +20,8 @@ public class DefaultKinesisLeaseConfigSupplierTest { @Mock KinesisLeaseCoordinationTableConfig kinesisLeaseCoordinationTableConfig; - private DefaultKinesisLeaseConfigSupplier createObjectUnderTest() { - return new DefaultKinesisLeaseConfigSupplier(kinesisLeaseConfig); + private KinesisLeaseConfigSupplier createObjectUnderTest() { + return new KinesisLeaseConfigSupplier(kinesisLeaseConfig); } @Test @@ -37,14 +37,14 @@ void testGetters() { @Test void testGettersWithNullTableConfig() { when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(null); - DefaultKinesisLeaseConfigSupplier defaultKinesisLeaseConfigSupplier = createObjectUnderTest(); + KinesisLeaseConfigSupplier defaultKinesisLeaseConfigSupplier = createObjectUnderTest(); assertThat(defaultKinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable(), equalTo(null)); } @Test void testGettersWithNullConfig() { - KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier = new DefaultKinesisLeaseConfigSupplier(null); + KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier = new KinesisLeaseConfigSupplier(null); assertThat(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig(), equalTo(Optional.empty())); } } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java index 5c6e298760..fc81595f82 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java @@ -1,9 +1,11 @@ package org.opensearch.dataprepper.plugins.kinesis.source; import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; import org.opensearch.dataprepper.test.helper.ReflectivelySetField; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.cloudwatch.CloudWatchAsyncClient; import software.amazon.awssdk.services.dynamodb.DynamoDbAsyncClient; @@ -14,7 +16,9 @@ import java.util.UUID; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class KinesisClientFactoryTest { private Region region = Region.US_EAST_1; @@ -33,12 +37,15 @@ void testCreateClient() throws NoSuchFieldException, IllegalAccessException { ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", roleArn); + AwsCredentialsProvider defaultCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(eq(AwsCredentialsOptions.defaultOptions()))).thenReturn(defaultCredentialsProvider); + KinesisClientFactory clientFactory = new KinesisClientFactory(awsCredentialsSupplier, awsAuthenticationOptionsConfig); final DynamoDbAsyncClient dynamoDbAsyncClient = clientFactory.buildDynamoDBClient(Region.US_EAST_1); assertNotNull(dynamoDbAsyncClient); - final KinesisAsyncClient kinesisAsyncClient = clientFactory.buildKinesisAsyncClient(); + final KinesisAsyncClient kinesisAsyncClient = clientFactory.buildKinesisAsyncClient(Region.US_EAST_1); assertNotNull(kinesisAsyncClient); final CloudWatchAsyncClient cloudWatchAsyncClient = clientFactory.buildCloudWatchAsyncClient(Region.US_EAST_1); diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java index d5d8a93c4f..541fd10bd5 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java @@ -30,6 +30,7 @@ import software.amazon.kinesis.coordinator.Scheduler; import software.amazon.kinesis.metrics.MetricsLevel; +import java.time.Duration; import java.time.Instant; import java.util.ArrayList; import java.util.List; @@ -53,7 +54,7 @@ public class KinesisServiceTest { private final String PIPELINE_NAME = "kinesis-pipeline-test"; private final String streamId = "stream-1"; - private static final int CHECKPOINT_INTERVAL_MS = 0; + private static final Duration CHECKPOINT_INTERVAL = Duration.ofMillis(0); private static final int NUMBER_OF_RECORDS_TO_ACCUMULATE = 10; private static final int DEFAULT_MAX_RECORDS = 10000; private static final int IDLE_TIME_BETWEEN_READS_IN_MILLIS = 250; @@ -111,6 +112,9 @@ public class KinesisServiceTest { @Mock KinesisLeaseCoordinationTableConfig kinesisLeaseCoordinationTableConfig; + @Mock + WorkerIdentifierGenerator workerIdentifierGenerator; + @BeforeEach void setup() { awsAuthenticationConfig = mock(AwsAuthenticationConfig.class); @@ -126,6 +130,7 @@ void setup() { buffer = mock(Buffer.class); kinesisLeaseConfigSupplier = mock(KinesisLeaseConfigSupplier.class); kinesisLeaseConfig = mock(KinesisLeaseConfig.class); + workerIdentifierGenerator = mock(WorkerIdentifierGenerator.class); kinesisLeaseCoordinationTableConfig = mock(KinesisLeaseCoordinationTableConfig.class); when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(kinesisLeaseCoordinationTableConfig); when(kinesisLeaseCoordinationTableConfig.getTableName()).thenReturn("kinesis-lease-table"); @@ -156,12 +161,12 @@ void setup() { when(kinesisSourceConfig.getAwsAuthenticationConfig()).thenReturn(awsAuthenticationConfig); when(kinesisStreamConfig.getName()).thenReturn(streamId); - when(kinesisStreamConfig.getCheckPointIntervalInMilliseconds()).thenReturn(CHECKPOINT_INTERVAL_MS); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(CHECKPOINT_INTERVAL); when(kinesisStreamConfig.getInitialPosition()).thenReturn(InitialPositionInStream.LATEST); - when(kinesisSourceConfig.getConsumerStrategy()).thenReturn(ConsumerStrategy.POLLING); + when(kinesisSourceConfig.getConsumerStrategy()).thenReturn(ConsumerStrategy.ENHANCED_FAN_OUT); when(kinesisSourceConfig.getPollingConfig()).thenReturn(kinesisStreamPollingConfig); when(kinesisStreamPollingConfig.getMaxPollingRecords()).thenReturn(DEFAULT_MAX_RECORDS); - when(kinesisStreamPollingConfig.getIdleTimeBetweenReadsInMillis()).thenReturn(IDLE_TIME_BETWEEN_READS_IN_MILLIS); + when(kinesisStreamPollingConfig.getIdleTimeBetweenReads()).thenReturn(Duration.ofMillis(IDLE_TIME_BETWEEN_READS_IN_MILLIS)); List streamConfigs = new ArrayList<>(); streamConfigs.add(kinesisStreamConfig); @@ -169,16 +174,17 @@ void setup() { when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); when(kinesisClientFactory.buildDynamoDBClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(dynamoDbClient); - when(kinesisClientFactory.buildKinesisAsyncClient()).thenReturn(kinesisClient); + when(kinesisClientFactory.buildKinesisAsyncClient(awsAuthenticationConfig.getAwsRegion())).thenReturn(kinesisClient); when(kinesisClientFactory.buildCloudWatchAsyncClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(cloudWatchClient); when(kinesisClient.serviceClientConfiguration()).thenReturn(KinesisServiceClientConfiguration.builder().region(Region.US_EAST_1).build()); when(scheduler.startGracefulShutdown()).thenReturn(CompletableFuture.completedFuture(true)); when(pipelineDescription.getPipelineName()).thenReturn(PIPELINE_NAME); + when(workerIdentifierGenerator.generate()).thenReturn(UUID.randomUUID().toString()); } public KinesisService createObjectUnderTest() { return new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, - pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier, workerIdentifierGenerator); } @Test @@ -186,19 +192,20 @@ void testServiceStart() { KinesisService kinesisService = createObjectUnderTest(); kinesisService.start(buffer); assertNotNull(kinesisService.getScheduler(buffer)); + verify(workerIdentifierGenerator, times(1)).generate(); } @Test void testServiceThrowsWhenLeaseConfigIsInvalid() { when(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig()).thenReturn(Optional.empty()); assertThrows(IllegalStateException.class, () -> new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, - pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier)); + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier, workerIdentifierGenerator)); } @Test void testCreateScheduler() { KinesisService kinesisService = new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, - pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier, workerIdentifierGenerator); Scheduler schedulerObjectUnderTest = kinesisService.createScheduler(buffer); assertNotNull(schedulerObjectUnderTest); @@ -210,13 +217,14 @@ void testCreateScheduler() { assertSame(schedulerObjectUnderTest.metricsConfig().metricsLevel(), MetricsLevel.DETAILED); assertNotNull(schedulerObjectUnderTest.processorConfig()); assertNotNull(schedulerObjectUnderTest.retrievalConfig()); + verify(workerIdentifierGenerator, times(1)).generate(); } @Test void testCreateSchedulerWithPollingStrategy() { when(kinesisSourceConfig.getConsumerStrategy()).thenReturn(ConsumerStrategy.POLLING); KinesisService kinesisService = new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, - pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier); + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier, workerIdentifierGenerator); Scheduler schedulerObjectUnderTest = kinesisService.createScheduler(buffer); assertNotNull(schedulerObjectUnderTest); @@ -228,6 +236,7 @@ void testCreateSchedulerWithPollingStrategy() { assertSame(schedulerObjectUnderTest.metricsConfig().metricsLevel(), MetricsLevel.DETAILED); assertNotNull(schedulerObjectUnderTest.processorConfig()); assertNotNull(schedulerObjectUnderTest.retrievalConfig()); + verify(workerIdentifierGenerator, times(1)).generate(); } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java index 14ac812138..eeb7f70eee 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java @@ -3,6 +3,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mock; +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; @@ -17,6 +18,7 @@ import org.opensearch.dataprepper.plugins.kinesis.source.configuration.AwsAuthenticationConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.regions.Region; import java.util.List; @@ -85,12 +87,16 @@ void setup() { kinesisLeaseConfig = mock(KinesisLeaseConfig.class); kinesisLeaseCoordinationTableConfig = mock(KinesisLeaseCoordinationTableConfig.class); when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(kinesisLeaseCoordinationTableConfig); - when(kinesisLeaseCoordinationTableConfig.getTableName()).thenReturn("us-east-1"); + when(kinesisLeaseCoordinationTableConfig.getTableName()).thenReturn("table-name"); + when(kinesisLeaseCoordinationTableConfig.getRegion()).thenReturn("us-east-1"); + when(kinesisLeaseCoordinationTableConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); when(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig()).thenReturn(Optional.ofNullable(kinesisLeaseConfig)); - when(awsAuthenticationConfig.getAwsRegion()).thenReturn(Region.of("us-west-2")); + when(awsAuthenticationConfig.getAwsRegion()).thenReturn(Region.US_EAST_1); when(awsAuthenticationConfig.getAwsStsRoleArn()).thenReturn(UUID.randomUUID().toString()); when(awsAuthenticationConfig.getAwsStsExternalId()).thenReturn(UUID.randomUUID().toString()); final Map stsHeaderOverrides = Map.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + AwsCredentialsProvider defaultCredentialsProvider = mock(AwsCredentialsProvider.class); + when(awsCredentialsSupplier.getProvider(AwsCredentialsOptions.defaultOptions())).thenReturn(defaultCredentialsProvider); when(awsAuthenticationConfig.getAwsStsHeaderOverrides()).thenReturn(stsHeaderOverrides); when(kinesisSourceConfig.getAwsAuthenticationConfig()).thenReturn(awsAuthenticationConfig); when(pipelineDescription.getPipelineName()).thenReturn(PIPELINE_NAME); diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java index 09c64966f7..00f2b668e0 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java @@ -1,44 +1,21 @@ package org.opensearch.dataprepper.plugins.kinesis.source.configuration; import com.fasterxml.jackson.databind.ObjectMapper; -import org.hamcrest.CoreMatchers; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Nested; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import org.mockito.ArgumentCaptor; -import org.mockito.MockedStatic; -import org.opensearch.dataprepper.test.helper.ReflectivelySetField; -import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; -import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; -import software.amazon.awssdk.awscore.AwsRequestOverrideConfiguration; import software.amazon.awssdk.regions.Region; -import software.amazon.awssdk.services.sts.StsClient; -import software.amazon.awssdk.services.sts.StsClientBuilder; -import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; import java.util.Collections; import java.util.Map; import java.util.UUID; -import java.util.function.Consumer; -import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.CoreMatchers.sameInstance; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.mockStatic; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; public class AwsAuthenticationConfigTest { private ObjectMapper objectMapper = new ObjectMapper(); - private final String TEST_ROLE = "arn:aws:iam::123456789012:role/test-role"; @ParameterizedTest @ValueSource(strings = {"us-east-1", "us-west-2", "eu-central-1"}) @@ -100,164 +77,4 @@ void getAwsStsHeaderOverridesReturnsNullIfNotInJSON() { final AwsAuthenticationConfig objectUnderTest = objectMapper.convertValue(jsonMap, AwsAuthenticationConfig.class); assertThat(objectUnderTest.getAwsStsHeaderOverrides(), nullValue()); } - - @Test - void authenticateAWSConfigurationShouldReturnWithoutStsRoleArn() throws NoSuchFieldException, IllegalAccessException { - AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", null); - - final DefaultCredentialsProvider mockedCredentialsProvider = mock(DefaultCredentialsProvider.class); - final AwsCredentialsProvider actualCredentialsProvider; - try (final MockedStatic defaultCredentialsProviderMockedStatic = mockStatic(DefaultCredentialsProvider.class)) { - defaultCredentialsProviderMockedStatic.when(DefaultCredentialsProvider::create) - .thenReturn(mockedCredentialsProvider); - actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); - } - - assertThat(actualCredentialsProvider, sameInstance(mockedCredentialsProvider)); - } - - - @Nested - class WithSts { - private StsClient stsClient; - private StsClientBuilder stsClientBuilder; - - @BeforeEach - void setUp() { - stsClient = mock(StsClient.class); - stsClientBuilder = mock(StsClientBuilder.class); - - when(stsClientBuilder.build()).thenReturn(stsClient); - } - - @Test - void authenticateAWSConfigurationShouldReturnWithStsRoleArn() throws NoSuchFieldException, IllegalAccessException { - AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", TEST_ROLE); - - when(stsClientBuilder.region(Region.US_EAST_1)).thenReturn(stsClientBuilder); - final AssumeRoleRequest.Builder assumeRoleRequestBuilder = mock(AssumeRoleRequest.Builder.class); - when(assumeRoleRequestBuilder.roleSessionName(anyString())) - .thenReturn(assumeRoleRequestBuilder); - when(assumeRoleRequestBuilder.roleArn(anyString())) - .thenReturn(assumeRoleRequestBuilder); - - final AwsCredentialsProvider actualCredentialsProvider; - try (final MockedStatic stsClientMockedStatic = mockStatic(StsClient.class); - final MockedStatic assumeRoleRequestMockedStatic = mockStatic(AssumeRoleRequest.class)) { - stsClientMockedStatic.when(StsClient::builder).thenReturn(stsClientBuilder); - assumeRoleRequestMockedStatic.when(AssumeRoleRequest::builder).thenReturn(assumeRoleRequestBuilder); - actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); - } - - assertThat(actualCredentialsProvider, instanceOf(AwsCredentialsProvider.class)); - - verify(assumeRoleRequestBuilder).roleArn(TEST_ROLE); - verify(assumeRoleRequestBuilder).roleSessionName(anyString()); - verify(assumeRoleRequestBuilder).build(); - verifyNoMoreInteractions(assumeRoleRequestBuilder); - } - - @Test - void authenticateAWSConfigurationShouldReturnWithStsRoleArnWhenNoRegion() throws NoSuchFieldException, IllegalAccessException { - AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", null); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", TEST_ROLE); - assertThat(awsAuthenticationOptionsConfig.getAwsRegion(), CoreMatchers.equalTo(null)); - - when(stsClientBuilder.region(null)).thenReturn(stsClientBuilder); - - final AwsCredentialsProvider actualCredentialsProvider; - try (final MockedStatic stsClientMockedStatic = mockStatic(StsClient.class)) { - stsClientMockedStatic.when(StsClient::builder).thenReturn(stsClientBuilder); - actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); - } - - assertThat(actualCredentialsProvider, instanceOf(AwsCredentialsProvider.class)); - } - - @Test - void authenticateAWSConfigurationShouldOverrideSTSHeadersWhenHeaderOverridesSet() throws NoSuchFieldException, IllegalAccessException { - final String headerName1 = UUID.randomUUID().toString(); - final String headerValue1 = UUID.randomUUID().toString(); - final String headerName2 = UUID.randomUUID().toString(); - final String headerValue2 = UUID.randomUUID().toString(); - final Map overrideHeaders = Map.of(headerName1, headerValue1, headerName2, headerValue2); - - AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", TEST_ROLE); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsHeaderOverrides", overrideHeaders); - - when(stsClientBuilder.region(Region.US_EAST_1)).thenReturn(stsClientBuilder); - - final AssumeRoleRequest.Builder assumeRoleRequestBuilder = mock(AssumeRoleRequest.Builder.class); - when(assumeRoleRequestBuilder.roleSessionName(anyString())) - .thenReturn(assumeRoleRequestBuilder); - when(assumeRoleRequestBuilder.roleArn(anyString())) - .thenReturn(assumeRoleRequestBuilder); - when(assumeRoleRequestBuilder.overrideConfiguration(any(Consumer.class))) - .thenReturn(assumeRoleRequestBuilder); - - final AwsCredentialsProvider actualCredentialsProvider; - try (final MockedStatic stsClientMockedStatic = mockStatic(StsClient.class); - final MockedStatic assumeRoleRequestMockedStatic = mockStatic(AssumeRoleRequest.class)) { - stsClientMockedStatic.when(StsClient::builder).thenReturn(stsClientBuilder); - assumeRoleRequestMockedStatic.when(AssumeRoleRequest::builder).thenReturn(assumeRoleRequestBuilder); - actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); - } - - assertThat(actualCredentialsProvider, instanceOf(AwsCredentialsProvider.class)); - - final ArgumentCaptor> configurationCaptor = ArgumentCaptor.forClass(Consumer.class); - - verify(assumeRoleRequestBuilder).roleArn(TEST_ROLE); - verify(assumeRoleRequestBuilder).roleSessionName(anyString()); - verify(assumeRoleRequestBuilder).overrideConfiguration(configurationCaptor.capture()); - verify(assumeRoleRequestBuilder).build(); - verifyNoMoreInteractions(assumeRoleRequestBuilder); - - final Consumer actualOverride = configurationCaptor.getValue(); - - final AwsRequestOverrideConfiguration.Builder configurationBuilder = mock(AwsRequestOverrideConfiguration.Builder.class); - actualOverride.accept(configurationBuilder); - verify(configurationBuilder).putHeader(headerName1, headerValue1); - verify(configurationBuilder).putHeader(headerName2, headerValue2); - verifyNoMoreInteractions(configurationBuilder); - } - - @Test - void authenticateAWSConfigurationShouldNotOverrideSTSHeadersWhenHeaderOverridesAreEmpty() throws NoSuchFieldException, IllegalAccessException { - - AwsAuthenticationConfig awsAuthenticationOptionsConfig = new AwsAuthenticationConfig(); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsRegion", "us-east-1"); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsRoleArn", TEST_ROLE); - ReflectivelySetField.setField(AwsAuthenticationConfig.class, awsAuthenticationOptionsConfig, "awsStsHeaderOverrides", Collections.emptyMap()); - - when(stsClientBuilder.region(Region.US_EAST_1)).thenReturn(stsClientBuilder); - final AssumeRoleRequest.Builder assumeRoleRequestBuilder = mock(AssumeRoleRequest.Builder.class); - when(assumeRoleRequestBuilder.roleSessionName(anyString())) - .thenReturn(assumeRoleRequestBuilder); - when(assumeRoleRequestBuilder.roleArn(anyString())) - .thenReturn(assumeRoleRequestBuilder); - - final AwsCredentialsProvider actualCredentialsProvider; - try (final MockedStatic stsClientMockedStatic = mockStatic(StsClient.class); - final MockedStatic assumeRoleRequestMockedStatic = mockStatic(AssumeRoleRequest.class)) { - stsClientMockedStatic.when(StsClient::builder).thenReturn(stsClientBuilder); - assumeRoleRequestMockedStatic.when(AssumeRoleRequest::builder).thenReturn(assumeRoleRequestBuilder); - actualCredentialsProvider = awsAuthenticationOptionsConfig.authenticateAwsConfiguration(); - } - - assertThat(actualCredentialsProvider, instanceOf(AwsCredentialsProvider.class)); - - verify(assumeRoleRequestBuilder).roleArn(TEST_ROLE); - verify(assumeRoleRequestBuilder).roleSessionName(anyString()); - verify(assumeRoleRequestBuilder).build(); - verifyNoMoreInteractions(assumeRoleRequestBuilder); - } - } } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfigTest.java new file mode 100644 index 0000000000..68d66acf86 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfigTest.java @@ -0,0 +1,28 @@ +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; + +import org.junit.jupiter.api.Test; +import software.amazon.kinesis.common.InitialPositionInStream; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +public class InitialPositionInStreamConfigTest { + + @Test + void testInitialPositionGetByNameLATEST() { + final InitialPositionInStreamConfig initialPositionInStreamConfig = InitialPositionInStreamConfig.fromPositionValue("latest"); + assertEquals(initialPositionInStreamConfig, InitialPositionInStreamConfig.LATEST); + assertEquals(initialPositionInStreamConfig.toString(), "latest"); + assertEquals(initialPositionInStreamConfig.getPosition(), "latest"); + assertEquals(initialPositionInStreamConfig.getPositionInStream(), InitialPositionInStream.LATEST); + } + + @Test + void testInitialPositionGetByNameEarliest() { + final InitialPositionInStreamConfig initialPositionInStreamConfig = InitialPositionInStreamConfig.fromPositionValue("earliest"); + assertEquals(initialPositionInStreamConfig, InitialPositionInStreamConfig.EARLIEST); + assertEquals(initialPositionInStreamConfig.toString(), "earliest"); + assertEquals(initialPositionInStreamConfig.getPosition(), "earliest"); + assertEquals(initialPositionInStreamConfig.getPositionInStream(), InitialPositionInStream.TRIM_HORIZON); + } + +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java index 863e7fb4b2..0de9e870e8 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java @@ -1,19 +1,23 @@ package org.opensearch.dataprepper.plugins.kinesis.source.configuration; import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.module.SimpleModule; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; import com.fasterxml.jackson.datatype.jsr310.JavaTimeModule; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; -import org.yaml.snakeyaml.Yaml; +import org.opensearch.dataprepper.pipeline.parser.DataPrepperDurationDeserializer; import software.amazon.awssdk.regions.Region; import software.amazon.kinesis.common.InitialPositionInStream; -import java.io.FileReader; +import java.io.File; import java.io.IOException; import java.io.Reader; import java.io.StringReader; +import java.time.Duration; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -26,35 +30,36 @@ import static org.junit.jupiter.api.Assertions.assertTrue; public class KinesisSourceConfigTest { - private static final String SIMPLE_PIPELINE_CONFIG = "simple-pipeline.yaml"; - private static final String SIMPLE_PIPELINE_CONFIG_2 = "simple-pipeline-2.yaml"; - private static final int MINIMAL_CHECKPOINT_INTERVAL_MILLIS = 2 * 60 * 1000; // 2 minute - private static final int DEFAULT_MAX_RECORDS = 10000; - private static final int IDLE_TIME_BETWEEN_READS_IN_MILLIS = 250; + private static final String PIPELINE_CONFIG_WITH_ACKS_ENABLED = "pipeline_with_acks_enabled.yaml"; + private static final String PIPELINE_CONFIG_WITH_POLLING_CONFIG_ENABLED = "pipeline_with_polling_config_enabled.yaml"; + private static final String PIPELINE_CONFIG_CHECKPOINT_ENABLED = "pipeline_with_checkpoint_enabled.yaml"; + private static final Duration MINIMAL_CHECKPOINT_INTERVAL = Duration.ofMillis(2 * 60 * 1000); // 2 minute KinesisSourceConfig kinesisSourceConfig; + ObjectMapper objectMapper; + @BeforeEach void setUp(TestInfo testInfo) throws IOException { String fileName = testInfo.getTags().stream().findFirst().orElse(""); - Yaml yaml = new Yaml(); - FileReader fileReader = new FileReader(getClass().getClassLoader().getResource(fileName).getFile()); - Object data = yaml.load(fileReader); - ObjectMapper mapper = new ObjectMapper(); - if (data instanceof Map) { - Map propertyMap = (Map) data; - Map logPipelineMap = (Map) propertyMap.get("kinesis-pipeline"); - Map sourceMap = (Map) logPipelineMap.get("source"); - Map kinesisConfigMap = (Map) sourceMap.get("kinesis"); - mapper.registerModule(new JavaTimeModule()); - String json = mapper.writeValueAsString(kinesisConfigMap); - Reader reader = new StringReader(json); - kinesisSourceConfig = mapper.readValue(reader, KinesisSourceConfig.class); - } + final File configurationFile = new File(getClass().getClassLoader().getResource(fileName).getFile()); + objectMapper = new ObjectMapper(new YAMLFactory()); + SimpleModule simpleModule = new SimpleModule(); + simpleModule.addDeserializer(Duration.class, new DataPrepperDurationDeserializer()); + objectMapper.registerModule(new JavaTimeModule()); + objectMapper.registerModule(simpleModule); + + final Map pipelineConfig = objectMapper.readValue(configurationFile, Map.class); + final Map sourceMap = (Map) pipelineConfig.get("source"); + final Map kinesisConfigMap = (Map) sourceMap.get("kinesis"); + String json = objectMapper.writeValueAsString(kinesisConfigMap); + final Reader reader = new StringReader(json); + kinesisSourceConfig = objectMapper.readValue(reader, KinesisSourceConfig.class); + } @Test - @Tag(SIMPLE_PIPELINE_CONFIG) + @Tag(PIPELINE_CONFIG_WITH_ACKS_ENABLED) void testSourceConfig() { assertThat(kinesisSourceConfig, notNullValue()); @@ -69,21 +74,20 @@ void testSourceConfig() { assertNull(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsHeaderOverrides()); List streamConfigs = kinesisSourceConfig.getStreams(); - assertNull(kinesisSourceConfig.getCodec()); + assertNotNull(kinesisSourceConfig.getCodec()); assertEquals(kinesisSourceConfig.getConsumerStrategy(), ConsumerStrategy.ENHANCED_FAN_OUT); assertNull(kinesisSourceConfig.getPollingConfig()); for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { assertTrue(kinesisStreamConfig.getName().contains("stream")); - assertTrue(kinesisStreamConfig.getArn().contains("123456789012:stream/stream")); assertFalse(kinesisStreamConfig.isEnableCheckPoint()); assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.LATEST); - assertEquals(kinesisStreamConfig.getCheckPointIntervalInMilliseconds(), MINIMAL_CHECKPOINT_INTERVAL_MILLIS); + assertEquals(kinesisStreamConfig.getCheckPointInterval(), MINIMAL_CHECKPOINT_INTERVAL); } } @Test - @Tag(SIMPLE_PIPELINE_CONFIG_2) + @Tag(PIPELINE_CONFIG_WITH_POLLING_CONFIG_ENABLED) void testSourceConfigWithStreamCodec() { assertThat(kinesisSourceConfig, notNullValue()); @@ -101,14 +105,44 @@ void testSourceConfigWithStreamCodec() { assertEquals(kinesisSourceConfig.getConsumerStrategy(), ConsumerStrategy.POLLING); assertNotNull(kinesisSourceConfig.getPollingConfig()); assertEquals(kinesisSourceConfig.getPollingConfig().getMaxPollingRecords(), 10); - assertEquals(kinesisSourceConfig.getPollingConfig().getIdleTimeBetweenReadsInMillis(), 10); + assertEquals(kinesisSourceConfig.getPollingConfig().getIdleTimeBetweenReads(), Duration.ofSeconds(10)); for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { assertTrue(kinesisStreamConfig.getName().contains("stream")); - assertTrue(kinesisStreamConfig.getArn().contains("123456789012:stream/stream")); assertFalse(kinesisStreamConfig.isEnableCheckPoint()); assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.LATEST); - assertEquals(kinesisStreamConfig.getCheckPointIntervalInMilliseconds(), MINIMAL_CHECKPOINT_INTERVAL_MILLIS); + assertEquals(kinesisStreamConfig.getCheckPointInterval(), MINIMAL_CHECKPOINT_INTERVAL); + } + } + + @Test + @Tag(PIPELINE_CONFIG_CHECKPOINT_ENABLED) + void testSourceConfigWithInitialPosition() { + + assertThat(kinesisSourceConfig, notNullValue()); + assertEquals(KinesisSourceConfig.DEFAULT_NUMBER_OF_RECORDS_TO_ACCUMULATE, kinesisSourceConfig.getNumberOfRecordsToAccumulate()); + assertEquals(KinesisSourceConfig.DEFAULT_TIME_OUT_IN_MILLIS, kinesisSourceConfig.getBufferTimeout()); + assertFalse(kinesisSourceConfig.isAcknowledgments()); + assertEquals(KinesisSourceConfig.DEFAULT_SHARD_ACKNOWLEDGEMENT_TIMEOUT, kinesisSourceConfig.getShardAcknowledgmentTimeout()); + assertThat(kinesisSourceConfig.getAwsAuthenticationConfig(), notNullValue()); + assertEquals(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsRegion(), Region.US_EAST_1); + assertEquals(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsRoleArn(), "arn:aws:iam::123456789012:role/OSI-PipelineRole"); + assertNull(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsExternalId()); + assertNull(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsStsHeaderOverrides()); + assertNotNull(kinesisSourceConfig.getCodec()); + List streamConfigs = kinesisSourceConfig.getStreams(); + assertEquals(kinesisSourceConfig.getConsumerStrategy(), ConsumerStrategy.ENHANCED_FAN_OUT); + + Map expectedCheckpointIntervals = new HashMap<>(); + expectedCheckpointIntervals.put("stream-1", Duration.ofSeconds(20)); + expectedCheckpointIntervals.put("stream-2", Duration.ofMinutes(15)); + expectedCheckpointIntervals.put("stream-3", Duration.ofHours(2)); + + for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { + assertTrue(kinesisStreamConfig.getName().contains("stream")); + assertTrue(kinesisStreamConfig.isEnableCheckPoint()); + assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.TRIM_HORIZON); + assertEquals(kinesisStreamConfig.getCheckPointInterval(), expectedCheckpointIntervals.get(kinesisStreamConfig.getName())); } } } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java index 21e83e4e39..9d5d8aefec 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java @@ -2,6 +2,8 @@ import org.junit.jupiter.api.Test; +import java.time.Duration; + import static org.junit.jupiter.api.Assertions.assertEquals; public class KinesisStreamPollingConfigTest { @@ -12,7 +14,7 @@ public class KinesisStreamPollingConfigTest { void testConfig() { KinesisStreamPollingConfig kinesisStreamPollingConfig = new KinesisStreamPollingConfig(); assertEquals(kinesisStreamPollingConfig.getMaxPollingRecords(), DEFAULT_MAX_RECORDS); - assertEquals(kinesisStreamPollingConfig.getIdleTimeBetweenReadsInMillis(), IDLE_TIME_BETWEEN_READS_IN_MILLIS); + assertEquals(kinesisStreamPollingConfig.getIdleTimeBetweenReads(), Duration.ofMillis(IDLE_TIME_BETWEEN_READS_IN_MILLIS)); } } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java index e226596849..c9a3617b1a 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java @@ -62,7 +62,7 @@ public class KinesisRecordProcessorTest { private static final String streamId = "stream-1"; private static final String codec_plugin_name = "json"; - private static final int CHECKPOINT_INTERVAL_MS = 1000; + private static final Duration CHECKPOINT_INTERVAL = Duration.ofMillis(1000); private static final int NUMBER_OF_RECORDS_TO_ACCUMULATE = 10; @Mock @@ -124,7 +124,7 @@ public void setup() { InputCodec codec = mock(InputCodec.class); when(pluginFactory.loadPlugin(eq(InputCodec.class), any())).thenReturn(codec); - when(kinesisStreamConfig.getCheckPointIntervalInMilliseconds()).thenReturn(CHECKPOINT_INTERVAL_MS); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(CHECKPOINT_INTERVAL); when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); when(kinesisSourceConfig.getStreams()).thenReturn(List.of(kinesisStreamConfig)); when(processRecordsInput.checkpointer()).thenReturn(checkpointer); diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_acks_enabled.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_acks_enabled.yaml new file mode 100644 index 0000000000..e5260372f5 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_acks_enabled.yaml @@ -0,0 +1,12 @@ +source: + kinesis: + streams: + - stream_name: "stream-1" + - stream_name: "stream-2" + - stream_name: "stream-3" + codec: + ndjson: + aws: + sts_role_arn: "arn:aws:iam::123456789012:role/OSI-PipelineRole" + region: "us-east-1" + acknowledgments: true \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_checkpoint_enabled.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_checkpoint_enabled.yaml new file mode 100644 index 0000000000..e918048529 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_checkpoint_enabled.yaml @@ -0,0 +1,20 @@ +source: + kinesis: + streams: + - stream_name: "stream-1" + initial_position: "EARLIEST" + enable_checkpoint: true + checkpoint_interval: "20s" + - stream_name: "stream-2" + initial_position: "EARLIEST" + enable_checkpoint: true + checkpoint_interval: "PT15M" + - stream_name: "stream-3" + initial_position: "EARLIEST" + enable_checkpoint: true + checkpoint_interval: "PT2H" + codec: + ndjson: + aws: + sts_role_arn: "arn:aws:iam::123456789012:role/OSI-PipelineRole" + region: "us-east-1" \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_polling_config_enabled.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_polling_config_enabled.yaml new file mode 100644 index 0000000000..4a3156ec2a --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_polling_config_enabled.yaml @@ -0,0 +1,15 @@ +source: + kinesis: + streams: + - stream_name: "stream-1" + - stream_name: "stream-2" + - stream_name: "stream-3" + codec: + ndjson: + aws: + sts_role_arn: "arn:aws:iam::123456789012:role/OSI-PipelineRole" + region: "us-east-1" + consumer_strategy: "polling" + polling: + max_polling_records: 10 + idle_time_between_reads: 10s \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline-2.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline-2.yaml deleted file mode 100644 index 46234d50fb..0000000000 --- a/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline-2.yaml +++ /dev/null @@ -1,21 +0,0 @@ -kinesis-pipeline: - source: - kinesis: - streams: - - stream_name: "stream-1" - stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-1" - - stream_name: "stream-2" - stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-2" - - stream_name: "stream-3" - stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-3" - codec: - ndjson: - aws: - sts_role_arn: "arn:aws:iam::123456789012:role/OSI-PipelineRole" - region: "us-east-1" - consumer_strategy: "Polling" - polling: - maxPollingRecords: 10 - idleTimeBetweenReadsInMillis: 10 - sink: - - stdout: \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline.yaml deleted file mode 100644 index 07caf50d02..0000000000 --- a/data-prepper-plugins/kinesis-source/src/test/resources/simple-pipeline.yaml +++ /dev/null @@ -1,16 +0,0 @@ -kinesis-pipeline: - source: - kinesis: - streams: - - stream_name: "stream-1" - stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-1" - - stream_name: "stream-2" - stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-2" - - stream_name: "stream-3" - stream_arn: "arn:aws:kinesis:us-east-1:123456789012:stream/stream-3" - aws: - sts_role_arn: "arn:aws:iam::123456789012:role/OSI-PipelineRole" - region: "us-east-1" - acknowledgments: true - sink: - - stdout: \ No newline at end of file From 0fd763b4125633f1a564b6a4c4acf893d6dce703 Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Tue, 10 Sep 2024 17:01:05 -0700 Subject: [PATCH 6/8] Add license header to all the classes Signed-off-by: Souvik Bose --- .../plugins/kinesis/extension/KinesisLeaseConfig.java | 10 ++++++++++ .../kinesis/extension/KinesisLeaseConfigExtension.java | 10 ++++++++++ .../kinesis/extension/KinesisLeaseConfigProvider.java | 10 ++++++++++ .../kinesis/extension/KinesisLeaseConfigSupplier.java | 10 ++++++++++ .../extension/KinesisLeaseCoordinationTableConfig.java | 10 ++++++++++ .../source/HostNameWorkerIdentifierGenerator.java | 10 ++++++++++ .../plugins/kinesis/source/KinesisClientFactory.java | 10 ++++++++++ .../kinesis/source/KinesisMultiStreamTracker.java | 10 ++++++++++ .../plugins/kinesis/source/KinesisService.java | 10 ++++++++++ .../plugins/kinesis/source/KinesisSource.java | 10 ++++++++++ .../kinesis/source/WorkerIdentifierGenerator.java | 10 ++++++++++ .../source/configuration/AwsAuthenticationConfig.java | 10 ++++++++++ .../kinesis/source/configuration/ConsumerStrategy.java | 10 ++++++++++ .../configuration/InitialPositionInStreamConfig.java | 10 ++++++++++ .../source/configuration/KinesisSourceConfig.java | 10 ++++++++++ .../source/configuration/KinesisStreamConfig.java | 10 ++++++++++ .../configuration/KinesisStreamPollingConfig.java | 10 ++++++++++ .../kinesis/source/KinesisClientFactoryTest.java | 10 ++++++++++ .../kinesis/source/KinesisMultiStreamTrackerTest.java | 10 ++++++++++ .../plugins/kinesis/source/KinesisServiceTest.java | 10 ++++++++++ .../plugins/kinesis/source/KinesisSourceTest.java | 10 ++++++++++ .../configuration/AwsAuthenticationConfigTest.java | 10 ++++++++++ .../InitialPositionInStreamConfigTest.java | 10 ++++++++++ .../source/configuration/KinesisSourceConfigTest.java | 10 ++++++++++ .../configuration/KinesisStreamPollingConfigTest.java | 10 ++++++++++ .../source/processor/KinesisRecordProcessorTest.java | 10 ++++++++++ .../KinesisShardRecordProcessorFactoryTest.java | 10 ++++++++++ 27 files changed, 270 insertions(+) diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java index 7690c05b52..68981c5cba 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.extension; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java index e2cf1ad475..2cca52e565 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.extension; import org.opensearch.dataprepper.model.annotations.DataPrepperExtensionPlugin; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProvider.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProvider.java index 30165c001c..9140ca9e92 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProvider.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProvider.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.extension; import org.opensearch.dataprepper.model.plugin.ExtensionProvider; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java index 6d61e3dd78..6c00e40405 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.extension; import java.util.Optional; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseCoordinationTableConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseCoordinationTableConfig.java index 5369b325aa..d497f01369 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseCoordinationTableConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseCoordinationTableConfig.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.extension; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/HostNameWorkerIdentifierGenerator.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/HostNameWorkerIdentifierGenerator.java index 962b9f5d82..61383304d0 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/HostNameWorkerIdentifierGenerator.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/HostNameWorkerIdentifierGenerator.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; import java.net.InetAddress; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java index ed699bae20..8f3bac38aa 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java index 166be331ea..4414d9fd39 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java index 33cf786884..442f8e4fe6 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; import lombok.Setter; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java index afca88ba3d..220d19cac8 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; import lombok.Setter; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java index 91baa9a831..75bad8761a 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; public interface WorkerIdentifierGenerator { diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java index 25a1db6fc5..6a98f70c3b 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; /* diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java index 59a2fd2522..05fc88f62a 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; import com.fasterxml.jackson.annotation.JsonValue; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfig.java index 058be8109c..37019cc9af 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfig.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; import lombok.Getter; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java index c8aeb01bfd..1414229813 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java index a336acece3..b6e4480229 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java index b205df2b2f..cd7b7a59f6 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java index fc81595f82..f476754eb9 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTrackerTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTrackerTest.java index 4f90243939..edf23b8033 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTrackerTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTrackerTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; import com.google.common.collect.ImmutableList; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java index 541fd10bd5..49f0c898a0 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java index eeb7f70eee..e65ce1dc1a 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source; import org.junit.jupiter.api.BeforeEach; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java index 00f2b668e0..499711c4a9 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfigTest.java index 68d66acf86..2e1b638342 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfigTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java index 0de9e870e8..166d5c587e 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java index 9d5d8aefec..02ac1960ed 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.configuration; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java index c9a3617b1a..87ae19031a 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.processor; import io.micrometer.core.instrument.Counter; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java index a0cdf07ea8..1a74514b6b 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.source.processor; import org.junit.jupiter.api.BeforeEach; From 92a797b0286bbec5355e57102ed9c04dbfdb17d0 Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Tue, 10 Sep 2024 18:39:27 -0700 Subject: [PATCH 7/8] Address review comments Signed-off-by: Souvik Bose --- .../kinesis-source/build.gradle | 5 +- .../kinesis/source/KinesisService.java | 36 +- .../configuration/KinesisStreamConfig.java | 4 - .../converter/KinesisRecordConverter.java | 47 +++ .../processor/KinesisCheckpointerRecord.java | 26 ++ .../processor/KinesisCheckpointerTracker.java | 68 ++++ .../processor/KinesisRecordProcessor.java | 205 ++++++++--- .../KinesisShardRecordProcessorFactory.java | 24 +- .../processor/MetadataKeyAttributes.java | 15 + .../KinesisLeaseConfigExtensionTest.java | 10 + .../KinesisLeaseConfigProviderTest.java | 10 + .../KinesisLeaseConfigSupplierTest.java | 10 + .../extension/KinesisLeaseConfigTest.java | 10 + .../kinesis/source/KinesisServiceTest.java | 20 +- .../kinesis/source/KinesisSourceTest.java | 15 + .../KinesisSourceConfigTest.java | 9 +- .../converter/KinesisRecordConverterTest.java | 96 ++++++ .../KinesisCheckpointerRecordTest.java | 64 ++++ .../KinesisCheckpointerTrackerTest.java | 98 ++++++ .../processor/KinesisRecordProcessorTest.java | 317 +++++++++++++----- ...inesisShardRecordProcessorFactoryTest.java | 10 +- .../pipeline_with_checkpoint_enabled.yaml | 3 - 22 files changed, 923 insertions(+), 179 deletions(-) create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverter.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerRecord.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTracker.java create mode 100644 data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/MetadataKeyAttributes.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverterTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerRecordTest.java create mode 100644 data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTrackerTest.java diff --git a/data-prepper-plugins/kinesis-source/build.gradle b/data-prepper-plugins/kinesis-source/build.gradle index ae380ba0e1..c4a0614e36 100644 --- a/data-prepper-plugins/kinesis-source/build.gradle +++ b/data-prepper-plugins/kinesis-source/build.gradle @@ -10,12 +10,13 @@ plugins { dependencies { implementation project(':data-prepper-api') implementation project(':data-prepper-plugins:common') + implementation project(path: ':data-prepper-plugins:buffer-common') + implementation project(path: ':data-prepper-plugins:aws-plugin-api') implementation 'com.fasterxml.jackson.core:jackson-core' implementation 'io.micrometer:micrometer-core' implementation 'software.amazon.kinesis:amazon-kinesis-client:2.6.0' compileOnly 'org.projectlombok:lombok:1.18.20' annotationProcessor 'org.projectlombok:lombok:1.18.20' - implementation project(path: ':data-prepper-plugins:aws-plugin-api') testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' @@ -25,6 +26,8 @@ dependencies { testImplementation project(':data-prepper-plugin-framework') testImplementation project(':data-prepper-pipeline-parser') testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + testImplementation project(':data-prepper-plugins:parse-json-processor') + testImplementation project(':data-prepper-plugins:newline-codecs') } jacocoTestCoverageVerification { diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java index 442f8e4fe6..4ed15833f6 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java @@ -14,8 +14,12 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfig; @@ -53,18 +57,18 @@ public class KinesisService { private final String kclMetricsNamespaceName; private final String pipelineName; private final AcknowledgementSetManager acknowledgementSetManager; - private final KinesisSourceConfig sourceConfig; + private final KinesisSourceConfig kinesisSourceConfig; private final KinesisAsyncClient kinesisClient; private final DynamoDbAsyncClient dynamoDbClient; private final CloudWatchAsyncClient cloudWatchClient; private final WorkerIdentifierGenerator workerIdentifierGenerator; + private final InputCodec codec; @Setter private Scheduler scheduler; - private final ExecutorService executorService; - public KinesisService(final KinesisSourceConfig sourceConfig, + public KinesisService(final KinesisSourceConfig kinesisSourceConfig, final KinesisClientFactory kinesisClientFactory, final PluginMetrics pluginMetrics, final PluginFactory pluginFactory, @@ -73,7 +77,7 @@ public KinesisService(final KinesisSourceConfig sourceConfig, final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier, final WorkerIdentifierGenerator workerIdentifierGenerator ){ - this.sourceConfig = sourceConfig; + this.kinesisSourceConfig = kinesisSourceConfig; this.pluginMetrics = pluginMetrics; this.pluginFactory = pluginFactory; this.acknowledgementSetManager = acknowledgementSetManager; @@ -85,21 +89,24 @@ public KinesisService(final KinesisSourceConfig sourceConfig, this.tableName = kinesisLeaseConfig.getLeaseCoordinationTable().getTableName(); this.kclMetricsNamespaceName = this.tableName; this.dynamoDbClient = kinesisClientFactory.buildDynamoDBClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); - this.kinesisClient = kinesisClientFactory.buildKinesisAsyncClient(sourceConfig.getAwsAuthenticationConfig().getAwsRegion()); + this.kinesisClient = kinesisClientFactory.buildKinesisAsyncClient(kinesisSourceConfig.getAwsAuthenticationConfig().getAwsRegion()); this.cloudWatchClient = kinesisClientFactory.buildCloudWatchAsyncClient(kinesisLeaseConfig.getLeaseCoordinationTable().getAwsRegion()); this.pipelineName = pipelineDescription.getPipelineName(); this.applicationName = pipelineName; this.workerIdentifierGenerator = workerIdentifierGenerator; this.executorService = Executors.newFixedThreadPool(1); + final PluginModel codecConfiguration = kinesisSourceConfig.getCodec(); + final PluginSetting codecPluginSettings = new PluginSetting(codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); + this.codec = pluginFactory.loadPlugin(InputCodec.class, codecPluginSettings); } public void start(final Buffer> buffer) { if (buffer == null) { - throw new IllegalStateException("Buffer provided is null"); + throw new IllegalStateException("Buffer provided is null."); } - if (sourceConfig.getStreams() == null || sourceConfig.getStreams().isEmpty()) { - throw new IllegalStateException("Streams are empty!"); + if (kinesisSourceConfig.getStreams() == null || kinesisSourceConfig.getStreams().isEmpty()) { + throw new InvalidPluginConfigurationException("No Kinesis streams provided."); } scheduler = getScheduler(buffer); @@ -129,31 +136,30 @@ public Scheduler getScheduler(final Buffer> buffer) { public Scheduler createScheduler(final Buffer> buffer) { final ShardRecordProcessorFactory processorFactory = new KinesisShardRecordProcessorFactory( - buffer, sourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory); + buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, codec); ConfigsBuilder configsBuilder = new ConfigsBuilder( - new KinesisMultiStreamTracker(kinesisClient, sourceConfig, applicationName), + new KinesisMultiStreamTracker(kinesisClient, kinesisSourceConfig, applicationName), applicationName, kinesisClient, dynamoDbClient, cloudWatchClient, workerIdentifierGenerator.generate(), processorFactory ) .tableName(tableName) .namespace(kclMetricsNamespaceName); - ConsumerStrategy consumerStrategy = sourceConfig.getConsumerStrategy(); + ConsumerStrategy consumerStrategy = kinesisSourceConfig.getConsumerStrategy(); if (consumerStrategy == ConsumerStrategy.POLLING) { configsBuilder.retrievalConfig().retrievalSpecificConfig( new PollingConfig(kinesisClient) - .maxRecords(sourceConfig.getPollingConfig().getMaxPollingRecords()) + .maxRecords(kinesisSourceConfig.getPollingConfig().getMaxPollingRecords()) .idleTimeBetweenReadsInMillis( - sourceConfig.getPollingConfig().getIdleTimeBetweenReads().toMillis())); + kinesisSourceConfig.getPollingConfig().getIdleTimeBetweenReads().toMillis())); } return new Scheduler( configsBuilder.checkpointConfig(), configsBuilder.coordinatorConfig(), - configsBuilder.leaseManagementConfig() - .billingMode(BillingMode.PAY_PER_REQUEST), + configsBuilder.leaseManagementConfig().billingMode(BillingMode.PAY_PER_REQUEST), configsBuilder.lifecycleConfig(), configsBuilder.metricsConfig(), configsBuilder.processorConfig(), diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java index b6e4480229..b26732e357 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java @@ -35,10 +35,6 @@ public class KinesisStreamConfig { @JsonProperty("checkpoint_interval") private Duration checkPointInterval = MINIMAL_CHECKPOINT_INTERVAL; - @Getter - @JsonProperty("enable_checkpoint") - private boolean enableCheckPoint = DEFAULT_ENABLE_CHECKPOINT; - public InitialPositionInStream getInitialPosition() { return initialPosition.getPositionInStream(); } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverter.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverter.java new file mode 100644 index 0000000000..5a70b95c10 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverter.java @@ -0,0 +1,47 @@ +/* + * 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.kinesis.source.converter; + +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.function.Consumer; + +public class KinesisRecordConverter { + + private final InputCodec codec; + + public KinesisRecordConverter(final InputCodec codec) { + this.codec = codec; + } + + public List> convert(List kinesisClientRecords) throws IOException { + List> records = new ArrayList<>(); + for (KinesisClientRecord record : kinesisClientRecords) { + processRecord(record, records::add); + } + return records; + } + + private void processRecord(KinesisClientRecord record, Consumer> eventConsumer) throws IOException { + // Read bytebuffer + byte[] arr = new byte[record.data().remaining()]; + record.data().get(arr); + ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(arr); + codec.parse(byteArrayInputStream, eventConsumer); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerRecord.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerRecord.java new file mode 100644 index 0000000000..b891de2bd0 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerRecord.java @@ -0,0 +1,26 @@ +/* + * 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.kinesis.source.processor; + +import lombok.Builder; +import lombok.Getter; +import lombok.Setter; +import software.amazon.kinesis.processor.RecordProcessorCheckpointer; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +@Builder +@Getter +@Setter +public class KinesisCheckpointerRecord { + private RecordProcessorCheckpointer checkpointer; + private ExtendedSequenceNumber extendedSequenceNumber; + private boolean readyToCheckpoint; +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTracker.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTracker.java new file mode 100644 index 0000000000..9f8e5f7625 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTracker.java @@ -0,0 +1,68 @@ +/* + * 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.kinesis.source.processor; + +import software.amazon.kinesis.processor.RecordProcessorCheckpointer; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +public class KinesisCheckpointerTracker { + private final Map checkpointerRecordList = new LinkedHashMap<>(); + + public synchronized void addRecordForCheckpoint(final ExtendedSequenceNumber extendedSequenceNumber, + final RecordProcessorCheckpointer checkpointer) { + checkpointerRecordList.put(extendedSequenceNumber, KinesisCheckpointerRecord.builder() + .extendedSequenceNumber(extendedSequenceNumber) + .checkpointer(checkpointer) + .readyToCheckpoint(false) + .build()); + } + + public synchronized void markSequenceNumberForCheckpoint(final ExtendedSequenceNumber extendedSequenceNumber) { + if (!checkpointerRecordList.containsKey(extendedSequenceNumber)) { + throw new IllegalArgumentException("checkpointer not available"); + } + checkpointerRecordList.get(extendedSequenceNumber).setReadyToCheckpoint(true); + } + + public synchronized Optional getLatestAvailableCheckpointRecord() { + Optional kinesisCheckpointerRecordOptional = Optional.empty(); + List toRemoveRecords = new ArrayList<>(); + + for (Map.Entry entry: checkpointerRecordList.entrySet()) { + KinesisCheckpointerRecord kinesisCheckpointerRecord = entry.getValue(); + + // Break out of the loop on the first record which is not ready for checkpoint + if (!kinesisCheckpointerRecord.isReadyToCheckpoint()) { + break; + } + + kinesisCheckpointerRecordOptional = Optional.of(kinesisCheckpointerRecord); + toRemoveRecords.add(entry.getKey()); + } + + //Cleanup the ones which are already marked for checkpoint + for (ExtendedSequenceNumber extendedSequenceNumber: toRemoveRecords) { + checkpointerRecordList.remove(extendedSequenceNumber); + } + + return kinesisCheckpointerRecordOptional; + } + + public synchronized int size() { + return checkpointerRecordList.size(); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java index 956ba88f7a..7d02ff959a 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java @@ -1,19 +1,28 @@ +/* + * 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.kinesis.source.processor; +import com.google.common.annotations.VisibleForTesting; import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.common.concurrent.BackgroundThreadFactory; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; -import org.opensearch.dataprepper.model.buffer.Buffer; -import org.opensearch.dataprepper.model.codec.InputCodec; -import org.opensearch.dataprepper.model.configuration.PluginModel; -import org.opensearch.dataprepper.model.configuration.PluginSetting; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.event.EventMetadata; import org.opensearch.dataprepper.model.record.Record; -import org.opensearch.dataprepper.plugins.kinesis.source.KinesisSource; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.converter.KinesisRecordConverter; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.kinesis.common.StreamIdentifier; @@ -28,55 +37,67 @@ import software.amazon.kinesis.processor.RecordProcessorCheckpointer; import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.retrieval.KinesisClientRecord; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; -import java.io.ByteArrayInputStream; -import java.io.IOException; import java.time.Duration; -import java.util.ArrayList; import java.util.List; +import java.util.ListIterator; import java.util.Optional; -import java.util.function.Consumer; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicBoolean; public class KinesisRecordProcessor implements ShardRecordProcessor { - private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class); + private static final Logger LOG = LoggerFactory.getLogger(KinesisRecordProcessor.class); + private static final int DEFAULT_MONITOR_WAIT_TIME_MS = 15_000; private final StreamIdentifier streamIdentifier; private final KinesisStreamConfig kinesisStreamConfig; private final Duration checkpointInterval; private final KinesisSourceConfig kinesisSourceConfig; - private final Buffer> buffer; + private final BufferAccumulator> bufferAccumulator; + private final KinesisRecordConverter kinesisRecordConverter; private String kinesisShardId; - private final InputCodec codec; private long lastCheckpointTimeInMillis; private final int bufferTimeoutMillis; private final AcknowledgementSetManager acknowledgementSetManager; - private final Counter acknowledgementSetCallbackCounter; + private final Counter acknowledgementSetSuccesses; + private final Counter acknowledgementSetFailures; + private final Counter recordsProcessed; private final Counter recordProcessingErrors; private final Counter checkpointFailures; private static final Duration ACKNOWLEDGEMENT_SET_TIMEOUT = Duration.ofSeconds(20); - public static final String ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME = "acknowledgementSetCallbackCounter"; + public static final String ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME = "acknowledgementSetSuccesses"; + public static final String ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME = "acknowledgementSetFailures"; + public static final String KINESIS_RECORD_PROCESSED = "recordProcessed"; public static final String KINESIS_RECORD_PROCESSING_ERRORS = "recordProcessingErrors"; public static final String KINESIS_CHECKPOINT_FAILURES = "checkpointFailures"; public static final String KINESIS_STREAM_TAG_KEY = "stream"; + private KinesisCheckpointerTracker kinesisCheckpointerTracker; + private final ExecutorService executorService; + private AtomicBoolean isStopRequested; - public KinesisRecordProcessor(Buffer> buffer, + public KinesisRecordProcessor(final BufferAccumulator> bufferAccumulator, final KinesisSourceConfig kinesisSourceConfig, final AcknowledgementSetManager acknowledgementSetManager, final PluginMetrics pluginMetrics, - final PluginFactory pluginFactory, + final KinesisRecordConverter kinesisRecordConverter, final StreamIdentifier streamIdentifier) { this.bufferTimeoutMillis = (int) kinesisSourceConfig.getBufferTimeout().toMillis(); this.streamIdentifier = streamIdentifier; this.kinesisSourceConfig = kinesisSourceConfig; this.kinesisStreamConfig = getStreamConfig(kinesisSourceConfig); - final PluginModel codecConfiguration = kinesisSourceConfig.getCodec(); - final PluginSetting codecPluginSettings = new PluginSetting(codecConfiguration.getPluginName(), codecConfiguration.getPluginSettings()); - this.codec = pluginFactory.loadPlugin(InputCodec.class, codecPluginSettings); + this.kinesisRecordConverter = kinesisRecordConverter; this.acknowledgementSetManager = acknowledgementSetManager; - this.acknowledgementSetCallbackCounter = pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); + this.acknowledgementSetSuccesses = pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); + this.acknowledgementSetFailures = pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); + this.recordsProcessed = pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSED, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.recordProcessingErrors = pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.checkpointFailures = pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.checkpointInterval = kinesisStreamConfig.getCheckPointInterval(); - this.buffer = buffer; + this.bufferAccumulator = bufferAccumulator; + this.kinesisCheckpointerTracker = new KinesisCheckpointerTracker(); + this.executorService = Executors.newSingleThreadExecutor(BackgroundThreadFactory.defaultExecutorThreadFactory("kinesis-ack-monitor")); + this.isStopRequested = new AtomicBoolean(false); } private KinesisStreamConfig getStreamConfig(final KinesisSourceConfig kinesisSourceConfig) { @@ -87,49 +108,101 @@ private KinesisStreamConfig getStreamConfig(final KinesisSourceConfig kinesisSou public void initialize(InitializationInput initializationInput) { // Called once when the processor is initialized. kinesisShardId = initializationInput.shardId(); - LOG.info("Initialize Processor for shard: {}", kinesisShardId); + String kinesisStreamName = streamIdentifier.streamName(); + LOG.info("Initialize Processor for stream: {}, shard: {}", kinesisStreamName, kinesisShardId); lastCheckpointTimeInMillis = System.currentTimeMillis(); + + if (kinesisSourceConfig.isAcknowledgments()) { + executorService.submit(() -> monitorCheckpoint(executorService)); + } } - private AcknowledgementSet createAcknowledgmentSet(final ProcessRecordsInput processRecordsInput) { + private void monitorCheckpoint(final ExecutorService executorService) { + while (!isStopRequested.get()) { + if (System.currentTimeMillis() - lastCheckpointTimeInMillis >= checkpointInterval.toMillis()) { + LOG.debug("Regular checkpointing for shard {}", kinesisShardId); + + Optional kinesisCheckpointerRecordOptional = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + if (kinesisCheckpointerRecordOptional.isPresent()) { + RecordProcessorCheckpointer recordProcessorCheckpointer = kinesisCheckpointerRecordOptional.get().getCheckpointer(); + String sequenceNumber = kinesisCheckpointerRecordOptional.get().getExtendedSequenceNumber().sequenceNumber(); + Long subSequenceNumber = kinesisCheckpointerRecordOptional.get().getExtendedSequenceNumber().subSequenceNumber(); + checkpoint(recordProcessorCheckpointer, sequenceNumber, subSequenceNumber); + lastCheckpointTimeInMillis = System.currentTimeMillis(); + } + } + try { + Thread.sleep(DEFAULT_MONITOR_WAIT_TIME_MS); + } catch (InterruptedException ex) { + break; + } + } + executorService.shutdown(); + } + + private AcknowledgementSet createAcknowledgmentSet(final ProcessRecordsInput processRecordsInput, + final ExtendedSequenceNumber extendedSequenceNumber) { return acknowledgementSetManager.create((result) -> { - acknowledgementSetCallbackCounter.increment(); + String kinesisStreamName = streamIdentifier.streamName(); if (result) { - LOG.info("acknowledgements received"); - checkpoint(processRecordsInput.checkpointer()); + acknowledgementSetSuccesses.increment(); + kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(extendedSequenceNumber); + LOG.debug("acknowledgements received for stream: {}, shardId: {}", kinesisStreamName, kinesisShardId); } else { - LOG.info("acknowledgements received with false"); + acknowledgementSetFailures.increment(); + LOG.debug("acknowledgements received with false for stream: {}, shardId: {}", kinesisStreamName, kinesisShardId); } - }, ACKNOWLEDGEMENT_SET_TIMEOUT); } @Override public void processRecords(ProcessRecordsInput processRecordsInput) { - List> records = new ArrayList<>(); - try { Optional acknowledgementSetOpt = Optional.empty(); boolean acknowledgementsEnabled = kinesisSourceConfig.isAcknowledgments(); + ExtendedSequenceNumber extendedSequenceNumber = getLatestSequenceNumberFromInput(processRecordsInput); if (acknowledgementsEnabled) { - acknowledgementSetOpt = Optional.of(createAcknowledgmentSet(processRecordsInput)); + acknowledgementSetOpt = Optional.of(createAcknowledgmentSet(processRecordsInput, extendedSequenceNumber)); } - for (KinesisClientRecord record : processRecordsInput.records()) { - processRecord(record, records::add); + // Track the records for checkpoint purpose + kinesisCheckpointerTracker.addRecordForCheckpoint(extendedSequenceNumber, processRecordsInput.checkpointer()); + List> records = kinesisRecordConverter.convert(processRecordsInput.records()); + + int eventCount = 0; + for (Record record: records) { + Event event = record.getData(); + acknowledgementSetOpt.ifPresent(acknowledgementSet -> acknowledgementSet.add(event)); + EventMetadata eventMetadata = event.getMetadata(); + eventMetadata.setAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE, + streamIdentifier.streamName().toLowerCase()); + bufferAccumulator.add(record); + eventCount++; } - acknowledgementSetOpt.ifPresent(acknowledgementSet -> records.forEach(record -> acknowledgementSet.add(record.getData()))); + // Flush buffer at the end + bufferAccumulator.flush(); + recordsProcessed.increment(eventCount); - buffer.writeAll(records, bufferTimeoutMillis); + // If acks are not enabled, mark the sequence number for checkpoint + if (!acknowledgementsEnabled) { + kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(extendedSequenceNumber); + } + + LOG.debug("Number of Records {} written for stream: {}, shardId: {} to buffer: {}", eventCount, streamIdentifier.streamName(), kinesisShardId, records.size()); acknowledgementSetOpt.ifPresent(AcknowledgementSet::complete); // Checkpoint for shard - if (kinesisStreamConfig.isEnableCheckPoint() && System.currentTimeMillis() - lastCheckpointTimeInMillis > checkpointInterval.toMillis()) { - LOG.info("Regular checkpointing for shard " + kinesisShardId); - checkpoint(processRecordsInput.checkpointer()); - lastCheckpointTimeInMillis = System.currentTimeMillis(); + if (!acknowledgementsEnabled && (System.currentTimeMillis() - lastCheckpointTimeInMillis >= checkpointInterval.toMillis())) { + LOG.debug("Regular checkpointing for shard {}", kinesisShardId); + + Optional KinesisCheckpointerRecordOptional = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + if (KinesisCheckpointerRecordOptional.isPresent()) { + ExtendedSequenceNumber lastExtendedSequenceNumber = KinesisCheckpointerRecordOptional.get().getExtendedSequenceNumber(); + checkpoint(processRecordsInput.checkpointer(), lastExtendedSequenceNumber.sequenceNumber(), lastExtendedSequenceNumber.subSequenceNumber()); + lastCheckpointTimeInMillis = System.currentTimeMillis(); + } } } catch (Exception ex) { recordProcessingErrors.increment(); @@ -137,14 +210,6 @@ public void processRecords(ProcessRecordsInput processRecordsInput) { } } - private void processRecord(KinesisClientRecord record, Consumer> eventConsumer) throws IOException { - // Read bytebuffer - byte[] arr = new byte[record.data().remaining()]; - record.data().get(arr); - ByteArrayInputStream byteArrayInputStream = new ByteArrayInputStream(arr); - codec.parse(byteArrayInputStream, eventConsumer); - } - @Override public void leaseLost(LeaseLostInput leaseLostInput) { LOG.debug("Lease Lost"); @@ -152,22 +217,60 @@ public void leaseLost(LeaseLostInput leaseLostInput) { @Override public void shardEnded(ShardEndedInput shardEndedInput) { - LOG.debug("Reached shard end, checkpointing shard: {}", kinesisShardId); + String kinesisStream = streamIdentifier.streamName(); + LOG.debug("Reached shard end, checkpointing for stream: {}, shardId: {}", kinesisStream, kinesisShardId); checkpoint(shardEndedInput.checkpointer()); } @Override public void shutdownRequested(ShutdownRequestedInput shutdownRequestedInput) { - LOG.info("Scheduler is shutting down, checkpointing shard: {}", kinesisShardId); + String kinesisStream = streamIdentifier.streamName(); + isStopRequested.set(true); + LOG.debug("Scheduler is shutting down, checkpointing for stream: {}, shardId: {}", kinesisStream, kinesisShardId); checkpoint(shutdownRequestedInput.checkpointer()); } + @VisibleForTesting + public void checkpoint(RecordProcessorCheckpointer checkpointer, String sequenceNumber, long subSequenceNumber) { + try { + String kinesisStream = streamIdentifier.streamName(); + LOG.debug("Checkpoint for stream: {}, shardId: {}, sequence: {}, subsequence: {}", kinesisStream, kinesisShardId, sequenceNumber, subSequenceNumber); + checkpointer.checkpoint(sequenceNumber, subSequenceNumber); + } catch (ShutdownException | ThrottlingException | InvalidStateException ex) { + LOG.debug("Caught exception at checkpoint, skipping checkpoint.", ex); + checkpointFailures.increment(); + } + } + private void checkpoint(RecordProcessorCheckpointer checkpointer) { try { + String kinesisStream = streamIdentifier.streamName(); + LOG.debug("Checkpoint for stream: {}, shardId: {}", kinesisStream, kinesisShardId); checkpointer.checkpoint(); } catch (ShutdownException | ThrottlingException | InvalidStateException ex) { - LOG.info("Caught exception at checkpoint, skipping checkpoint.", ex); + LOG.debug("Caught exception at checkpoint, skipping checkpoint.", ex); checkpointFailures.increment(); } } + + private ExtendedSequenceNumber getLatestSequenceNumberFromInput(final ProcessRecordsInput processRecordsInput) { + ListIterator recordIterator = processRecordsInput.records().listIterator(); + ExtendedSequenceNumber largestExtendedSequenceNumber = null; + while (recordIterator.hasNext()) { + KinesisClientRecord record = recordIterator.next(); + ExtendedSequenceNumber extendedSequenceNumber = + new ExtendedSequenceNumber(record.sequenceNumber(), record.subSequenceNumber()); + + if (largestExtendedSequenceNumber == null + || largestExtendedSequenceNumber.compareTo(extendedSequenceNumber) < 0) { + largestExtendedSequenceNumber = extendedSequenceNumber; + } + } + return largestExtendedSequenceNumber; + } + + @VisibleForTesting + public void setKinesisCheckpointerTracker(final KinesisCheckpointerTracker kinesisCheckpointerTracker) { + this.kinesisCheckpointerTracker = kinesisCheckpointerTracker; + } } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java index f551c503e5..e326789312 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java @@ -1,12 +1,24 @@ +/* + * 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.kinesis.source.processor; +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.event.Event; -import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.converter.KinesisRecordConverter; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.processor.ShardRecordProcessor; import software.amazon.kinesis.processor.ShardRecordProcessorFactory; @@ -17,18 +29,18 @@ public class KinesisShardRecordProcessorFactory implements ShardRecordProcessorF private final KinesisSourceConfig kinesisSourceConfig; private final AcknowledgementSetManager acknowledgementSetManager; private final PluginMetrics pluginMetrics; - private final PluginFactory pluginFactory; + private final KinesisRecordConverter kinesisRecordConverter; public KinesisShardRecordProcessorFactory(Buffer> buffer, KinesisSourceConfig kinesisSourceConfig, final AcknowledgementSetManager acknowledgementSetManager, final PluginMetrics pluginMetrics, - final PluginFactory pluginFactory) { + final InputCodec codec) { this.kinesisSourceConfig = kinesisSourceConfig; this.buffer = buffer; this.acknowledgementSetManager = acknowledgementSetManager; this.pluginMetrics = pluginMetrics; - this.pluginFactory = pluginFactory; + this.kinesisRecordConverter = new KinesisRecordConverter(codec); } @Override @@ -38,6 +50,8 @@ public ShardRecordProcessor shardRecordProcessor() { @Override public ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) { - return new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + BufferAccumulator> bufferAccumulator = BufferAccumulator.create(buffer, + kinesisSourceConfig.getNumberOfRecordsToAccumulate(), kinesisSourceConfig.getBufferTimeout()); + return new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); } } \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/MetadataKeyAttributes.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/MetadataKeyAttributes.java new file mode 100644 index 0000000000..e2debba54e --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/MetadataKeyAttributes.java @@ -0,0 +1,15 @@ +/* + * 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.kinesis.source.processor; + +public class MetadataKeyAttributes { + static final String KINESIS_STREAM_NAME_METADATA_ATTRIBUTE = "kinesis_stream_name"; +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtensionTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtensionTest.java index 4476d28def..852baab195 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtensionTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtensionTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.extension; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProviderTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProviderTest.java index 50bc02d628..1fa17f5f42 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProviderTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProviderTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.extension; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplierTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplierTest.java index c3eeb28d40..4cfc323ed5 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplierTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplierTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.extension; import org.junit.jupiter.api.Test; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java index 582bf68f69..30194a9659 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java @@ -1,3 +1,13 @@ +/* + * 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.kinesis.extension; import com.fasterxml.jackson.databind.ObjectMapper; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java index 49f0c898a0..12986d9969 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java @@ -16,8 +16,11 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfig; @@ -43,6 +46,7 @@ import java.time.Duration; import java.time.Instant; import java.util.ArrayList; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -55,6 +59,8 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -63,6 +69,7 @@ public class KinesisServiceTest { private final String PIPELINE_NAME = "kinesis-pipeline-test"; private final String streamId = "stream-1"; + private static final String codec_plugin_name = "json"; private static final Duration CHECKPOINT_INTERVAL = Duration.ofMillis(0); private static final int NUMBER_OF_RECORDS_TO_ACCUMULATE = 10; @@ -183,6 +190,15 @@ void setup() { when(kinesisSourceConfig.getStreams()).thenReturn(streamConfigs); when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); + PluginModel pluginModel = mock(PluginModel.class); + when(pluginModel.getPluginName()).thenReturn(codec_plugin_name); + when(pluginModel.getPluginSettings()).thenReturn(Collections.emptyMap()); + when(kinesisSourceConfig.getCodec()).thenReturn(pluginModel); + + pluginFactory = mock(PluginFactory.class); + InputCodec codec = mock(InputCodec.class); + when(pluginFactory.loadPlugin(eq(InputCodec.class), any())).thenReturn(codec); + when(kinesisClientFactory.buildDynamoDBClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(dynamoDbClient); when(kinesisClientFactory.buildKinesisAsyncClient(awsAuthenticationConfig.getAwsRegion())).thenReturn(kinesisClient); when(kinesisClientFactory.buildCloudWatchAsyncClient(kinesisLeaseCoordinationTableConfig.getAwsRegion())).thenReturn(cloudWatchClient); @@ -263,7 +279,7 @@ void testServiceStartNullStreams() { when(kinesisSourceConfig.getStreams()).thenReturn(null); KinesisService kinesisService = createObjectUnderTest(); - assertThrows(IllegalStateException.class, () -> kinesisService.start(buffer)); + assertThrows(InvalidPluginConfigurationException.class, () -> kinesisService.start(buffer)); verify(scheduler, times(0)).run(); } @@ -273,7 +289,7 @@ void testServiceStartEmptyStreams() { when(kinesisSourceConfig.getStreams()).thenReturn(new ArrayList<>()); KinesisService kinesisService = createObjectUnderTest(); - assertThrows(IllegalStateException.class, () -> kinesisService.start(buffer)); + assertThrows(InvalidPluginConfigurationException.class, () -> kinesisService.start(buffer)); verify(scheduler, times(0)).run(); } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java index e65ce1dc1a..fad335dd63 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java @@ -18,7 +18,9 @@ import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.configuration.PipelineDescription; +import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.event.Event; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; @@ -31,6 +33,7 @@ import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.regions.Region; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -40,6 +43,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -48,6 +52,7 @@ public class KinesisSourceTest { private final String PIPELINE_NAME = "kinesis-pipeline-test"; private final String streamId = "stream-1"; + private static final String codec_plugin_name = "json"; @Mock private PluginMetrics pluginMetrics; @@ -93,6 +98,16 @@ void setup() { awsAuthenticationConfig = mock(AwsAuthenticationConfig.class); acknowledgementSetManager = mock(AcknowledgementSetManager.class); kinesisService = mock(KinesisService.class); + + PluginModel pluginModel = mock(PluginModel.class); + when(pluginModel.getPluginName()).thenReturn(codec_plugin_name); + when(pluginModel.getPluginSettings()).thenReturn(Collections.emptyMap()); + when(kinesisSourceConfig.getCodec()).thenReturn(pluginModel); + + pluginFactory = mock(PluginFactory.class); + InputCodec codec = mock(InputCodec.class); + when(pluginFactory.loadPlugin(eq(InputCodec.class), any())).thenReturn(codec); + kinesisLeaseConfigSupplier = mock(KinesisLeaseConfigSupplier.class); kinesisLeaseConfig = mock(KinesisLeaseConfig.class); kinesisLeaseCoordinationTableConfig = mock(KinesisLeaseCoordinationTableConfig.class); diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java index 166d5c587e..5846fe4b04 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java @@ -88,9 +88,10 @@ void testSourceConfig() { assertEquals(kinesisSourceConfig.getConsumerStrategy(), ConsumerStrategy.ENHANCED_FAN_OUT); assertNull(kinesisSourceConfig.getPollingConfig()); + assertEquals(streamConfigs.size(), 3); + for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { assertTrue(kinesisStreamConfig.getName().contains("stream")); - assertFalse(kinesisStreamConfig.isEnableCheckPoint()); assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.LATEST); assertEquals(kinesisStreamConfig.getCheckPointInterval(), MINIMAL_CHECKPOINT_INTERVAL); } @@ -117,9 +118,10 @@ void testSourceConfigWithStreamCodec() { assertEquals(kinesisSourceConfig.getPollingConfig().getMaxPollingRecords(), 10); assertEquals(kinesisSourceConfig.getPollingConfig().getIdleTimeBetweenReads(), Duration.ofSeconds(10)); + assertEquals(streamConfigs.size(), 3); + for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { assertTrue(kinesisStreamConfig.getName().contains("stream")); - assertFalse(kinesisStreamConfig.isEnableCheckPoint()); assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.LATEST); assertEquals(kinesisStreamConfig.getCheckPointInterval(), MINIMAL_CHECKPOINT_INTERVAL); } @@ -148,9 +150,10 @@ void testSourceConfigWithInitialPosition() { expectedCheckpointIntervals.put("stream-2", Duration.ofMinutes(15)); expectedCheckpointIntervals.put("stream-3", Duration.ofHours(2)); + assertEquals(streamConfigs.size(), 3); + for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { assertTrue(kinesisStreamConfig.getName().contains("stream")); - assertTrue(kinesisStreamConfig.isEnableCheckPoint()); assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.TRIM_HORIZON); assertEquals(kinesisStreamConfig.getCheckPointInterval(), expectedCheckpointIntervals.get(kinesisStreamConfig.getName())); } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverterTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverterTest.java new file mode 100644 index 0000000000..6b0646e993 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/converter/KinesisRecordConverterTest.java @@ -0,0 +1,96 @@ +/* + * 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.kinesis.source.converter; + +import com.fasterxml.jackson.databind.ObjectMapper; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.event.TestEventFactory; +import org.opensearch.dataprepper.model.codec.InputCodec; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.codec.json.NdjsonInputCodec; +import org.opensearch.dataprepper.plugins.codec.json.NdjsonInputConfig; +import software.amazon.kinesis.retrieval.KinesisClientRecord; + +import java.io.IOException; +import java.io.InputStream; +import java.io.StringWriter; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.IntStream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +public class KinesisRecordConverterTest { + + @Test + void setup() throws IOException { + InputCodec codec = mock(InputCodec.class); + KinesisRecordConverter kinesisRecordConverter = new KinesisRecordConverter(codec); + doNothing().when(codec).parse(any(InputStream.class), any(Consumer.class)); + + String sample_record_data = "sample record data"; + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(sample_record_data.getBytes())) + .build(); + kinesisRecordConverter.convert(List.of(kinesisClientRecord)); + verify(codec, times(1)).parse(any(InputStream.class), any(Consumer.class)); + } + + @Test + public void testRecordConverterWithNdJsonInputCodec() throws IOException { + + ObjectMapper objectMapper = new ObjectMapper(); + + int numRecords = 10; + final List> jsonObjects = IntStream.range(0, numRecords) + .mapToObj(i -> generateJson()) + .collect(Collectors.toList()); + + final StringWriter writer = new StringWriter(); + + for (final Map jsonObject : jsonObjects) { + writer.append(objectMapper.writeValueAsString(jsonObject)); + writer.append(System.lineSeparator()); + } + + KinesisRecordConverter kinesisRecordConverter = new KinesisRecordConverter( + new NdjsonInputCodec(new NdjsonInputConfig(), TestEventFactory.getTestEventFactory())); + + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(writer.toString().getBytes())) + .build(); + List> events = kinesisRecordConverter.convert(List.of(kinesisClientRecord)); + + assertEquals(events.size(), numRecords); + } + + private static Map generateJson() { + final Map jsonObject = new LinkedHashMap<>(); + for (int i = 0; i < 1; i++) { + jsonObject.put(UUID.randomUUID().toString(), UUID.randomUUID().toString()); + } + jsonObject.put(UUID.randomUUID().toString(), Arrays.asList(UUID.randomUUID().toString(), UUID.randomUUID().toString(), UUID.randomUUID().toString())); + + return jsonObject; + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerRecordTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerRecordTest.java new file mode 100644 index 0000000000..a2cf8fecaf --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerRecordTest.java @@ -0,0 +1,64 @@ +/* + * 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.kinesis.source.processor; + +import org.junit.jupiter.api.Test; +import software.amazon.kinesis.processor.RecordProcessorCheckpointer; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.mock; + +public class KinesisCheckpointerRecordTest { + private String shardId = "shardId-123"; + private String testConcurrencyToken = "testToken"; + + @Test + public void validateTwoRecords() { + + KinesisCheckpointerRecord kinesisCheckpointerRecord1 = KinesisCheckpointerRecord.builder() + .extendedSequenceNumber(ExtendedSequenceNumber.LATEST) + .readyToCheckpoint(false) + .build(); + KinesisCheckpointerRecord kinesisCheckpointerRecord2 = KinesisCheckpointerRecord.builder() + .extendedSequenceNumber(ExtendedSequenceNumber.LATEST) + .readyToCheckpoint(false) + .build(); + + assertEquals(kinesisCheckpointerRecord1.isReadyToCheckpoint(), kinesisCheckpointerRecord2.isReadyToCheckpoint()); + assertEquals(kinesisCheckpointerRecord1.getCheckpointer(), kinesisCheckpointerRecord2.getCheckpointer()); + assertEquals(kinesisCheckpointerRecord1.getExtendedSequenceNumber(), kinesisCheckpointerRecord2.getExtendedSequenceNumber()); + } + + @Test + public void validateTwoRecordsWithSetterMethods() { + RecordProcessorCheckpointer recordProcessorCheckpointer = mock(RecordProcessorCheckpointer.class); + KinesisCheckpointerRecord kinesisCheckpointerRecord1 = KinesisCheckpointerRecord.builder().build(); + kinesisCheckpointerRecord1.setCheckpointer(recordProcessorCheckpointer); + kinesisCheckpointerRecord1.setExtendedSequenceNumber(ExtendedSequenceNumber.LATEST); + kinesisCheckpointerRecord1.setReadyToCheckpoint(false); + + KinesisCheckpointerRecord kinesisCheckpointerRecord2 = KinesisCheckpointerRecord.builder().build(); + kinesisCheckpointerRecord2.setCheckpointer(recordProcessorCheckpointer); + kinesisCheckpointerRecord2.setExtendedSequenceNumber(ExtendedSequenceNumber.LATEST); + kinesisCheckpointerRecord2.setReadyToCheckpoint(false); + + assertEquals(kinesisCheckpointerRecord1.isReadyToCheckpoint(), kinesisCheckpointerRecord2.isReadyToCheckpoint()); + assertEquals(kinesisCheckpointerRecord1.getCheckpointer(), kinesisCheckpointerRecord2.getCheckpointer()); + assertEquals(kinesisCheckpointerRecord1.getExtendedSequenceNumber(), kinesisCheckpointerRecord2.getExtendedSequenceNumber()); + } + + @Test + public void testInvalidRecords() { + KinesisCheckpointerRecord kinesisCheckpointerRecord = KinesisCheckpointerRecord.builder().build(); + assertNotNull(kinesisCheckpointerRecord); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTrackerTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTrackerTest.java new file mode 100644 index 0000000000..ea76d1f789 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTrackerTest.java @@ -0,0 +1,98 @@ +/* + * 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.kinesis.source.processor; + +import org.junit.jupiter.api.Test; +import software.amazon.kinesis.processor.RecordProcessorCheckpointer; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.Random; + +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.mockito.Mockito.mock; + +public class KinesisCheckpointerTrackerTest { + + private Random random = new Random(); + + @Test + void testCheckPointerAddAndGet() { + KinesisCheckpointerTracker kinesisCheckpointerTracker = new KinesisCheckpointerTracker(); + + List extendedSequenceNumberList = new ArrayList<>(); + int numRecords = 10; + for (int i=0; i checkpointRecord = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + assertTrue(checkpointRecord.isEmpty()); + assertEquals(kinesisCheckpointerTracker.size(), numRecords); + + int idx = random.nextInt(numRecords); + ExtendedSequenceNumber extendedSequenceNumber1 = extendedSequenceNumberList.get(idx); + kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(extendedSequenceNumber1); + + Optional firstcheckpointer = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + if (idx != 0) { + assertTrue(firstcheckpointer.isEmpty()); + assertEquals(kinesisCheckpointerTracker.size(), numRecords); + } else { + assertFalse(firstcheckpointer.isEmpty()); + assertEquals(kinesisCheckpointerTracker.size(), numRecords-1); + } + } + @Test + void testGetLastCheckpointerAndStoreIsEmpty() { + KinesisCheckpointerTracker kinesisCheckpointerTracker = new KinesisCheckpointerTracker(); + + List extendedSequenceNumberList = new ArrayList<>(); + int numRecords = 10; + for (int i=0; i checkpointer = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + assertTrue(checkpointer.isPresent()); + assertEquals(0, kinesisCheckpointerTracker.size()); + } + + @Test + public void testMarkCheckpointerReadyForCheckpoint() { + + KinesisCheckpointerTracker kinesisCheckpointerTracker = new KinesisCheckpointerTracker(); + + ExtendedSequenceNumber extendedSequenceNumber = mock(ExtendedSequenceNumber.class); + assertThrows(IllegalArgumentException.class, () -> kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(extendedSequenceNumber)); + + Optional checkpointer = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + assertTrue(checkpointer.isEmpty()); + } +} diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java index 87ae19031a..902a70bfa6 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java @@ -15,20 +15,23 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.ArgumentCaptor; import org.mockito.Mock; import org.mockito.MockitoAnnotations; +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; import org.opensearch.dataprepper.metrics.PluginMetrics; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; -import org.opensearch.dataprepper.model.buffer.Buffer; import org.opensearch.dataprepper.model.codec.InputCodec; import org.opensearch.dataprepper.model.configuration.PluginModel; import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; import org.opensearch.dataprepper.model.event.JacksonEvent; import org.opensearch.dataprepper.model.plugin.PluginFactory; import org.opensearch.dataprepper.model.record.Record; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.converter.KinesisRecordConverter; import software.amazon.kinesis.common.StreamIdentifier; import software.amazon.kinesis.exceptions.InvalidStateException; import software.amazon.kinesis.exceptions.ShutdownException; @@ -39,21 +42,24 @@ import software.amazon.kinesis.lifecycle.events.ShutdownRequestedInput; import software.amazon.kinesis.processor.RecordProcessorCheckpointer; import software.amazon.kinesis.retrieval.KinesisClientRecord; +import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; import java.nio.ByteBuffer; import java.time.Duration; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.anyCollection; -import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyDouble; import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.atLeastOnce; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; @@ -61,8 +67,10 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoInteractions; import static org.mockito.Mockito.when; -import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_CHECKPOINT_FAILURES; +import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSED; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_RECORD_PROCESSING_ERRORS; import static org.opensearch.dataprepper.plugins.kinesis.source.processor.KinesisRecordProcessor.KINESIS_STREAM_TAG_KEY; @@ -71,6 +79,8 @@ public class KinesisRecordProcessorTest { private static final String shardId = "123"; private static final String streamId = "stream-1"; private static final String codec_plugin_name = "json"; + private static final String sequence_number = "10001"; + private static final Long sub_sequence_number = 1L; private static final Duration CHECKPOINT_INTERVAL = Duration.ofMillis(1000); private static final int NUMBER_OF_RECORDS_TO_ACCUMULATE = 10; @@ -96,9 +106,6 @@ public class KinesisRecordProcessorTest { @Mock private RecordProcessorCheckpointer checkpointer; - @Mock - private Buffer> buffer; - @Mock StreamIdentifier streamIdentifier; @@ -108,6 +115,9 @@ public class KinesisRecordProcessorTest { @Mock private AcknowledgementSet acknowledgementSet; + @Mock + private Counter recordProcessed; + @Mock private Counter recordProcessingErrors; @@ -115,7 +125,22 @@ public class KinesisRecordProcessorTest { private Counter checkpointFailures; @Mock - private Counter acknowledgementSetCallbackCounter; + private Counter acknowledgementSetSuccesses; + + @Mock + private Counter acknowledgementSetFailures; + + @Mock + private InputCodec codec; + + @Mock + private BufferAccumulator> bufferAccumulator; + + @Mock + private KinesisRecordConverter kinesisRecordConverter; + + @Mock + private KinesisCheckpointerTracker kinesisCheckpointerTracker; @BeforeEach public void setup() { @@ -123,6 +148,9 @@ public void setup() { pluginMetrics = mock(PluginMetrics.class); pluginFactory = mock(PluginFactory.class); acknowledgementSet = mock(AcknowledgementSet.class); + bufferAccumulator = mock(BufferAccumulator.class); + kinesisRecordConverter = mock(KinesisRecordConverter.class); + kinesisCheckpointerTracker = mock(KinesisCheckpointerTracker.class); when(initializationInput.shardId()).thenReturn(shardId); when(streamIdentifier.streamName()).thenReturn(streamId); @@ -132,73 +160,116 @@ public void setup() { when(pluginModel.getPluginSettings()).thenReturn(Collections.emptyMap()); when(kinesisSourceConfig.getCodec()).thenReturn(pluginModel); - InputCodec codec = mock(InputCodec.class); + codec = mock(InputCodec.class); when(pluginFactory.loadPlugin(eq(InputCodec.class), any())).thenReturn(codec); when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(CHECKPOINT_INTERVAL); when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(NUMBER_OF_RECORDS_TO_ACCUMULATE); when(kinesisSourceConfig.getStreams()).thenReturn(List.of(kinesisStreamConfig)); when(processRecordsInput.checkpointer()).thenReturn(checkpointer); - when(pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_CALLBACK_METRIC_NAME, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(acknowledgementSetCallbackCounter); + when(pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME, KINESIS_STREAM_TAG_KEY, + streamIdentifier.streamName())).thenReturn(acknowledgementSetSuccesses); + when(pluginMetrics.counterWithTags(ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME, KINESIS_STREAM_TAG_KEY, + streamIdentifier.streamName())).thenReturn(acknowledgementSetFailures); + + recordProcessed = mock(Counter.class); + when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSED, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(recordProcessed); + + recordProcessingErrors = mock(Counter.class); + when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(recordProcessingErrors); } @Test - void testProcessRecordsWithoutAcknowledgementsCheckpointsEnabled() + void testProcessRecordsWithoutAcknowledgementsWithCheckpointApplied() throws Exception { - when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); - when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(true); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); - kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + List> records = new ArrayList<>(); + Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + Record record = new Record<>(event); + records.add(record); + when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); + + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + KinesisCheckpointerRecord kinesisCheckpointerRecord = mock(KinesisCheckpointerRecord.class); + ExtendedSequenceNumber extendedSequenceNumber = mock(ExtendedSequenceNumber.class); + when(extendedSequenceNumber.sequenceNumber()).thenReturn(sequence_number); + when(extendedSequenceNumber.subSequenceNumber()).thenReturn(sub_sequence_number); + when(kinesisCheckpointerRecord.getExtendedSequenceNumber()).thenReturn(extendedSequenceNumber); + when(kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); + kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); kinesisRecordProcessor.initialize(initializationInput); - Thread.sleep(2000); - kinesisRecordProcessor.processRecords(processRecordsInput); - verify(checkpointer).checkpoint(); - verify(buffer).writeAll(anyCollection(), anyInt()); + verify(checkpointer).checkpoint(eq(sequence_number), eq(sub_sequence_number)); + + final ArgumentCaptor> recordArgumentCaptor = ArgumentCaptor.forClass(Record.class); + + verify(bufferAccumulator).add(recordArgumentCaptor.capture()); + verify(bufferAccumulator).flush(); + + List> recordsCaptured = recordArgumentCaptor.getAllValues(); + assertEquals(recordsCaptured.size(), records.size()); + for (Record eventRecord: recordsCaptured) { + EventMetadata eventMetadata = eventRecord.getData().getMetadata(); + assertEquals(eventMetadata.getAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE), streamIdentifier.streamName()); + } + verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); + verify(recordProcessed, times(1)).increment(anyDouble()); } @Test - void testProcessRecordsWithAcknowledgementsCheckpointsEnabled() + public void testProcessRecordsWithoutAcknowledgementsEnabled() throws Exception { List kinesisClientRecords = createInputKinesisClientRecords(); when(processRecordsInput.records()).thenReturn(kinesisClientRecords); - when(kinesisSourceConfig.isAcknowledgments()).thenReturn(true); - when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); - AtomicReference numEventsAdded = new AtomicReference<>(0); - doAnswer(a -> { - numEventsAdded.getAndSet(numEventsAdded.get() + 1); - return null; - }).when(acknowledgementSet).add(any()); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); + when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); - doAnswer(invocation -> { - Consumer consumer = invocation.getArgument(0); - consumer.accept(true); - return acknowledgementSet; - }).when(acknowledgementSetManager).create(any(Consumer.class), any(Duration.class)); + List> records = new ArrayList<>(); + Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + Record record = new Record<>(event); + records.add(record); + when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); - kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + when(kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord()).thenReturn(Optional.empty()); + kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); kinesisRecordProcessor.initialize(initializationInput); - Thread.sleep(2000); - kinesisRecordProcessor.processRecords(processRecordsInput); - verify(checkpointer).checkpoint(); - verify(buffer).writeAll(anyCollection(), anyInt()); - verify(acknowledgementSetManager, times(1)).create(any(), any(Duration.class)); + verifyNoInteractions(checkpointer); + + final ArgumentCaptor> recordArgumentCaptor = ArgumentCaptor.forClass(Record.class); + + verify(bufferAccumulator).add(recordArgumentCaptor.capture()); + verify(bufferAccumulator).flush(); + + List> recordsCaptured = recordArgumentCaptor.getAllValues(); + assertEquals(recordsCaptured.size(), records.size()); + for (Record eventRecord: recordsCaptured) { + EventMetadata eventMetadata = eventRecord.getData().getMetadata(); + assertEquals(eventMetadata.getAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE), streamIdentifier.streamName()); + } + + verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); + verify(recordProcessed, times(1)).increment(anyDouble()); } @Test - void testProcessRecordsWithAcknowledgementsEnabledAndAcksReturnFalse() + void testProcessRecordsWithAcknowledgementsEnabled() throws Exception { List kinesisClientRecords = createInputKinesisClientRecords(); when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(true); - when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); AtomicReference numEventsAdded = new AtomicReference<>(0); doAnswer(a -> { numEventsAdded.getAndSet(numEventsAdded.get() + 1); @@ -207,28 +278,52 @@ void testProcessRecordsWithAcknowledgementsEnabledAndAcksReturnFalse() doAnswer(invocation -> { Consumer consumer = invocation.getArgument(0); - consumer.accept(false); + consumer.accept(true); return acknowledgementSet; }).when(acknowledgementSetManager).create(any(Consumer.class), any(Duration.class)); - kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + List> records = new ArrayList<>(); + Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + Record record = new Record<>(event); + records.add(record); + when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); + + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + KinesisCheckpointerRecord kinesisCheckpointerRecord = mock(KinesisCheckpointerRecord.class); + ExtendedSequenceNumber extendedSequenceNumber = mock(ExtendedSequenceNumber.class); + when(extendedSequenceNumber.sequenceNumber()).thenReturn(sequence_number); + when(extendedSequenceNumber.subSequenceNumber()).thenReturn(sub_sequence_number); + when(kinesisCheckpointerRecord.getExtendedSequenceNumber()).thenReturn(extendedSequenceNumber); + when(kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); + kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); kinesisRecordProcessor.initialize(initializationInput); - Thread.sleep(2000); - kinesisRecordProcessor.processRecords(processRecordsInput); - verify(checkpointer, times(0)).checkpoint(); - verify(buffer).writeAll(anyCollection(), anyInt()); + final ArgumentCaptor> recordArgumentCaptor = ArgumentCaptor.forClass(Record.class); + + verify(bufferAccumulator).add(recordArgumentCaptor.capture()); + verify(bufferAccumulator).flush(); + + List> recordsCaptured = recordArgumentCaptor.getAllValues(); + assertEquals(recordsCaptured.size(), records.size()); + for (Record eventRecord: recordsCaptured) { + EventMetadata eventMetadata = eventRecord.getData().getMetadata(); + assertEquals(eventMetadata.getAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE), streamIdentifier.streamName()); + } verify(acknowledgementSetManager, times(1)).create(any(), any(Duration.class)); + verify(acknowledgementSetSuccesses, atLeastOnce()).increment(); + verify(recordProcessed, times(1)).increment(anyDouble()); + verifyNoInteractions(recordProcessingErrors); } @Test void testProcessRecordsWithNDJsonInputCodec() throws Exception { - when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); - when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(true); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); PluginModel pluginModel = mock(PluginModel.class); when(pluginModel.getPluginName()).thenReturn("ndjson"); @@ -239,79 +334,96 @@ void testProcessRecordsWithNDJsonInputCodec() when(pluginFactory.loadPlugin(eq(InputCodec.class), any())).thenReturn(codec); when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); - kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + + List> records = new ArrayList<>(); + Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + Record record = new Record<>(event); + records.add(record); + when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); + + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + KinesisCheckpointerRecord kinesisCheckpointerRecord = mock(KinesisCheckpointerRecord.class); + ExtendedSequenceNumber extendedSequenceNumber = mock(ExtendedSequenceNumber.class); + when(extendedSequenceNumber.sequenceNumber()).thenReturn(sequence_number); + when(extendedSequenceNumber.subSequenceNumber()).thenReturn(sub_sequence_number); + when(kinesisCheckpointerRecord.getExtendedSequenceNumber()).thenReturn(extendedSequenceNumber); + when(kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); kinesisRecordProcessor.initialize(initializationInput); - Thread.sleep(2000); kinesisRecordProcessor.processRecords(processRecordsInput); - verify(checkpointer).checkpoint(); - verify(buffer).writeAll(anyCollection(), anyInt()); + verify(checkpointer).checkpoint(eq(sequence_number), eq(sub_sequence_number)); + final ArgumentCaptor> recordArgumentCaptor = ArgumentCaptor.forClass(Record.class); + + verify(bufferAccumulator).add(recordArgumentCaptor.capture()); + + List> recordsCaptured = recordArgumentCaptor.getAllValues(); + assertEquals(recordsCaptured.size(), records.size()); + for (Record eventRecord: recordsCaptured) { + EventMetadata eventMetadata = eventRecord.getData().getMetadata(); + assertEquals(eventMetadata.getAttribute(MetadataKeyAttributes.KINESIS_STREAM_NAME_METADATA_ATTRIBUTE), streamIdentifier.streamName()); + } + verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); + verify(recordProcessed, times(1)).increment(anyDouble()); } - @Test void testProcessRecordsNoThrowException() throws Exception { - when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); - when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); + List> records = new ArrayList<>(); + Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + Record record = new Record<>(event); + records.add(record); + when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); final Throwable exception = mock(RuntimeException.class); - doThrow(exception).when(buffer).writeAll(any(), anyInt()); - - recordProcessingErrors = mock(Counter.class); - when(pluginMetrics.counterWithTags(KINESIS_RECORD_PROCESSING_ERRORS, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(recordProcessingErrors); + doThrow(exception).when(bufferAccumulator).add(any(Record.class)); - kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); kinesisRecordProcessor.initialize(initializationInput); assertDoesNotThrow(() -> kinesisRecordProcessor.processRecords(processRecordsInput)); verify(recordProcessingErrors, times(1)).increment(); + verify(recordProcessed, times(0)).increment(anyDouble()); } @Test - void testProcessRecordsWithoutAcknowledgementsAndCheckpoints() + void testProcessRecordsBufferFlushNoThrowException() throws Exception { - when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); - when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); - when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); - - kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); - kinesisRecordProcessor.initialize(initializationInput); - - kinesisRecordProcessor.processRecords(processRecordsInput); - verifyNoInteractions(checkpointer); - verify(buffer).writeAll(anyCollection(), anyInt()); - verify(acknowledgementSetManager, times(0)).create(any(), any(Duration.class)); - } - - @Test - void testProcessRecordsWithAcknowledgements() - throws Exception { - when(processRecordsInput.records()).thenReturn(createInputKinesisClientRecords()); - when(kinesisSourceConfig.isAcknowledgments()).thenReturn(true); - when(kinesisStreamConfig.isEnableCheckPoint()).thenReturn(false); - when(acknowledgementSetManager.create(any(), any(Duration.class))).thenReturn(acknowledgementSet); + List> records = new ArrayList<>(); + Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); + Record record = new Record<>(event); + records.add(record); + when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); + final Throwable exception = mock(RuntimeException.class); + doThrow(exception).when(bufferAccumulator).flush(); - kinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); kinesisRecordProcessor.initialize(initializationInput); - kinesisRecordProcessor.processRecords(processRecordsInput); + assertDoesNotThrow(() -> kinesisRecordProcessor.processRecords(processRecordsInput)); + verify(recordProcessingErrors, times(1)).increment(); + verify(recordProcessed, times(0)).increment(anyDouble()); - verifyNoInteractions(checkpointer); - verify(buffer).writeAll(anyCollection(), anyInt()); - verify(acknowledgementSetManager, times(1)).create(any(), any(Duration.class)); } @Test - void testShardEndedFlushCalled() throws Exception { - KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + void testShardEndedLatestCheckpoint() { + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); ShardEndedInput shardEndedInput = mock(ShardEndedInput.class); when(shardEndedInput.checkpointer()).thenReturn(checkpointer); + mockKinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); mockKinesisRecordProcessor.shardEnded(shardEndedInput); @@ -324,7 +436,8 @@ void testShardEndedCheckpointerThrowsNoThrowException(final Class exc checkpointFailures = mock(Counter.class); when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(checkpointFailures); - KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + mockKinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); ShardEndedInput shardEndedInput = mock(ShardEndedInput.class); when(shardEndedInput.checkpointer()).thenReturn(checkpointer); doThrow(exceptionType).when(checkpointer).checkpoint(); @@ -337,11 +450,12 @@ void testShardEndedCheckpointerThrowsNoThrowException(final Class exc } @Test - void testShutdownRequested() { + void testShutdownRequestedWithLatestCheckpoint() { checkpointFailures = mock(Counter.class); when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(checkpointFailures); - KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory, streamIdentifier); + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + mockKinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); ShutdownRequestedInput shutdownRequestedInput = mock(ShutdownRequestedInput.class); when(shutdownRequestedInput.checkpointer()).thenReturn(checkpointer); @@ -357,7 +471,24 @@ void testShutdownRequestedCheckpointerThrowsNoThrowException(final Class mockKinesisRecordProcessor.checkpoint(checkpointer, sequence_number, sub_sequence_number)); + + verify(checkpointer).checkpoint(eq(sequence_number), eq(sub_sequence_number)); + verify(checkpointFailures, times(1)).increment(); + } + + @ParameterizedTest + @ValueSource(classes = {ShutdownException.class, ThrottlingException.class, InvalidStateException.class}) + void testShutdownRequestedCheckpointerThrowsNoThrowExceptionRegularCheckpoint(final Class exceptionType) throws Exception { + checkpointFailures = mock(Counter.class); + when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(checkpointFailures); + + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + mockKinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); ShutdownRequestedInput shutdownRequestedInput = mock(ShutdownRequestedInput.class); when(shutdownRequestedInput.checkpointer()).thenReturn(checkpointer); doThrow(exceptionType).when(checkpointer).checkpoint(); @@ -373,7 +504,9 @@ private List createInputKinesisClientRecords() { List kinesisClientRecords = new ArrayList<>(); for (int i = 0; i< KinesisRecordProcessorTest.NUMBER_OF_RECORDS_TO_ACCUMULATE; i++) { Event event = JacksonEvent.fromMessage(UUID.randomUUID().toString()); - KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder().data(ByteBuffer.wrap(event.toJsonString().getBytes())).build(); + KinesisClientRecord kinesisClientRecord = KinesisClientRecord.builder() + .data(ByteBuffer.wrap(event.toJsonString().getBytes())) + .sequenceNumber(Integer.toString(100 + i)).subSequenceNumber(i).build(); kinesisClientRecords.add(kinesisClientRecord); } return kinesisClientRecords; diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java index 1a74514b6b..9f0a555253 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.java @@ -63,6 +63,9 @@ public class KinesisShardRecordProcessorFactoryTest { @Mock private AcknowledgementSetManager acknowledgementSetManager; + @Mock + private InputCodec codec; + @BeforeEach void setup() { MockitoAnnotations.initMocks(this); @@ -71,8 +74,9 @@ void setup() { when(pluginModel.getPluginName()).thenReturn(codec_plugin_name); when(pluginModel.getPluginSettings()).thenReturn(Collections.emptyMap()); when(kinesisSourceConfig.getCodec()).thenReturn(pluginModel); + when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(100); - InputCodec codec = mock(InputCodec.class); + codec = mock(InputCodec.class); when(pluginFactory.loadPlugin(eq(InputCodec.class), any())).thenReturn(codec); when(streamIdentifier.streamName()).thenReturn(streamId); @@ -82,13 +86,13 @@ void setup() { @Test void testKinesisRecordProcessFactoryReturnsKinesisRecordProcessor() { - kinesisShardRecordProcessorFactory = new KinesisShardRecordProcessorFactory(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory); + kinesisShardRecordProcessorFactory = new KinesisShardRecordProcessorFactory(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, codec); assertInstanceOf(KinesisRecordProcessor.class, kinesisShardRecordProcessorFactory.shardRecordProcessor(streamIdentifier)); } @Test void testKinesisRecordProcessFactoryDefaultUnsupported() { - kinesisShardRecordProcessorFactory = new KinesisShardRecordProcessorFactory(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, pluginFactory); + kinesisShardRecordProcessorFactory = new KinesisShardRecordProcessorFactory(buffer, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, codec); assertThrows(UnsupportedOperationException.class, () -> kinesisShardRecordProcessorFactory.shardRecordProcessor()); } } diff --git a/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_checkpoint_enabled.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_checkpoint_enabled.yaml index e918048529..c8b58725fd 100644 --- a/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_checkpoint_enabled.yaml +++ b/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_checkpoint_enabled.yaml @@ -3,15 +3,12 @@ source: streams: - stream_name: "stream-1" initial_position: "EARLIEST" - enable_checkpoint: true checkpoint_interval: "20s" - stream_name: "stream-2" initial_position: "EARLIEST" - enable_checkpoint: true checkpoint_interval: "PT15M" - stream_name: "stream-3" initial_position: "EARLIEST" - enable_checkpoint: true checkpoint_interval: "PT2H" codec: ndjson: From 806f78d574a2d49b71acba07a54b2509e643e5f3 Mon Sep 17 00:00:00 2001 From: Souvik Bose Date: Tue, 17 Sep 2024 10:17:41 -0700 Subject: [PATCH 8/8] Rename function and address comments. Signed-off-by: Souvik Bose --- .../source/KinesisMultiStreamTracker.java | 16 ++---- .../processor/KinesisCheckpointerTracker.java | 2 +- .../processor/KinesisRecordProcessor.java | 48 +++++++--------- .../KinesisShardRecordProcessorFactory.java | 4 +- .../KinesisCheckpointerTrackerTest.java | 8 +-- .../processor/KinesisRecordProcessorTest.java | 55 ++++++++++--------- 6 files changed, 64 insertions(+), 69 deletions(-) diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java index 4414d9fd39..638751f17e 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java @@ -10,6 +10,7 @@ package org.opensearch.dataprepper.plugins.kinesis.source; +import com.amazonaws.arn.Arn; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisSourceConfig; import org.opensearch.dataprepper.plugins.kinesis.source.configuration.KinesisStreamConfig; import software.amazon.awssdk.services.kinesis.KinesisAsyncClient; @@ -44,34 +45,29 @@ public KinesisMultiStreamTracker(KinesisAsyncClient kinesisClient, final Kinesis public List streamConfigList() { List streamConfigList = new ArrayList<>(); for (KinesisStreamConfig kinesisStreamConfig : sourceConfig.getStreams()) { - StreamConfig streamConfig; - try { - streamConfig = getStreamConfig(kinesisStreamConfig); - } catch (Exception e) { - throw new RuntimeException(e); - } + StreamConfig streamConfig = getStreamConfig(kinesisStreamConfig); streamConfigList.add(streamConfig); } return streamConfigList; } - private StreamConfig getStreamConfig(KinesisStreamConfig kinesisStreamConfig) throws Exception { + private StreamConfig getStreamConfig(KinesisStreamConfig kinesisStreamConfig) { StreamIdentifier sourceStreamIdentifier = getStreamIdentifier(kinesisStreamConfig); return new StreamConfig(sourceStreamIdentifier, InitialPositionInStreamExtended.newInitialPosition(kinesisStreamConfig.getInitialPosition())); } - private StreamIdentifier getStreamIdentifier(KinesisStreamConfig kinesisStreamConfig) throws Exception { + private StreamIdentifier getStreamIdentifier(KinesisStreamConfig kinesisStreamConfig) { DescribeStreamRequest describeStreamRequest = DescribeStreamRequest.builder() .streamName(kinesisStreamConfig.getName()) .build(); - DescribeStreamResponse describeStreamResponse = kinesisClient.describeStream(describeStreamRequest).get(); + DescribeStreamResponse describeStreamResponse = kinesisClient.describeStream(describeStreamRequest).join(); String streamIdentifierString = getStreamIdentifierString(describeStreamResponse.streamDescription()); return StreamIdentifier.multiStreamInstance(streamIdentifierString); } private String getStreamIdentifierString(StreamDescription streamDescription) { - String accountId = streamDescription.streamARN().split(COLON)[4]; + String accountId = Arn.fromString(streamDescription.streamARN()).getAccountId(); long creationEpochSecond = streamDescription.streamCreationTimestamp().getEpochSecond(); return String.join(COLON, accountId, streamDescription.streamName(), String.valueOf(creationEpochSecond)); } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTracker.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTracker.java index 9f8e5f7625..8fb7c5ec6c 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTracker.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTracker.java @@ -38,7 +38,7 @@ public synchronized void markSequenceNumberForCheckpoint(final ExtendedSequenceN checkpointerRecordList.get(extendedSequenceNumber).setReadyToCheckpoint(true); } - public synchronized Optional getLatestAvailableCheckpointRecord() { + public synchronized Optional popLatestReadyToCheckpointRecord() { Optional kinesisCheckpointerRecordOptional = Optional.empty(); List toRemoveRecords = new ArrayList<>(); diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java index 7d02ff959a..6df0760ca3 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java @@ -49,31 +49,34 @@ public class KinesisRecordProcessor implements ShardRecordProcessor { private static final Logger LOG = LoggerFactory.getLogger(KinesisRecordProcessor.class); + private static final int DEFAULT_MONITOR_WAIT_TIME_MS = 15_000; + private static final Duration ACKNOWLEDGEMENT_SET_TIMEOUT = Duration.ofSeconds(20); + private final StreamIdentifier streamIdentifier; private final KinesisStreamConfig kinesisStreamConfig; private final Duration checkpointInterval; private final KinesisSourceConfig kinesisSourceConfig; private final BufferAccumulator> bufferAccumulator; private final KinesisRecordConverter kinesisRecordConverter; + private final KinesisCheckpointerTracker kinesisCheckpointerTracker; + private final ExecutorService executorService; private String kinesisShardId; private long lastCheckpointTimeInMillis; private final int bufferTimeoutMillis; private final AcknowledgementSetManager acknowledgementSetManager; + private final Counter acknowledgementSetSuccesses; private final Counter acknowledgementSetFailures; private final Counter recordsProcessed; private final Counter recordProcessingErrors; private final Counter checkpointFailures; - private static final Duration ACKNOWLEDGEMENT_SET_TIMEOUT = Duration.ofSeconds(20); public static final String ACKNOWLEDGEMENT_SET_SUCCESS_METRIC_NAME = "acknowledgementSetSuccesses"; public static final String ACKNOWLEDGEMENT_SET_FAILURES_METRIC_NAME = "acknowledgementSetFailures"; public static final String KINESIS_RECORD_PROCESSED = "recordProcessed"; public static final String KINESIS_RECORD_PROCESSING_ERRORS = "recordProcessingErrors"; public static final String KINESIS_CHECKPOINT_FAILURES = "checkpointFailures"; public static final String KINESIS_STREAM_TAG_KEY = "stream"; - private KinesisCheckpointerTracker kinesisCheckpointerTracker; - private final ExecutorService executorService; private AtomicBoolean isStopRequested; public KinesisRecordProcessor(final BufferAccumulator> bufferAccumulator, @@ -81,6 +84,7 @@ public KinesisRecordProcessor(final BufferAccumulator> bufferAccum final AcknowledgementSetManager acknowledgementSetManager, final PluginMetrics pluginMetrics, final KinesisRecordConverter kinesisRecordConverter, + final KinesisCheckpointerTracker kinesisCheckpointerTracker, final StreamIdentifier streamIdentifier) { this.bufferTimeoutMillis = (int) kinesisSourceConfig.getBufferTimeout().toMillis(); this.streamIdentifier = streamIdentifier; @@ -95,7 +99,7 @@ public KinesisRecordProcessor(final BufferAccumulator> bufferAccum this.checkpointFailures = pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName()); this.checkpointInterval = kinesisStreamConfig.getCheckPointInterval(); this.bufferAccumulator = bufferAccumulator; - this.kinesisCheckpointerTracker = new KinesisCheckpointerTracker(); + this.kinesisCheckpointerTracker = kinesisCheckpointerTracker; this.executorService = Executors.newSingleThreadExecutor(BackgroundThreadFactory.defaultExecutorThreadFactory("kinesis-ack-monitor")); this.isStopRequested = new AtomicBoolean(false); } @@ -120,16 +124,7 @@ public void initialize(InitializationInput initializationInput) { private void monitorCheckpoint(final ExecutorService executorService) { while (!isStopRequested.get()) { if (System.currentTimeMillis() - lastCheckpointTimeInMillis >= checkpointInterval.toMillis()) { - LOG.debug("Regular checkpointing for shard {}", kinesisShardId); - - Optional kinesisCheckpointerRecordOptional = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); - if (kinesisCheckpointerRecordOptional.isPresent()) { - RecordProcessorCheckpointer recordProcessorCheckpointer = kinesisCheckpointerRecordOptional.get().getCheckpointer(); - String sequenceNumber = kinesisCheckpointerRecordOptional.get().getExtendedSequenceNumber().sequenceNumber(); - Long subSequenceNumber = kinesisCheckpointerRecordOptional.get().getExtendedSequenceNumber().subSequenceNumber(); - checkpoint(recordProcessorCheckpointer, sequenceNumber, subSequenceNumber); - lastCheckpointTimeInMillis = System.currentTimeMillis(); - } + doCheckpoint(); } try { Thread.sleep(DEFAULT_MONITOR_WAIT_TIME_MS); @@ -195,14 +190,7 @@ public void processRecords(ProcessRecordsInput processRecordsInput) { // Checkpoint for shard if (!acknowledgementsEnabled && (System.currentTimeMillis() - lastCheckpointTimeInMillis >= checkpointInterval.toMillis())) { - LOG.debug("Regular checkpointing for shard {}", kinesisShardId); - - Optional KinesisCheckpointerRecordOptional = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); - if (KinesisCheckpointerRecordOptional.isPresent()) { - ExtendedSequenceNumber lastExtendedSequenceNumber = KinesisCheckpointerRecordOptional.get().getExtendedSequenceNumber(); - checkpoint(processRecordsInput.checkpointer(), lastExtendedSequenceNumber.sequenceNumber(), lastExtendedSequenceNumber.subSequenceNumber()); - lastCheckpointTimeInMillis = System.currentTimeMillis(); - } + doCheckpoint(); } } catch (Exception ex) { recordProcessingErrors.increment(); @@ -242,6 +230,17 @@ public void checkpoint(RecordProcessorCheckpointer checkpointer, String sequence } } + private void doCheckpoint() { + LOG.debug("Regular checkpointing for shard {}", kinesisShardId); + Optional kinesisCheckpointerRecordOptional = kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord(); + if (kinesisCheckpointerRecordOptional.isPresent()) { + ExtendedSequenceNumber lastExtendedSequenceNumber = kinesisCheckpointerRecordOptional.get().getExtendedSequenceNumber(); + RecordProcessorCheckpointer recordProcessorCheckpointer = kinesisCheckpointerRecordOptional.get().getCheckpointer(); + checkpoint(recordProcessorCheckpointer, lastExtendedSequenceNumber.sequenceNumber(), lastExtendedSequenceNumber.subSequenceNumber()); + lastCheckpointTimeInMillis = System.currentTimeMillis(); + } + } + private void checkpoint(RecordProcessorCheckpointer checkpointer) { try { String kinesisStream = streamIdentifier.streamName(); @@ -268,9 +267,4 @@ private ExtendedSequenceNumber getLatestSequenceNumberFromInput(final ProcessRec } return largestExtendedSequenceNumber; } - - @VisibleForTesting - public void setKinesisCheckpointerTracker(final KinesisCheckpointerTracker kinesisCheckpointerTracker) { - this.kinesisCheckpointerTracker = kinesisCheckpointerTracker; - } } diff --git a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java index e326789312..ff9943a41d 100644 --- a/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java @@ -52,6 +52,8 @@ public ShardRecordProcessor shardRecordProcessor() { public ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) { BufferAccumulator> bufferAccumulator = BufferAccumulator.create(buffer, kinesisSourceConfig.getNumberOfRecordsToAccumulate(), kinesisSourceConfig.getBufferTimeout()); - return new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + KinesisCheckpointerTracker kinesisCheckpointerTracker = new KinesisCheckpointerTracker(); + return new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, + pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); } } \ No newline at end of file diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTrackerTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTrackerTest.java index ea76d1f789..fe0ab06877 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTrackerTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisCheckpointerTrackerTest.java @@ -45,7 +45,7 @@ void testCheckPointerAddAndGet() { ExtendedSequenceNumber last = extendedSequenceNumberList.get(extendedSequenceNumberList.size()-1); kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(last); - Optional checkpointRecord = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + Optional checkpointRecord = kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord(); assertTrue(checkpointRecord.isEmpty()); assertEquals(kinesisCheckpointerTracker.size(), numRecords); @@ -53,7 +53,7 @@ void testCheckPointerAddAndGet() { ExtendedSequenceNumber extendedSequenceNumber1 = extendedSequenceNumberList.get(idx); kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(extendedSequenceNumber1); - Optional firstcheckpointer = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + Optional firstcheckpointer = kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord(); if (idx != 0) { assertTrue(firstcheckpointer.isEmpty()); assertEquals(kinesisCheckpointerTracker.size(), numRecords); @@ -79,7 +79,7 @@ void testGetLastCheckpointerAndStoreIsEmpty() { kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(extendedSequenceNumber); } - Optional checkpointer = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + Optional checkpointer = kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord(); assertTrue(checkpointer.isPresent()); assertEquals(0, kinesisCheckpointerTracker.size()); } @@ -92,7 +92,7 @@ public void testMarkCheckpointerReadyForCheckpoint() { ExtendedSequenceNumber extendedSequenceNumber = mock(ExtendedSequenceNumber.class); assertThrows(IllegalArgumentException.class, () -> kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(extendedSequenceNumber)); - Optional checkpointer = kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord(); + Optional checkpointer = kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord(); assertTrue(checkpointer.isEmpty()); } } diff --git a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java index 902a70bfa6..ea002e27e9 100644 --- a/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java @@ -193,14 +193,15 @@ void testProcessRecordsWithoutAcknowledgementsWithCheckpointApplied() records.add(record); when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); - kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); KinesisCheckpointerRecord kinesisCheckpointerRecord = mock(KinesisCheckpointerRecord.class); ExtendedSequenceNumber extendedSequenceNumber = mock(ExtendedSequenceNumber.class); when(extendedSequenceNumber.sequenceNumber()).thenReturn(sequence_number); when(extendedSequenceNumber.subSequenceNumber()).thenReturn(sub_sequence_number); when(kinesisCheckpointerRecord.getExtendedSequenceNumber()).thenReturn(extendedSequenceNumber); - when(kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); - kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + when(kinesisCheckpointerRecord.getCheckpointer()).thenReturn(checkpointer); + when(kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); kinesisRecordProcessor.initialize(initializationInput); kinesisRecordProcessor.processRecords(processRecordsInput); @@ -238,9 +239,9 @@ public void testProcessRecordsWithoutAcknowledgementsEnabled() records.add(record); when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); - kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); - when(kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord()).thenReturn(Optional.empty()); - kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); + when(kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord()).thenReturn(Optional.empty()); kinesisRecordProcessor.initialize(initializationInput); kinesisRecordProcessor.processRecords(processRecordsInput); @@ -288,14 +289,15 @@ void testProcessRecordsWithAcknowledgementsEnabled() records.add(record); when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); - kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); KinesisCheckpointerRecord kinesisCheckpointerRecord = mock(KinesisCheckpointerRecord.class); ExtendedSequenceNumber extendedSequenceNumber = mock(ExtendedSequenceNumber.class); when(extendedSequenceNumber.sequenceNumber()).thenReturn(sequence_number); when(extendedSequenceNumber.subSequenceNumber()).thenReturn(sub_sequence_number); when(kinesisCheckpointerRecord.getExtendedSequenceNumber()).thenReturn(extendedSequenceNumber); - when(kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); - kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + when(kinesisCheckpointerRecord.getCheckpointer()).thenReturn(checkpointer); + when(kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); kinesisRecordProcessor.initialize(initializationInput); kinesisRecordProcessor.processRecords(processRecordsInput); @@ -341,14 +343,15 @@ void testProcessRecordsWithNDJsonInputCodec() records.add(record); when(kinesisRecordConverter.convert(eq(kinesisClientRecords))).thenReturn(records); - kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); - kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); KinesisCheckpointerRecord kinesisCheckpointerRecord = mock(KinesisCheckpointerRecord.class); ExtendedSequenceNumber extendedSequenceNumber = mock(ExtendedSequenceNumber.class); when(extendedSequenceNumber.sequenceNumber()).thenReturn(sequence_number); when(extendedSequenceNumber.subSequenceNumber()).thenReturn(sub_sequence_number); + when(kinesisCheckpointerRecord.getCheckpointer()).thenReturn(checkpointer); when(kinesisCheckpointerRecord.getExtendedSequenceNumber()).thenReturn(extendedSequenceNumber); - when(kinesisCheckpointerTracker.getLatestAvailableCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); + when(kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); kinesisRecordProcessor.initialize(initializationInput); kinesisRecordProcessor.processRecords(processRecordsInput); @@ -384,8 +387,8 @@ void testProcessRecordsNoThrowException() final Throwable exception = mock(RuntimeException.class); doThrow(exception).when(bufferAccumulator).add(any(Record.class)); - kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); - kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); kinesisRecordProcessor.initialize(initializationInput); assertDoesNotThrow(() -> kinesisRecordProcessor.processRecords(processRecordsInput)); @@ -408,8 +411,8 @@ void testProcessRecordsBufferFlushNoThrowException() final Throwable exception = mock(RuntimeException.class); doThrow(exception).when(bufferAccumulator).flush(); - kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); - kinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); kinesisRecordProcessor.initialize(initializationInput); assertDoesNotThrow(() -> kinesisRecordProcessor.processRecords(processRecordsInput)); @@ -420,10 +423,10 @@ void testProcessRecordsBufferFlushNoThrowException() @Test void testShardEndedLatestCheckpoint() { - KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); ShardEndedInput shardEndedInput = mock(ShardEndedInput.class); when(shardEndedInput.checkpointer()).thenReturn(checkpointer); - mockKinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); mockKinesisRecordProcessor.shardEnded(shardEndedInput); @@ -436,8 +439,8 @@ void testShardEndedCheckpointerThrowsNoThrowException(final Class exc checkpointFailures = mock(Counter.class); when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(checkpointFailures); - KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); - mockKinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); ShardEndedInput shardEndedInput = mock(ShardEndedInput.class); when(shardEndedInput.checkpointer()).thenReturn(checkpointer); doThrow(exceptionType).when(checkpointer).checkpoint(); @@ -454,8 +457,8 @@ void testShutdownRequestedWithLatestCheckpoint() { checkpointFailures = mock(Counter.class); when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(checkpointFailures); - KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); - mockKinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); ShutdownRequestedInput shutdownRequestedInput = mock(ShutdownRequestedInput.class); when(shutdownRequestedInput.checkpointer()).thenReturn(checkpointer); @@ -471,8 +474,8 @@ void testShutdownRequestedCheckpointerThrowsNoThrowException(final Class mockKinesisRecordProcessor.checkpoint(checkpointer, sequence_number, sub_sequence_number)); @@ -487,8 +490,8 @@ void testShutdownRequestedCheckpointerThrowsNoThrowExceptionRegularCheckpoint(fi checkpointFailures = mock(Counter.class); when(pluginMetrics.counterWithTags(KINESIS_CHECKPOINT_FAILURES, KINESIS_STREAM_TAG_KEY, streamIdentifier.streamName())).thenReturn(checkpointFailures); - KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, streamIdentifier); - mockKinesisRecordProcessor.setKinesisCheckpointerTracker(kinesisCheckpointerTracker); + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); ShutdownRequestedInput shutdownRequestedInput = mock(ShutdownRequestedInput.class); when(shutdownRequestedInput.checkpointer()).thenReturn(checkpointer); doThrow(exceptionType).when(checkpointer).checkpoint();