diff --git a/data-prepper-plugins/kinesis-source/build.gradle b/data-prepper-plugins/kinesis-source/build.gradle new file mode 100644 index 0000000000..c4a0614e36 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/build.gradle @@ -0,0 +1,42 @@ +/* + * 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(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' + + testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' + testImplementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' + testImplementation project(':data-prepper-test-common') + 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' + testImplementation project(':data-prepper-plugins:parse-json-processor') + testImplementation project(':data-prepper-plugins:newline-codecs') +} + +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/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..68981c5cba --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfig.java @@ -0,0 +1,20 @@ +/* + * 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; +import lombok.Getter; + +@Getter +public class KinesisLeaseConfig { + @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 new file mode 100644 index 0000000000..2cca52e565 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtension.java @@ -0,0 +1,31 @@ +/* + * 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; +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", allowInPipelineConfigurations = true) +public class KinesisLeaseConfigExtension implements ExtensionPlugin { + + private KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; + @DataPrepperPluginConstructor + public KinesisLeaseConfigExtension(final KinesisLeaseConfig kinesisLeaseConfig) { + this.kinesisLeaseConfigSupplier = new KinesisLeaseConfigSupplier(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..9140ca9e92 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProvider.java @@ -0,0 +1,33 @@ +/* + * 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; + +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..6c00e40405 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplier.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.extension; + +import java.util.Optional; + +public class KinesisLeaseConfigSupplier { + + 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/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..d497f01369 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseCoordinationTableConfig.java @@ -0,0 +1,32 @@ +/* + * 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; +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/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..61383304d0 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/HostNameWorkerIdentifierGenerator.java @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.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 new file mode 100644 index 0000000000..8f3bac38aa --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactory.java @@ -0,0 +1,61 @@ +/* + * 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; +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.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.kinesis.common.KinesisClientUtil; + +public class KinesisClientFactory { + private final AwsCredentialsProvider awsCredentialsProvider; + private final AwsCredentialsProvider defaultCredentialsProvider; + private final AwsAuthenticationConfig awsAuthenticationConfig; + + public KinesisClientFactory(final AwsCredentialsSupplier awsCredentialsSupplier, + final AwsAuthenticationConfig awsAuthenticationConfig) { + awsCredentialsProvider = awsCredentialsSupplier.getProvider(AwsCredentialsOptions.builder() + .withRegion(awsAuthenticationConfig.getAwsRegion()) + .withStsRoleArn(awsAuthenticationConfig.getAwsStsRoleArn()) + .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) + .region(region) + .build(); + } + + public KinesisAsyncClient buildKinesisAsyncClient(Region region) { + return KinesisClientUtil.createKinesisAsyncClient( + KinesisAsyncClient.builder() + .credentialsProvider(awsCredentialsProvider) + .region(region) + ); + } + + public CloudWatchAsyncClient buildCloudWatchAsyncClient(Region region) { + return CloudWatchAsyncClient.builder() + .credentialsProvider(defaultCredentialsProvider) + .region(region) + .build(); + } +} 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 new file mode 100644 index 0000000000..638751f17e --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTracker.java @@ -0,0 +1,88 @@ +/* + * 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.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; +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 = getStreamConfig(kinesisStreamConfig); + streamConfigList.add(streamConfig); + } + return streamConfigList; + } + + private StreamConfig getStreamConfig(KinesisStreamConfig kinesisStreamConfig) { + StreamIdentifier sourceStreamIdentifier = getStreamIdentifier(kinesisStreamConfig); + return new StreamConfig(sourceStreamIdentifier, + InitialPositionInStreamExtended.newInitialPosition(kinesisStreamConfig.getInitialPosition())); + } + + private StreamIdentifier getStreamIdentifier(KinesisStreamConfig kinesisStreamConfig) { + DescribeStreamRequest describeStreamRequest = DescribeStreamRequest.builder() + .streamName(kinesisStreamConfig.getName()) + .build(); + DescribeStreamResponse describeStreamResponse = kinesisClient.describeStream(describeStreamRequest).join(); + String streamIdentifierString = getStreamIdentifierString(describeStreamResponse.streamDescription()); + return StreamIdentifier.multiStreamInstance(streamIdentifierString); + } + + private String getStreamIdentifierString(StreamDescription streamDescription) { + String accountId = Arn.fromString(streamDescription.streamARN()).getAccountId(); + 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/kinesis/source/KinesisService.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java new file mode 100644 index 0000000000..4ed15833f6 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisService.java @@ -0,0 +1,169 @@ +/* + * 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; +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; +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; +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 kclMetricsNamespaceName; + private final String pipelineName; + private final AcknowledgementSetManager acknowledgementSetManager; + 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 kinesisSourceConfig, + final KinesisClientFactory kinesisClientFactory, + final PluginMetrics pluginMetrics, + final PluginFactory pluginFactory, + final PipelineDescription pipelineDescription, + final AcknowledgementSetManager acknowledgementSetManager, + final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier, + final WorkerIdentifierGenerator workerIdentifierGenerator + ){ + this.kinesisSourceConfig = kinesisSourceConfig; + this.pluginMetrics = pluginMetrics; + this.pluginFactory = pluginFactory; + this.acknowledgementSetManager = acknowledgementSetManager; + if (kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().isEmpty()) { + throw new IllegalStateException("Lease Coordination table should be provided!"); + } + KinesisLeaseConfig kinesisLeaseConfig = + kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get(); + this.tableName = kinesisLeaseConfig.getLeaseCoordinationTable().getTableName(); + this.kclMetricsNamespaceName = this.tableName; + this.dynamoDbClient = kinesisClientFactory.buildDynamoDBClient(kinesisLeaseConfig.getLeaseCoordinationTable().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."); + } + + if (kinesisSourceConfig.getStreams() == null || kinesisSourceConfig.getStreams().isEmpty()) { + throw new InvalidPluginConfigurationException("No Kinesis streams provided."); + } + + 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, kinesisSourceConfig, acknowledgementSetManager, pluginMetrics, codec); + + ConfigsBuilder configsBuilder = + new ConfigsBuilder( + new KinesisMultiStreamTracker(kinesisClient, kinesisSourceConfig, applicationName), + applicationName, kinesisClient, dynamoDbClient, cloudWatchClient, + workerIdentifierGenerator.generate(), processorFactory + ) + .tableName(tableName) + .namespace(kclMetricsNamespaceName); + + ConsumerStrategy consumerStrategy = kinesisSourceConfig.getConsumerStrategy(); + if (consumerStrategy == ConsumerStrategy.POLLING) { + configsBuilder.retrievalConfig().retrievalSpecificConfig( + new PollingConfig(kinesisClient) + .maxRecords(kinesisSourceConfig.getPollingConfig().getMaxPollingRecords()) + .idleTimeBetweenReadsInMillis( + kinesisSourceConfig.getPollingConfig().getIdleTimeBetweenReads().toMillis())); + } + + 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/kinesis/source/KinesisSource.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java new file mode 100644 index 0000000000..220d19cac8 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSource.java @@ -0,0 +1,71 @@ +/* + * 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; +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.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.extension.KinesisLeaseConfigSupplier; +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; + private final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; + + @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, + final KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier) { + this.kinesisSourceConfig = kinesisSourceConfig; + this.kinesisLeaseConfigSupplier = kinesisLeaseConfigSupplier; + KinesisClientFactory kinesisClientFactory = new KinesisClientFactory(awsCredentialsSupplier, kinesisSourceConfig.getAwsAuthenticationConfig()); + this.kinesisService = new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier, new HostNameWorkerIdentifierGenerator()); + } + @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/kinesis/source/WorkerIdentifierGenerator.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java new file mode 100644 index 0000000000..75bad8761a --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/WorkerIdentifierGenerator.java @@ -0,0 +1,16 @@ +/* + * 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 { + + 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 new file mode 100644 index 0000000000..6a98f70c3b --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfig.java @@ -0,0 +1,51 @@ +/* + * 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; + +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import lombok.Getter; +import software.amazon.awssdk.regions.Region; + +import java.util.Map; + +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; + + @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; + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } +} \ 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 new file mode 100644 index 0000000000..05fc88f62a --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/ConsumerStrategy.java @@ -0,0 +1,35 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; + +import com.fasterxml.jackson.annotation.JsonValue; + +/** + * @see Enhanced Consumers + */ + +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/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..37019cc9af --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfig.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.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 new file mode 100644 index 0000000000..1414229813 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfig.java @@ -0,0 +1,75 @@ +/* + * 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; +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 = "Provide 1-4 streams to read from.") + 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; + + @Getter + @NotNull + @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/kinesis/source/configuration/KinesisStreamConfig.java b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java new file mode 100644 index 0000000000..b26732e357 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamConfig.java @@ -0,0 +1,41 @@ +/* + * 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; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import lombok.Getter; +import software.amazon.kinesis.common.InitialPositionInStream; + +import java.time.Duration; + +@Getter +public class KinesisStreamConfig { + // Checkpointing interval + 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") + @NotNull + @Valid + private String name; + + @JsonProperty("initial_position") + private InitialPositionInStreamConfig initialPosition = InitialPositionInStreamConfig.LATEST; + + @JsonProperty("checkpoint_interval") + private Duration checkPointInterval = MINIMAL_CHECKPOINT_INTERVAL; + + 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 new file mode 100644 index 0000000000..cd7b7a59f6 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfig.java @@ -0,0 +1,29 @@ +/* + * 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; +import lombok.Getter; + +import java.time.Duration; + +public class KinesisStreamPollingConfig { + private static final int DEFAULT_MAX_RECORDS = 10000; + private static final Duration IDLE_TIME_BETWEEN_READS = Duration.ofMillis(250); + @Getter + @JsonProperty("max_polling_records") + private int maxPollingRecords = DEFAULT_MAX_RECORDS; + + @Getter + @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/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..8fb7c5ec6c --- /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 popLatestReadyToCheckpointRecord() { + 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 new file mode 100644 index 0000000000..6df0760ca3 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessor.java @@ -0,0 +1,270 @@ +/* + * 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.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +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 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 software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber; + +import java.time.Duration; +import java.util.List; +import java.util.ListIterator; +import java.util.Optional; +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(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; + 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 AtomicBoolean isStopRequested; + + public KinesisRecordProcessor(final BufferAccumulator> bufferAccumulator, + final KinesisSourceConfig kinesisSourceConfig, + final AcknowledgementSetManager acknowledgementSetManager, + final PluginMetrics pluginMetrics, + final KinesisRecordConverter kinesisRecordConverter, + final KinesisCheckpointerTracker kinesisCheckpointerTracker, + final StreamIdentifier streamIdentifier) { + this.bufferTimeoutMillis = (int) kinesisSourceConfig.getBufferTimeout().toMillis(); + this.streamIdentifier = streamIdentifier; + this.kinesisSourceConfig = kinesisSourceConfig; + this.kinesisStreamConfig = getStreamConfig(kinesisSourceConfig); + this.kinesisRecordConverter = kinesisRecordConverter; + this.acknowledgementSetManager = acknowledgementSetManager; + 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.bufferAccumulator = bufferAccumulator; + this.kinesisCheckpointerTracker = kinesisCheckpointerTracker; + this.executorService = Executors.newSingleThreadExecutor(BackgroundThreadFactory.defaultExecutorThreadFactory("kinesis-ack-monitor")); + this.isStopRequested = new AtomicBoolean(false); + } + + 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(); + String kinesisStreamName = streamIdentifier.streamName(); + LOG.info("Initialize Processor for stream: {}, shard: {}", kinesisStreamName, kinesisShardId); + lastCheckpointTimeInMillis = System.currentTimeMillis(); + + if (kinesisSourceConfig.isAcknowledgments()) { + executorService.submit(() -> monitorCheckpoint(executorService)); + } + } + + private void monitorCheckpoint(final ExecutorService executorService) { + while (!isStopRequested.get()) { + if (System.currentTimeMillis() - lastCheckpointTimeInMillis >= checkpointInterval.toMillis()) { + doCheckpoint(); + } + 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) -> { + String kinesisStreamName = streamIdentifier.streamName(); + if (result) { + acknowledgementSetSuccesses.increment(); + kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(extendedSequenceNumber); + LOG.debug("acknowledgements received for stream: {}, shardId: {}", kinesisStreamName, kinesisShardId); + } else { + acknowledgementSetFailures.increment(); + LOG.debug("acknowledgements received with false for stream: {}, shardId: {}", kinesisStreamName, kinesisShardId); + } + }, ACKNOWLEDGEMENT_SET_TIMEOUT); + } + + @Override + public void processRecords(ProcessRecordsInput processRecordsInput) { + try { + Optional acknowledgementSetOpt = Optional.empty(); + boolean acknowledgementsEnabled = kinesisSourceConfig.isAcknowledgments(); + ExtendedSequenceNumber extendedSequenceNumber = getLatestSequenceNumberFromInput(processRecordsInput); + if (acknowledgementsEnabled) { + acknowledgementSetOpt = Optional.of(createAcknowledgmentSet(processRecordsInput, extendedSequenceNumber)); + } + + // 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++; + } + + // Flush buffer at the end + bufferAccumulator.flush(); + recordsProcessed.increment(eventCount); + + // 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 (!acknowledgementsEnabled && (System.currentTimeMillis() - lastCheckpointTimeInMillis >= checkpointInterval.toMillis())) { + doCheckpoint(); + } + } catch (Exception ex) { + recordProcessingErrors.increment(); + LOG.error("Failed writing shard data to buffer: ", ex); + } + } + + @Override + public void leaseLost(LeaseLostInput leaseLostInput) { + LOG.debug("Lease Lost"); + } + + @Override + public void shardEnded(ShardEndedInput shardEndedInput) { + String kinesisStream = streamIdentifier.streamName(); + LOG.debug("Reached shard end, checkpointing for stream: {}, shardId: {}", kinesisStream, kinesisShardId); + checkpoint(shardEndedInput.checkpointer()); + } + + @Override + public void shutdownRequested(ShutdownRequestedInput shutdownRequestedInput) { + 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 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(); + LOG.debug("Checkpoint for stream: {}, shardId: {}", kinesisStream, kinesisShardId); + checkpointer.checkpoint(); + } catch (ShutdownException | ThrottlingException | InvalidStateException 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; + } +} 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 new file mode 100644 index 0000000000..ff9943a41d --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/main/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactory.java @@ -0,0 +1,59 @@ +/* + * 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.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; + +public class KinesisShardRecordProcessorFactory implements ShardRecordProcessorFactory { + + private final Buffer> buffer; + private final KinesisSourceConfig kinesisSourceConfig; + private final AcknowledgementSetManager acknowledgementSetManager; + private final PluginMetrics pluginMetrics; + private final KinesisRecordConverter kinesisRecordConverter; + + public KinesisShardRecordProcessorFactory(Buffer> buffer, + KinesisSourceConfig kinesisSourceConfig, + final AcknowledgementSetManager acknowledgementSetManager, + final PluginMetrics pluginMetrics, + final InputCodec codec) { + this.kinesisSourceConfig = kinesisSourceConfig; + this.buffer = buffer; + this.acknowledgementSetManager = acknowledgementSetManager; + this.pluginMetrics = pluginMetrics; + this.kinesisRecordConverter = new KinesisRecordConverter(codec); + } + + @Override + public ShardRecordProcessor shardRecordProcessor() { + throw new UnsupportedOperationException("Use the method with stream details!"); + } + + @Override + public ShardRecordProcessor shardRecordProcessor(StreamIdentifier streamIdentifier) { + BufferAccumulator> bufferAccumulator = BufferAccumulator.create(buffer, + kinesisSourceConfig.getNumberOfRecordsToAccumulate(), kinesisSourceConfig.getBufferTimeout()); + 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/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 new file mode 100644 index 0000000000..852baab195 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigExtensionTest.java @@ -0,0 +1,48 @@ +/* + * 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; +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..1fa17f5f42 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigProviderTest.java @@ -0,0 +1,57 @@ +/* + * 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; +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/KinesisLeaseConfigSupplierTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplierTest.java new file mode 100644 index 0000000000..4cfc323ed5 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigSupplierTest.java @@ -0,0 +1,60 @@ +/* + * 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; +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 KinesisLeaseConfigSupplierTest { + private static final String LEASE_COORDINATION_TABLE = "lease-table"; + @Mock + KinesisLeaseConfig kinesisLeaseConfig; + + @Mock + KinesisLeaseCoordinationTableConfig kinesisLeaseCoordinationTableConfig; + + private KinesisLeaseConfigSupplier createObjectUnderTest() { + return new KinesisLeaseConfigSupplier(kinesisLeaseConfig); + } + + @Test + void testGetters() { + 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().getTableName(), equalTo(LEASE_COORDINATION_TABLE)); + assertThat(kinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable().getRegion(), equalTo("us-east-1")); + } + + @Test + void testGettersWithNullTableConfig() { + when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(null); + KinesisLeaseConfigSupplier defaultKinesisLeaseConfigSupplier = createObjectUnderTest(); + assertThat(defaultKinesisLeaseConfigSupplier.getKinesisExtensionLeaseConfig().get().getLeaseCoordinationTable(), equalTo(null)); + + } + + @Test + void testGettersWithNullConfig() { + 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/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..30194a9659 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/extension/KinesisLeaseConfigTest.java @@ -0,0 +1,65 @@ +/* + * 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; +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 software.amazon.awssdk.regions.Region; + +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"); + 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().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/KinesisClientFactoryTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.java new file mode 100644 index 0000000000..f476754eb9 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisClientFactoryTest.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; + +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; +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.ArgumentMatchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class KinesisClientFactoryTest { + 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); + + 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(Region.US_EAST_1); + assertNotNull(kinesisAsyncClient); + + 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/KinesisMultiStreamTrackerTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTrackerTest.java new file mode 100644 index 0000000000..edf23b8033 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisMultiStreamTrackerTest.java @@ -0,0 +1,153 @@ +/* + * 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; +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.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; +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/kinesis/source/KinesisServiceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java new file mode 100644 index 0000000000..12986d9969 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisServiceTest.java @@ -0,0 +1,353 @@ +/* + * 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; +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.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; +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; +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; +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.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; +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.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; +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 String codec_plugin_name = "json"; + + 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; + 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 KinesisClientFactory kinesisClientFactory; + + @Mock + private KinesisAsyncClient kinesisClient; + + @Mock + private DynamoDbAsyncClient dynamoDbClient; + + @Mock + private CloudWatchAsyncClient cloudWatchClient; + + @Mock + Buffer> buffer; + + @Mock + private Scheduler scheduler; + + @Mock + KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; + + @Mock + KinesisLeaseConfig kinesisLeaseConfig; + + @Mock + KinesisLeaseCoordinationTableConfig kinesisLeaseCoordinationTableConfig; + + @Mock + WorkerIdentifierGenerator workerIdentifierGenerator; + + @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); + kinesisClientFactory = mock(KinesisClientFactory.class); + scheduler = mock(Scheduler.class); + pipelineDescription = mock(PipelineDescription.class); + 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"); + 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.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.getCheckPointInterval()).thenReturn(CHECKPOINT_INTERVAL); + when(kinesisStreamConfig.getInitialPosition()).thenReturn(InitialPositionInStream.LATEST); + when(kinesisSourceConfig.getConsumerStrategy()).thenReturn(ConsumerStrategy.ENHANCED_FAN_OUT); + when(kinesisSourceConfig.getPollingConfig()).thenReturn(kinesisStreamPollingConfig); + when(kinesisStreamPollingConfig.getMaxPollingRecords()).thenReturn(DEFAULT_MAX_RECORDS); + when(kinesisStreamPollingConfig.getIdleTimeBetweenReads()).thenReturn(Duration.ofMillis(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); + + 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); + 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, workerIdentifierGenerator); + } + + @Test + 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, workerIdentifierGenerator)); + } + + @Test + void testCreateScheduler() { + KinesisService kinesisService = new KinesisService(kinesisSourceConfig, kinesisClientFactory, pluginMetrics, pluginFactory, + pipelineDescription, acknowledgementSetManager, kinesisLeaseConfigSupplier, workerIdentifierGenerator); + 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()); + 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, workerIdentifierGenerator); + 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()); + verify(workerIdentifierGenerator, times(1)).generate(); + } + + + @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(InvalidPluginConfigurationException.class, () -> kinesisService.start(buffer)); + + verify(scheduler, times(0)).run(); + } + + @Test + void testServiceStartEmptyStreams() { + when(kinesisSourceConfig.getStreams()).thenReturn(new ArrayList<>()); + + KinesisService kinesisService = createObjectUnderTest(); + assertThrows(InvalidPluginConfigurationException.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/kinesis/source/KinesisSourceTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java new file mode 100644 index 0000000000..fad335dd63 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/KinesisSourceTest.java @@ -0,0 +1,190 @@ +/* + * 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; +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; +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; +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; +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; +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.ArgumentMatchers.eq; +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"; + private static final String codec_plugin_name = "json"; + + @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; + + @Mock + KinesisLeaseConfigSupplier kinesisLeaseConfigSupplier; + + @Mock + KinesisLeaseConfig kinesisLeaseConfig; + + @Mock + KinesisLeaseCoordinationTableConfig kinesisLeaseCoordinationTableConfig; + + @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); + + 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); + when(kinesisLeaseConfig.getLeaseCoordinationTable()).thenReturn(kinesisLeaseCoordinationTableConfig); + 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.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); + } + + public KinesisSource createObjectUnderTest() { + return new KinesisSource(kinesisSourceConfig, pluginMetrics, pluginFactory, pipelineDescription, awsCredentialsSupplier, acknowledgementSetManager, kinesisLeaseConfigSupplier); + } + + @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/kinesis/source/configuration/AwsAuthenticationConfigTest.java b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java new file mode 100644 index 0000000000..499711c4a9 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/AwsAuthenticationConfigTest.java @@ -0,0 +1,90 @@ +/* + * 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; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; +import software.amazon.awssdk.regions.Region; + +import java.util.Collections; +import java.util.Map; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.nullValue; + +public class AwsAuthenticationConfigTest { + private ObjectMapper objectMapper = new ObjectMapper(); + + @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()); + } +} 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..2e1b638342 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/InitialPositionInStreamConfigTest.java @@ -0,0 +1,38 @@ +/* + * 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; +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 new file mode 100644 index 0000000000..5846fe4b04 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisSourceConfigTest.java @@ -0,0 +1,161 @@ +/* + * 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; +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.opensearch.dataprepper.pipeline.parser.DataPrepperDurationDeserializer; +import software.amazon.awssdk.regions.Region; +import software.amazon.kinesis.common.InitialPositionInStream; + +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; + +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 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(""); + 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(PIPELINE_CONFIG_WITH_ACKS_ENABLED) + 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(); + assertNotNull(kinesisSourceConfig.getCodec()); + assertEquals(kinesisSourceConfig.getConsumerStrategy(), ConsumerStrategy.ENHANCED_FAN_OUT); + assertNull(kinesisSourceConfig.getPollingConfig()); + + assertEquals(streamConfigs.size(), 3); + + for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { + assertTrue(kinesisStreamConfig.getName().contains("stream")); + assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.LATEST); + assertEquals(kinesisStreamConfig.getCheckPointInterval(), MINIMAL_CHECKPOINT_INTERVAL); + } + } + + @Test + @Tag(PIPELINE_CONFIG_WITH_POLLING_CONFIG_ENABLED) + 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().getIdleTimeBetweenReads(), Duration.ofSeconds(10)); + + assertEquals(streamConfigs.size(), 3); + + for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { + assertTrue(kinesisStreamConfig.getName().contains("stream")); + assertEquals(kinesisStreamConfig.getInitialPosition(), InitialPositionInStream.LATEST); + 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)); + + assertEquals(streamConfigs.size(), 3); + + for (KinesisStreamConfig kinesisStreamConfig: streamConfigs) { + assertTrue(kinesisStreamConfig.getName().contains("stream")); + 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 new file mode 100644 index 0000000000..02ac1960ed --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/configuration/KinesisStreamPollingConfigTest.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + * + * The OpenSearch Contributors require contributions made to + * this file be licensed under the Apache-2.0 license or a + * compatible open source license. + * + */ + +package org.opensearch.dataprepper.plugins.kinesis.source.configuration; + +import org.junit.jupiter.api.Test; + +import java.time.Duration; + +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.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/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..fe0ab06877 --- /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.popLatestReadyToCheckpointRecord(); + assertTrue(checkpointRecord.isEmpty()); + assertEquals(kinesisCheckpointerTracker.size(), numRecords); + + int idx = random.nextInt(numRecords); + ExtendedSequenceNumber extendedSequenceNumber1 = extendedSequenceNumberList.get(idx); + kinesisCheckpointerTracker.markSequenceNumberForCheckpoint(extendedSequenceNumber1); + + Optional firstcheckpointer = kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord(); + 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.popLatestReadyToCheckpointRecord(); + 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.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 new file mode 100644 index 0000000000..ea002e27e9 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisRecordProcessorTest.java @@ -0,0 +1,517 @@ +/* + * 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; +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.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.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; +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 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.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; +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_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; + +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 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; + + @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 + StreamIdentifier streamIdentifier; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + @Mock + private AcknowledgementSet acknowledgementSet; + + @Mock + private Counter recordProcessed; + + @Mock + private Counter recordProcessingErrors; + + @Mock + private Counter checkpointFailures; + + @Mock + 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() { + MockitoAnnotations.initMocks(this); + 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); + 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); + + 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_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 testProcessRecordsWithoutAcknowledgementsWithCheckpointApplied() + throws Exception { + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); + 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); + + 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(kinesisCheckpointerRecord.getCheckpointer()).thenReturn(checkpointer); + when(kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); + kinesisRecordProcessor.initialize(initializationInput); + + kinesisRecordProcessor.processRecords(processRecordsInput); + + 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 + public void testProcessRecordsWithoutAcknowledgementsEnabled() + throws Exception { + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); + 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); + + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); + when(kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord()).thenReturn(Optional.empty()); + kinesisRecordProcessor.initialize(initializationInput); + + kinesisRecordProcessor.processRecords(processRecordsInput); + + 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 testProcessRecordsWithAcknowledgementsEnabled() + throws Exception { + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(true); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); + 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)); + + 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, 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(kinesisCheckpointerRecord.getCheckpointer()).thenReturn(checkpointer); + when(kinesisCheckpointerTracker.popLatestReadyToCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); + kinesisRecordProcessor.initialize(initializationInput); + + kinesisRecordProcessor.processRecords(processRecordsInput); + + 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 { + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + when(kinesisSourceConfig.isAcknowledgments()).thenReturn(false); + when(kinesisStreamConfig.getCheckPointInterval()).thenReturn(Duration.ofMillis(0)); + + 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); + + 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, 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.popLatestReadyToCheckpointRecord()).thenReturn(Optional.of(kinesisCheckpointerRecord)); + kinesisRecordProcessor.initialize(initializationInput); + + kinesisRecordProcessor.processRecords(processRecordsInput); + + 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 { + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + when(kinesisSourceConfig.isAcknowledgments()).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(bufferAccumulator).add(any(Record.class)); + + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); + kinesisRecordProcessor.initialize(initializationInput); + + assertDoesNotThrow(() -> kinesisRecordProcessor.processRecords(processRecordsInput)); + verify(recordProcessingErrors, times(1)).increment(); + verify(recordProcessed, times(0)).increment(anyDouble()); + } + + @Test + void testProcessRecordsBufferFlushNoThrowException() + throws Exception { + List kinesisClientRecords = createInputKinesisClientRecords(); + when(processRecordsInput.records()).thenReturn(kinesisClientRecords); + when(kinesisSourceConfig.isAcknowledgments()).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(bufferAccumulator).flush(); + + kinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); + kinesisRecordProcessor.initialize(initializationInput); + + assertDoesNotThrow(() -> kinesisRecordProcessor.processRecords(processRecordsInput)); + verify(recordProcessingErrors, times(1)).increment(); + verify(recordProcessed, times(0)).increment(anyDouble()); + + } + + @Test + void testShardEndedLatestCheckpoint() { + KinesisRecordProcessor mockKinesisRecordProcessor = new KinesisRecordProcessor(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, 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(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, 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 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, kinesisCheckpointerTracker, 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(bufferAccumulator, kinesisSourceConfig, + acknowledgementSetManager, pluginMetrics, kinesisRecordConverter, kinesisCheckpointerTracker, streamIdentifier); + doThrow(exceptionType).when(checkpointer).checkpoint(eq(sequence_number), eq(sub_sequence_number)); + + assertDoesNotThrow(() -> 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, kinesisCheckpointerTracker, 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())) + .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 new file mode 100644 index 0000000000..9f0a555253 --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/java/org/opensearch/dataprepper/plugins/kinesis/source/processor/KinesisShardRecordProcessorFactoryTest.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.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.kinesis.source.configuration.KinesisSourceConfig; +import org.opensearch.dataprepper.plugins.kinesis.source.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; + + @Mock + private InputCodec codec; + + @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); + when(kinesisSourceConfig.getNumberOfRecordsToAccumulate()).thenReturn(100); + + 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, codec); + assertInstanceOf(KinesisRecordProcessor.class, kinesisShardRecordProcessorFactory.shardRecordProcessor(streamIdentifier)); + } + + @Test + void testKinesisRecordProcessFactoryDefaultUnsupported() { + 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_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..c8b58725fd --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/resources/pipeline_with_checkpoint_enabled.yaml @@ -0,0 +1,17 @@ +source: + kinesis: + streams: + - stream_name: "stream-1" + initial_position: "EARLIEST" + checkpoint_interval: "20s" + - stream_name: "stream-2" + initial_position: "EARLIEST" + checkpoint_interval: "PT15M" + - stream_name: "stream-3" + initial_position: "EARLIEST" + 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_with_extensions.yaml b/data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml new file mode 100644 index 0000000000..4f964cae7f --- /dev/null +++ b/data-prepper-plugins/kinesis-source/src/test/resources/simple_pipeline_with_extensions.yaml @@ -0,0 +1,5 @@ +extensions: + kinesis: + lease_coordination: + table_name: "kinesis-pipeline-kcl" + region: "us-east-1" \ 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