diff --git a/data-prepper-plugins/dynamodb-source/README.md b/data-prepper-plugins/dynamodb-source/README.md new file mode 100644 index 0000000000..1219b9abe5 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/README.md @@ -0,0 +1,83 @@ +# DynamoDB Source + +This is a source plugin that supports retrieve data from DynamoDB tables. Basic use case of this source plugin is to +sync the data from DynamoDB tables to OpenSearch indexes. With this CDC support, customer can run the end to end data +sync pipeline and capture changed data in near real-time without writing any codes and without any downtime of business. +Such pipeline can run on multiple nodes in parallel to support data capture of large scale tables. + +This plugin can support below three different modes: + +1. Full load only: One time full data export and load +2. CDC Only: DynamoDB Stream +3. Full load + CDC: One time full export and load + DynamoDB Stream. + +## Usages + +To get started with this DynamoDB source, create the following source configuration: + +```yaml +source: + dynamodb: + tables: + - table_arn: "arn:aws:dynamodb:us-west-2:123456789012:table/my-table" + stream: + start_position: + export: + s3_bucket: "my-bucket" + s3_prefix: "export/" + aws: + region: "us-west-2" + sts_role_arn: "arn:aws:iam::123456789012:role/DataPrepperRole" + + coordinator: + dynamodb: + table_name: "coordinator-demo" + region: "us-west-2" + + +``` + +## Configurations + +### Shared Configurations: + +* coordinator (Required): Coordination store setting. This design create a custom coordinator based on existing + coordination store implementation. Only DynamoDB is tested so far. +* aws (Required): High level AWS Auth. Note Data Prepper will use the same AWS auth to access all tables, check + Security for more details. + * region + * sts_role_arn + +### Export Configurations: + +* s3_bucket (Required): The destination bucket to store the exported data files +* s3_prefix (Optional): Custom prefix. + +### Stream Configurations + +* start_position (Optional): start position of the stream, can be either BEGINNING or LATEST. If export is required, + this value will be ignored and set to LATEST by default. This is useful if customer don’t want to run initial export, + so they can + choose either from the beginning of the stream (up to 24 hours) or from latest (from the time point when pipeline is + started) + +## Metrics + +### Counter + +- `exportJobsSuccess`: measures total number of export jobs run with status completed. +- `exportJobsErrors`: measures total number of export jobs cannot be submitted or run with status failed. +- `exportFilesTotal`: measures total number of export files generated. +- `exportFilesSuccess`: measures total number of export files read (till the last line) successfully. +- `exportRecordsTotal`: measures total number of export records generated +- `exportRecordsSuccess`: measures total number of export records processed successfully . +- `exportRecordsErrors`: measures total number of export records processed failed +- `changeEventsSucceeded`: measures total number of changed events in total processed successfully +- `changeEventsFailed`: measures total number of changed events in total processed failed + +## Developer Guide + +This plugin is compatible with Java 17. See + +- [CONTRIBUTING](https://github.com/opensearch-project/data-prepper/blob/main/CONTRIBUTING.md) +- [monitoring](https://github.com/opensearch-project/data-prepper/blob/main/docs/monitoring.md) diff --git a/data-prepper-plugins/dynamodb-source/build.gradle b/data-prepper-plugins/dynamodb-source/build.gradle new file mode 100644 index 0000000000..c6f53b9da0 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/build.gradle @@ -0,0 +1,40 @@ +plugins { + id 'java' +} + + +repositories { + mavenCentral() +} + +dependencies { + implementation project(path: ':data-prepper-api') + implementation project(path: ':data-prepper-core') + implementation project(path: ':data-prepper-plugins:dynamodb-source-coordination-store') + + implementation libs.armeria.core + implementation 'io.micrometer:micrometer-core' + + implementation 'software.amazon.awssdk:sts' + implementation 'software.amazon.awssdk:arns' + implementation 'software.amazon.awssdk:dynamodb' + implementation 'software.amazon.awssdk:dynamodb-enhanced' + implementation 'software.amazon.awssdk:s3' + implementation 'software.amazon.awssdk:netty-nio-client' + + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + // https://mvnrepository.com/artifact/com.fasterxml.jackson.dataformat/jackson-dataformat-ion + implementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-ion' + + implementation project(path: ':data-prepper-plugins:aws-plugin-api') + + + testImplementation platform('org.junit:junit-bom:5.9.1') + testImplementation 'org.junit.jupiter:junit-jupiter' + testImplementation 'com.fasterxml.jackson.dataformat:jackson-dataformat-yaml' +} + +test { + useJUnitPlatform() +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/ClientFactory.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/ClientFactory.java new file mode 100644 index 0000000000..867903e8e9 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/ClientFactory.java @@ -0,0 +1,50 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsOptions; +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.AwsAuthenticationConfig; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; +import software.amazon.awssdk.services.s3.S3Client; + +public class ClientFactory { + + private final AwsCredentialsProvider awsCredentialsProvider; + + public ClientFactory(AwsCredentialsSupplier awsCredentialsSupplier, AwsAuthenticationConfig awsAuthenticationConfig) { + awsCredentialsProvider = awsCredentialsSupplier.getProvider(AwsCredentialsOptions.builder() + .withRegion(awsAuthenticationConfig.getAwsRegion()) + .withStsRoleArn(awsAuthenticationConfig.getAwsStsRoleArn()) + .withStsExternalId(awsAuthenticationConfig.getAwsStsExternalId()) + .withStsHeaderOverrides(awsAuthenticationConfig.getAwsStsHeaderOverrides()) + .build()); + } + + + public DynamoDbStreamsClient buildDynamoDbStreamClient() { + return DynamoDbStreamsClient.builder() + .credentialsProvider(awsCredentialsProvider) + .build(); + } + + + public DynamoDbClient buildDynamoDBClient() { + return DynamoDbClient.builder() + .credentialsProvider(awsCredentialsProvider) + .build(); + } + + + public S3Client buildS3Client() { + return S3Client.builder() + .credentialsProvider(awsCredentialsProvider) + .build(); + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBService.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBService.java new file mode 100644 index 0000000000..e27d0ded70 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBService.java @@ -0,0 +1,305 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.TableConfig; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.InitPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.ExportProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.export.DataFileLoaderFactory; +import org.opensearch.dataprepper.plugins.source.dynamodb.export.DataFileScheduler; +import org.opensearch.dataprepper.plugins.source.dynamodb.export.ExportScheduler; +import org.opensearch.dataprepper.plugins.source.dynamodb.export.ManifestFileReader; +import org.opensearch.dataprepper.plugins.source.dynamodb.export.S3ObjectReader; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableMetadata; +import org.opensearch.dataprepper.plugins.source.dynamodb.stream.ShardConsumerFactory; +import org.opensearch.dataprepper.plugins.source.dynamodb.stream.ShardManager; +import org.opensearch.dataprepper.plugins.source.dynamodb.stream.StreamScheduler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.dynamodb.model.DescribeContinuousBackupsRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeContinuousBackupsResponse; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; +import software.amazon.awssdk.services.s3.S3Client; + +import java.time.Instant; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.stream.Collectors; + +public class DynamoDBService { + + private static final Logger LOG = LoggerFactory.getLogger(DynamoDBService.class); + + private final List tableConfigs; + + private final EnhancedSourceCoordinator coordinator; + + private final DynamoDbClient dynamoDbClient; + // + private final DynamoDbStreamsClient dynamoDbStreamsClient; + + private final S3Client s3Client; + + private final ShardManager shardManager; + + private final ExecutorService executor; + + private final PluginMetrics pluginMetrics; + + + public DynamoDBService(EnhancedSourceCoordinator coordinator, ClientFactory clientFactory, DynamoDBSourceConfig sourceConfig, PluginMetrics pluginMetrics) { + this.coordinator = coordinator; + this.pluginMetrics = pluginMetrics; + + // Initialize AWS clients + dynamoDbClient = clientFactory.buildDynamoDBClient(); + dynamoDbStreamsClient = clientFactory.buildDynamoDbStreamClient(); + s3Client = clientFactory.buildS3Client(); + + // A shard manager is responsible to retrieve the shard information from streams. + shardManager = new ShardManager(dynamoDbStreamsClient); + tableConfigs = sourceConfig.getTableConfigs(); + executor = Executors.newFixedThreadPool(3); + } + + /** + * This service start three long-running threads (scheduler) + * Each thread is responsible for one type of job. + * The data will be guaranteed to be sent to {@link Buffer} in order. + * + * @param buffer Data Prepper Buffer + */ + public void start(Buffer> buffer) { + + LOG.info("Start running DynamoDB service"); + ManifestFileReader manifestFileReader = new ManifestFileReader(new S3ObjectReader(s3Client)); + Runnable exportScheduler = new ExportScheduler(coordinator, dynamoDbClient, manifestFileReader, pluginMetrics); + + DataFileLoaderFactory loaderFactory = new DataFileLoaderFactory(coordinator, s3Client, pluginMetrics, buffer); + Runnable fileLoaderScheduler = new DataFileScheduler(coordinator, loaderFactory, pluginMetrics); + + ShardConsumerFactory consumerFactory = new ShardConsumerFactory(coordinator, dynamoDbStreamsClient, pluginMetrics, shardManager, buffer); + Runnable streamScheduler = new StreamScheduler(coordinator, consumerFactory, shardManager); + + // May consider start or shutdown the scheduler on demand + // Currently, event after the exports are done, the related scheduler will not be shutdown + // This is because in the future we may support incremental exports. + executor.submit(exportScheduler); + executor.submit(fileLoaderScheduler); + executor.submit(streamScheduler); + + } + + /** + * Interrupt the running of schedulers. + * Each scheduler must implement logic for gracefully shutdown. + */ + public void shutdown() { + LOG.info("shutdown DynamoDB schedulers"); + executor.shutdownNow(); + } + + /** + * Perform initialization of the service from pipeline configuration + * The initialization is currently performed once per pipeline. + * Hence, the configuration change after first initialization process will be ignored. + * This is controlled by a lease in the coordination table. + * Future optimization can be done to accept configuration changes + */ + public void init() { + LOG.info("Start initialize DynamoDB service"); + + final Optional initPartition = coordinator.acquireAvailablePartition(InitPartition.PARTITION_TYPE); + if (initPartition.isEmpty()) { + // Already initialized. Do nothing. + return; + } + + LOG.info("Start validating table configurations"); + List tableInfos; + try { + tableInfos = tableConfigs.stream().map(this::getTableInfo).collect(Collectors.toList()); + } catch (Exception e) { + coordinator.giveUpPartition(initPartition.get()); + throw e; + } + + tableInfos.forEach(tableInfo -> { + // Create a Global state in the coordination table for the configuration. + // Global State here is designed to be able to read whenever needed + // So that the jobs can refer to the configuration. + coordinator.createPartition(new GlobalState(tableInfo.getTableArn(), Optional.of(tableInfo.getMetadata().toMap()))); + + Instant startTime = Instant.now(); + + if (tableInfo.getMetadata().isExportRequired()) { +// exportTime = Instant.now(); + createExportPartition(tableInfo.getTableArn(), startTime, tableInfo.getMetadata().getExportBucket(), tableInfo.getMetadata().getExportPrefix()); + } + + if (tableInfo.getMetadata().isStreamRequired()) { + List shardIds; + // start position by default is beginning if not provided. + if (tableInfo.getMetadata().isExportRequired() || "LATEST".equals(tableInfo.getMetadata().getStreamStartPosition())) { + // For a continued data extraction process that involves both export and stream + // The export must be completed and loaded before stream can start. + // Moreover, there should not be any gaps between the export time and the time start reading the stream + // The design here is to start reading from the beginning of current active shards + // and then check if the change event datetime is greater than the export time. + shardIds = shardManager.getActiveShards(tableInfo.getMetadata().getStreamArn()); + shardIds.forEach(shardId -> { + createStreamPartition(tableInfo.getMetadata().getStreamArn(), shardId, startTime, tableInfo.getMetadata().isExportRequired()); + }); + } else { + shardIds = shardManager.getRootShardIds(tableInfo.getMetadata().getStreamArn()); + shardIds.forEach(shardId -> { + createStreamPartition(tableInfo.getMetadata().getStreamArn(), shardId, null, false); + }); + } + } + }); + // Mark initialization as done, so that it won't be triggered again. + coordinator.completePartition(initPartition.get()); + } + + + /** + * Create a partition for an export job in the coordination table. The bucket and prefix will be stored in the progress state. + * This is to support that different tables can use different destinations. + * + * @param tableArn Table Arn + * @param exportTime Export Time + * @param bucket Export bucket + * @param prefix Export Prefix + */ + private void createExportPartition(String tableArn, Instant exportTime, String bucket, String prefix) { + ExportProgressState exportProgressState = new ExportProgressState(); + exportProgressState.setBucket(bucket); + exportProgressState.setPrefix(prefix); + exportProgressState.setExportTime(exportTime.toString()); // information purpose + ExportPartition exportPartition = new ExportPartition(tableArn, exportTime, Optional.of(exportProgressState)); + coordinator.createPartition(exportPartition); + } + + + /** + * Create a partition for a stream job in the coordination table. + * + * @param streamArn Stream Arn + * @param shardId Shard Id + * @param exportTime the start time for change events, any change events with creation datetime before this should be ignored. + */ + private void createStreamPartition(String streamArn, String shardId, Instant exportTime, boolean waitForExport) { + StreamProgressState streamProgressState = new StreamProgressState(); + streamProgressState.setWaitForExport(waitForExport); + if (exportTime != null) { + streamProgressState.setStartTime(exportTime.toEpochMilli()); + } + coordinator.createPartition(new StreamPartition(streamArn, shardId, Optional.of(streamProgressState))); + } + + private String getContinuousBackupsStatus(String tableName) { + // Validate Point in time recovery is enabled or not + DescribeContinuousBackupsRequest req = DescribeContinuousBackupsRequest.builder() + .tableName(tableName) + .build(); + DescribeContinuousBackupsResponse resp = dynamoDbClient.describeContinuousBackups(req); + return resp.continuousBackupsDescription().pointInTimeRecoveryDescription().pointInTimeRecoveryStatus().toString(); + } + + private String getTableName(String tableArn) { + Arn arn = Arn.fromString(tableArn); + // resourceAsString is table/xxx + return arn.resourceAsString().substring(6); + } + + /** + * Conduct Metadata info for table and also perform validation on configuration. + * Once created, the info should not be changed. + */ + private TableInfo getTableInfo(TableConfig tableConfig) { + String tableName = getTableName(tableConfig.getTableArn()); + + // Need to call describe table to get the Key schema for table + // The key schema will be used when adding the metadata to event. + DescribeTableRequest req = DescribeTableRequest.builder() + .tableName(tableName) + .build(); + DescribeTableResponse describeTableResult = dynamoDbClient.describeTable(req); + Map keys = describeTableResult.table().keySchema().stream().collect(Collectors.toMap( + e -> e.keyTypeAsString(), e -> e.attributeName() + )); + + // Validate if PITR is turn on or not for exports. + if (tableConfig.getExportConfig() != null) { + String status = getContinuousBackupsStatus(tableName); + LOG.debug("The PITR status for table " + tableName + " is " + status); + if (!"ENABLED".equals(status)) { + String errorMessage = "Point-in-time recovery (PITR) needs to be enabled for exporting data from table " + tableConfig.getTableArn(); + LOG.error(errorMessage); + throw new InvalidPluginConfigurationException(errorMessage); + } + } + + String streamStartPosition = null; + + if (tableConfig.getStreamConfig() != null) { + // Validate if DynamoDB Stream is turn on or not + if (describeTableResult.table().streamSpecification() == null) { + String errorMessage = "Steam is not enabled for table " + tableConfig.getTableArn(); + LOG.error(errorMessage); + throw new InvalidPluginConfigurationException(errorMessage); + } + // Validate view type of DynamoDB stream + if (describeTableResult.table().streamSpecification() != null) { + String viewType = describeTableResult.table().streamSpecification().streamViewTypeAsString(); + LOG.debug("The stream view type for table " + tableName + " is " + viewType); + List supportedType = List.of("NEW_IMAGE", "NEW_AND_OLD_IMAGES"); + if (!supportedType.contains(viewType)) { + String errorMessage = "Stream " + tableConfig.getTableArn() + " is enabled with " + viewType + ". Supported types are " + supportedType; + LOG.error(errorMessage); + throw new InvalidPluginConfigurationException(errorMessage); + } + } + streamStartPosition = tableConfig.getStreamConfig().getStartPosition(); + } + + // Conduct metadata info + // May consider to remove export bucket and prefix + TableMetadata metadata = TableMetadata.builder() + .partitionKeyAttributeName(keys.get("HASH")) + .sortKeyAttributeName(keys.get("RANGE")) + .streamArn(describeTableResult.table().latestStreamArn()) + .streamRequired(tableConfig.getStreamConfig() != null) + .exportRequired(tableConfig.getExportConfig() != null) + .streamStartPosition(streamStartPosition) + .exportBucket(tableConfig.getExportConfig() == null ? null : tableConfig.getExportConfig().getS3Bucket()) + .exportPrefix(tableConfig.getExportConfig() == null ? null : tableConfig.getExportConfig().getS3Prefix()) + .build(); + return new TableInfo(tableConfig.getTableArn(), metadata); + } + + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSource.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSource.java new file mode 100644 index 0000000000..16ca5153f9 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSource.java @@ -0,0 +1,89 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb; + +import org.opensearch.dataprepper.aws.api.AwsCredentialsSupplier; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.configuration.PluginSetting; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.PluginFactory; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.model.source.SourceCoordinationStore; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.DefaultEnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.PartitionFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; + +@DataPrepperPlugin(name = "dynamodb", pluginType = Source.class, pluginConfigurationType = DynamoDBSourceConfig.class) +public class DynamoDBSource implements Source> { + + private static final Logger LOG = LoggerFactory.getLogger(DynamoDBSource.class); + + private static final String SOURCE_COORDINATOR_METRIC_PREFIX = "source-coordinator"; + + private final PluginMetrics pluginMetrics; + + private final DynamoDBSourceConfig sourceConfig; + + private final PluginFactory pluginFactory; + + private final SourceCoordinationStore coordinationStore; + + private final EnhancedSourceCoordinator coordinator; + + private final DynamoDBService dynamoDBService; + + + @DataPrepperPluginConstructor + public DynamoDBSource(PluginMetrics pluginMetrics, final DynamoDBSourceConfig sourceConfig, final PluginFactory pluginFactory, final PluginSetting pluginSetting, final AwsCredentialsSupplier awsCredentialsSupplier) { + LOG.info("Create DynamoDB Source"); + this.pluginMetrics = pluginMetrics; + this.sourceConfig = sourceConfig; + this.pluginFactory = pluginFactory; + + + // Load Coordination Store via PluginFactory + // This part will be updated. + PluginSetting sourceCoordinationStoreSetting = new PluginSetting(sourceConfig.getCoordinationStoreConfig().getPluginName(), sourceConfig.getCoordinationStoreConfig().getPluginSettings()); + sourceCoordinationStoreSetting.setPipelineName(SOURCE_COORDINATOR_METRIC_PREFIX); + coordinationStore = pluginFactory.loadPlugin(SourceCoordinationStore.class, sourceCoordinationStoreSetting); + String pipelineName = pluginSetting.getPipelineName(); + + // Create and initialize coordinator + coordinator = new DefaultEnhancedSourceCoordinator(coordinationStore, pipelineName, new PartitionFactory()); + coordinator.initialize(); + + ClientFactory clientFactory = new ClientFactory(awsCredentialsSupplier, sourceConfig.getAwsAuthenticationConfig()); + + // Create DynamoDB Service + dynamoDBService = new DynamoDBService(coordinator, clientFactory, sourceConfig, pluginMetrics); + dynamoDBService.init(); + } + + @Override + public void start(Buffer> buffer) { + LOG.info("Start DynamoDB service"); + dynamoDBService.start(buffer); + } + + + @Override + public void stop() { + LOG.info("Stop DynamoDB Source"); + if (Objects.nonNull(dynamoDBService)) { + dynamoDBService.shutdown(); + } + + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfig.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfig.java new file mode 100644 index 0000000000..379f837767 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfig.java @@ -0,0 +1,52 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import org.opensearch.dataprepper.model.configuration.PluginModel; +import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.TableConfig; + +import java.util.List; + +/** + * Configuration for DynamoDB Source + */ +public class DynamoDBSourceConfig { + + @JsonProperty("tables") + private List tableConfigs; + + + @JsonProperty("aws") + @NotNull + @Valid + private AwsAuthenticationConfig awsAuthenticationConfig; + + + @JsonProperty("coordinator") + private PluginModel coordinationStoreConfig; + + + public DynamoDBSourceConfig() { + } + + + public List getTableConfigs() { + return tableConfigs; + } + + public AwsAuthenticationConfig getAwsAuthenticationConfig() { + return awsAuthenticationConfig; + } + + public PluginModel getCoordinationStoreConfig() { + return coordinationStoreConfig; + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/AwsAuthenticationConfig.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/AwsAuthenticationConfig.java new file mode 100644 index 0000000000..124806e0c7 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/AwsAuthenticationConfig.java @@ -0,0 +1,48 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.regions.Region; + +import java.util.Map; + +public class AwsAuthenticationConfig { + + @JsonProperty("region") + @Size(min = 1, message = "Region cannot be empty string") + private String awsRegion; + + @JsonProperty("sts_role_arn") + @Size(min = 20, max = 2048, message = "awsStsRoleArn length should be between 1 and 2048 characters") + private String awsStsRoleArn; + + @JsonProperty("sts_external_id") + @Size(min = 2, max = 1224, message = "awsStsExternalId length should be between 2 and 1224 characters") + private String awsStsExternalId; + + @JsonProperty("sts_header_overrides") + @Size(max = 5, message = "sts_header_overrides supports a maximum of 5 headers to override") + private Map awsStsHeaderOverrides; + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Region getAwsRegion() { + return awsRegion != null ? Region.of(awsRegion) : null; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } +} + diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/ExportConfig.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/ExportConfig.java new file mode 100644 index 0000000000..cb3463a3b6 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/ExportConfig.java @@ -0,0 +1,26 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotBlank; + +public class ExportConfig { + + @JsonProperty("s3_bucket") + @NotBlank(message = "Bucket Name is required for export") + private String s3Bucket; + @JsonProperty("s3_prefix") + private String s3Prefix; + + public String getS3Bucket() { + return s3Bucket; + } + + public String getS3Prefix() { + return s3Prefix; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/StreamConfig.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/StreamConfig.java new file mode 100644 index 0000000000..fcbc433a13 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/StreamConfig.java @@ -0,0 +1,19 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class StreamConfig { + + @JsonProperty(value = "start_position") + private String startPosition; + + public String getStartPosition() { + return startPosition; + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/TableConfig.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/TableConfig.java new file mode 100644 index 0000000000..d7c90ea1dd --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/configuration/TableConfig.java @@ -0,0 +1,37 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.NotEmpty; +import jakarta.validation.constraints.NotNull; + +public class TableConfig { + + @JsonProperty("table_arn") + @NotNull + @NotEmpty(message = "DynamoDB Table ARN cannot be null or empty") + private String tableArn; + + @JsonProperty("export") + private ExportConfig exportConfig; + + @JsonProperty(value = "stream") + private StreamConfig streamConfig; + + + public String getTableArn() { + return tableArn; + } + + public ExportConfig getExportConfig() { + return exportConfig; + } + + public StreamConfig getStreamConfig() { + return streamConfig; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/ExportRecordConverter.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/ExportRecordConverter.java new file mode 100644 index 0000000000..b53d5df2b6 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/ExportRecordConverter.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.converter; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.dataformat.ion.IonObjectMapper; +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class ExportRecordConverter extends RecordConverter { + + private static final Logger LOG = LoggerFactory.getLogger(ExportRecordConverter.class); + + private static final String ITEM_KEY = "Item"; + + static final String EXPORT_RECORD_SUCCESS_COUNT = "exportRecordSuccess"; + static final String EXPORT_RECORD_ERROR_COUNT = "exportRecordErrors"; + + + IonObjectMapper MAPPER = new IonObjectMapper(); + + private final PluginMetrics pluginMetrics; + + private final Counter exportRecordSuccessCounter; + private final Counter exportRecordErrorCounter; + + public ExportRecordConverter(Buffer> buffer, TableInfo tableInfo, PluginMetrics pluginMetrics) { + super(buffer, tableInfo); + this.pluginMetrics = pluginMetrics; + this.exportRecordSuccessCounter = pluginMetrics.counter(EXPORT_RECORD_SUCCESS_COUNT); + this.exportRecordErrorCounter = pluginMetrics.counter(EXPORT_RECORD_ERROR_COUNT); + + } + + private Map convertToMap(String jsonData) { + try { + return MAPPER.readValue(jsonData, Map.class); + } catch (JsonProcessingException e) { + return null; + } + } + + + @Override + String getEventType() { + return "EXPORT"; + } + + public void writeToBuffer(List lines) { + List> data = lines.stream() + .map(this::convertToMap) + .map(d -> (Map) d.get(ITEM_KEY)) + .collect(Collectors.toList()); + + List> events = data.stream().map(this::convertToEvent).collect(Collectors.toList()); + + try { + writeEventsToBuffer(events); + exportRecordSuccessCounter.increment(events.size()); + } catch (Exception e) { + LOG.error("Failed to write {} events to buffer due to {}", events.size(), e.getMessage()); + exportRecordErrorCounter.increment(events.size()); + } + } + + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/RecordConverter.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/RecordConverter.java new file mode 100644 index 0000000000..a280824973 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/RecordConverter.java @@ -0,0 +1,92 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.converter; + +import org.opensearch.dataprepper.model.buffer.Buffer; +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.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; + +import java.time.Instant; +import java.util.List; +import java.util.Map; + +/** + * Base Record Processor definition. + * The record processor is to transform the source data into a JacksonEvent, + * and then write to buffer. + */ +public abstract class RecordConverter { + + + private static final String KEYS_TAG_NAME = "_id"; + + private static final String EVENT_TIMESTAMP_TAG_NAME = "ts"; + + private static final String EVENT_OP_TAG_NAME = "op"; + + private static final String EVENT_SOURCE_TAG_NAME = "source"; + + private static final String DEFAULT_ACTION = "index"; + + private static final int DEFAULT_WRITE_TIMEOUT_MILLIS = 60_000; + + private final Buffer> buffer; + + private final TableInfo tableInfo; + + public RecordConverter(Buffer> buffer, TableInfo tableInfo) { + this.buffer = buffer; + this.tableInfo = tableInfo; + } + + + abstract String getEventType(); + + /** + * Default method to conduct the document ID value, + * Using partition key plus sort key (if any) + */ + String getId(Map data) { + String partitionKey = String.valueOf(data.get(tableInfo.getMetadata().getPartitionKeyAttributeName())); + if (tableInfo.getMetadata().getSortKeyAttributeName() == null) { + return partitionKey; + } + String sortKey = String.valueOf(data.get(tableInfo.getMetadata().getSortKeyAttributeName())); + return partitionKey + "_" + sortKey; + } + + void writeEventsToBuffer(List> events) throws Exception { + buffer.writeAll(events, DEFAULT_WRITE_TIMEOUT_MILLIS); + } + + public Record convertToEvent(Map data, Instant eventCreationTime, String action) { + Event event; + event = JacksonEvent.builder() + .withEventType(getEventType()) + .withData(data) + .build(); + EventMetadata eventMetadata = event.getMetadata(); + + eventMetadata.setAttribute(EVENT_SOURCE_TAG_NAME, tableInfo.getTableArn()); + if (eventCreationTime != null) { + eventMetadata.setAttribute(EVENT_TIMESTAMP_TAG_NAME, eventCreationTime.toEpochMilli()); + } + + eventMetadata.setAttribute(EVENT_OP_TAG_NAME, action); + eventMetadata.setAttribute(KEYS_TAG_NAME, getId(data)); + + return new Record<>(event); + } + + public Record convertToEvent(Map data) { + return convertToEvent(data, null, DEFAULT_ACTION); + } + + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/StreamRecordConverter.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/StreamRecordConverter.java new file mode 100644 index 0000000000..2ba6fdf45e --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/StreamRecordConverter.java @@ -0,0 +1,80 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.converter; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.enhanced.dynamodb.document.EnhancedDocument; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class StreamRecordConverter extends RecordConverter { + private static final Logger LOG = LoggerFactory.getLogger(StreamRecordConverter.class); + + + static final String CHANGE_EVENT_SUCCESS_COUNT = "changeEventSuccess"; + static final String CHANGE_EVENT_ERROR_COUNT = "changeEventErrors"; + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference>() { + }; + + private final PluginMetrics pluginMetrics; + + private final Counter changeEventSuccessCounter; + private final Counter changeEventErrorCounter; + + public StreamRecordConverter(Buffer> buffer, TableInfo tableInfo, PluginMetrics pluginMetrics) { + super(buffer, tableInfo); + this.pluginMetrics = pluginMetrics; + this.changeEventSuccessCounter = pluginMetrics.counter(CHANGE_EVENT_SUCCESS_COUNT); + this.changeEventErrorCounter = pluginMetrics.counter(CHANGE_EVENT_ERROR_COUNT); + } + + @Override + String getEventType() { + return "STREAM"; + } + + public void writeToBuffer(List records) { + // TODO: What if convert failed. + List> events = records.stream() + .map(record -> convertToEvent( + toMap(EnhancedDocument.fromAttributeValueMap(record.dynamodb().newImage()).toJson()), + record.dynamodb().approximateCreationDateTime(), + record.eventNameAsString())) + .collect(Collectors.toList()); + + try { + writeEventsToBuffer(events); + changeEventSuccessCounter.increment(events.size()); + } catch (Exception e) { + LOG.error("Failed to write {} events to buffer due to {}", events.size(), e.getMessage()); + changeEventErrorCounter.increment(events.size()); + } + } + + + private Map toMap(String jsonData) { + try { + return MAPPER.readValue(jsonData, MAP_TYPE_REFERENCE); + } catch (JsonProcessingException e) { + return null; + } + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/DefaultEnhancedSourceCoordinator.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/DefaultEnhancedSourceCoordinator.java new file mode 100644 index 0000000000..743a9c1a81 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/DefaultEnhancedSourceCoordinator.java @@ -0,0 +1,230 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination; + +import org.opensearch.dataprepper.model.source.SourceCoordinationStore; +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStatus; +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.InitPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.time.Duration; +import java.time.Instant; +import java.util.Objects; +import java.util.Optional; +import java.util.function.Function; + + +/** + * An implemetation of {@link EnhancedSourceCoordinator} backend by {@link SourceCoordinationStore} + */ +public class DefaultEnhancedSourceCoordinator implements EnhancedSourceCoordinator { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultEnhancedSourceCoordinator.class); + + /** + * Default time out duration for lease. + */ + private static final Duration DEFAULT_LEASE_TIMEOUT = Duration.ofMinutes(10); + + /** + * Default identifier For global state + */ + private static final String DEFAULT_GLOBAL_STATE_PARTITION_TYPE = "GLOBAL"; + + /** + * A backend coordination store + */ + private final SourceCoordinationStore coordinationStore; + + /** + * A unique identifier for a source, normally the pipeline name/id. + * As the coordination store may be shared in different pipelines. + */ + private final String sourceIdentifier; + + /** + * In order to support different types of partitions. + * A custom factory is required to map a {@link SourcePartitionStoreItem} to a {@link SourcePartition} + */ + private final Function partitionFactory; + + /** + * Use host name of the node as the default ownerId + */ + static final String hostName; + + + static { + try { + hostName = InetAddress.getLocalHost().getHostName(); + } catch (final UnknownHostException e) { + throw new RuntimeException(e); + } + } + + + public DefaultEnhancedSourceCoordinator(final SourceCoordinationStore coordinationStore, String sourceIdentifier, Function partitionFactory) { + this.coordinationStore = coordinationStore; + this.sourceIdentifier = sourceIdentifier; + this.partitionFactory = partitionFactory; + + } + + @Override + public void initialize() { + coordinationStore.initializeStore(); + createPartition(new InitPartition()); + } + + @Override + public boolean createPartition(SourcePartition partition) { + String partitionType = partition.getPartitionType() == null ? DEFAULT_GLOBAL_STATE_PARTITION_TYPE : partition.getPartitionType(); + // Don't need the status for Global state which is not for lease. + SourcePartitionStatus status = partition.getPartitionType() == null ? null : SourcePartitionStatus.UNASSIGNED; + + boolean partitionCreated = coordinationStore.tryCreatePartitionItem( + this.sourceIdentifier + "|" + partitionType, + partition.getPartitionKey(), + status, + 0L, + partition.convertPartitionProgressStatetoString(partition.getProgressState()) + ); + return partitionCreated; + + } + + + @Override + public Optional acquireAvailablePartition(String partitionType) { + // Not available for global state. + Objects.nonNull(partitionType); + LOG.debug("Try to acquire an available {} partition", partitionType); + Optional sourceItem = coordinationStore.tryAcquireAvailablePartition(this.sourceIdentifier + "|" + partitionType, hostName, DEFAULT_LEASE_TIMEOUT); + if (sourceItem.isEmpty()) { + LOG.info("Partition owner {} failed to acquire a partition, no available {} partitions now", hostName, partitionType); + return Optional.empty(); + } + + return Optional.of(partitionFactory.apply(sourceItem.get())); + } + + + @Override + public void saveProgressStateForPartition(SourcePartition partition) { + String partitionType = partition.getPartitionType() == null ? DEFAULT_GLOBAL_STATE_PARTITION_TYPE : partition.getPartitionType(); + LOG.debug("Try to save progress for partition {} (Type {})", partition.getPartitionKey(), partitionType); + + // + if (partition.getSourcePartitionStoreItem() == null) { + LOG.error("Unable to save progress, the item was not found."); + return; + } + + final SourcePartitionStoreItem updateItem = partition.getSourcePartitionStoreItem(); + // Also extend the timeout of the lease (ownership) + if (updateItem.getPartitionOwnershipTimeout() != null) { + updateItem.setPartitionOwnershipTimeout(Instant.now().plus(DEFAULT_LEASE_TIMEOUT)); + } + updateItem.setPartitionProgressState(partition.convertPartitionProgressStatetoString(partition.getProgressState())); + + coordinationStore.tryUpdateSourcePartitionItem(updateItem); + LOG.info("Progress for for partition {} (Type {}) was saved", partition.getPartitionKey(), partitionType); + } + + @Override + public void giveUpPartition(SourcePartition partition) { + Objects.nonNull(partition.getPartitionType()); + + LOG.debug("Try to give up the ownership for partition {} (Type {})", partition.getPartitionKey(), partition.getPartitionType()); + + if (partition.getSourcePartitionStoreItem() == null) { + LOG.error("Unable to give up ownership, the item was not found."); + return; + } + + final SourcePartitionStoreItem updateItem = partition.getSourcePartitionStoreItem(); + // Clear the ownership and reset status. + updateItem.setSourcePartitionStatus(SourcePartitionStatus.UNASSIGNED); + updateItem.setPartitionOwner(null); + updateItem.setPartitionOwnershipTimeout(null); + + // Throws UpdateException if update failed. + coordinationStore.tryUpdateSourcePartitionItem(updateItem); + LOG.info("Partition key {} was given up by owner {}", partition.getPartitionKey(), hostName); + + } + + @Override + public void completePartition(SourcePartition partition) { + Objects.nonNull(partition.getPartitionType()); + + LOG.debug("Try to complete partition {} (Type {})", partition.getPartitionKey(), partition.getPartitionType()); + + if (partition.getSourcePartitionStoreItem() == null) { + LOG.error("Unable to complete, the item is not found."); + return; + } + + SourcePartitionStoreItem updateItem = partition.getSourcePartitionStoreItem(); + updateItem.setPartitionOwner(null); + updateItem.setReOpenAt(null); + updateItem.setPartitionOwnershipTimeout(null); + updateItem.setSourcePartitionStatus(SourcePartitionStatus.COMPLETED); + + updateItem.setPartitionProgressState(partition.convertPartitionProgressStatetoString(partition.getProgressState())); + + // Throws UpdateException if update failed. + coordinationStore.tryUpdateSourcePartitionItem(updateItem); + } + + @Override + public void closePartition(SourcePartition partition, final Duration reopenAfter, final int maxClosedCount) { + + Objects.nonNull(partition.getPartitionType()); + + LOG.debug("Try to close partition {} (Type {})", partition.getPartitionKey(), partition.getPartitionType()); + if (partition.getSourcePartitionStoreItem() == null) { + LOG.error("Unable to close, the item is not found."); + return; + } + + SourcePartitionStoreItem updateItem = partition.getSourcePartitionStoreItem(); + + // Reset ownership + updateItem.setPartitionOwner(null); + updateItem.setPartitionOwnershipTimeout(null); + updateItem.setPartitionProgressState(partition.convertPartitionProgressStatetoString(partition.getProgressState())); + + updateItem.setClosedCount(updateItem.getClosedCount() + 1L); + if (updateItem.getClosedCount() >= maxClosedCount) { + updateItem.setSourcePartitionStatus(SourcePartitionStatus.COMPLETED); + updateItem.setReOpenAt(null); + } else { + updateItem.setSourcePartitionStatus(SourcePartitionStatus.CLOSED); + updateItem.setReOpenAt(Instant.now().plus(reopenAfter)); + } + + // Throws UpdateException if update failed. + coordinationStore.tryUpdateSourcePartitionItem(updateItem); + } + + + @Override + public Optional getPartition(String partitionKey) { + // Default to Global State only. + final Optional sourceItem = coordinationStore.getSourcePartitionItem(this.sourceIdentifier + "|" + DEFAULT_GLOBAL_STATE_PARTITION_TYPE, partitionKey); + if (!sourceItem.isPresent()) { + LOG.error("Global state {} is not found.", partitionKey); + return Optional.empty(); + } + return Optional.of(partitionFactory.apply(sourceItem.get())); + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/EnhancedSourceCoordinator.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/EnhancedSourceCoordinator.java new file mode 100644 index 0000000000..3372586a43 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/EnhancedSourceCoordinator.java @@ -0,0 +1,103 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination; + +import java.time.Duration; +import java.util.Optional; + +/** + * A custom Lease based Coordinator Interface to enable distribution and coordination of work for the DynamoDB source + * and to track progress and save state for these partitions of work. + *

+ * Still trying to make this interface generic, considering that it may be merged with {@link org.opensearch.dataprepper.model.source.coordinator.SourceCoordinator} in the future. + *

+ * The major differences include: + *
    + *
  • Support multiple types of partition
  • + *
  • Support multiple leases at the same time
  • + *
+ */ +public interface EnhancedSourceCoordinator { + + /** + * This method is used to create the partition item in the coordination store. + * + * @param partition A specific partition that extends {@link SourcePartition} + * @param The progress state class + * @return True if partition is created successfully otherwise false. + */ + boolean createPartition(SourcePartition partition); + + + /** + * This method is used to acquire a lease on the partition item in the coordination store. + * + * @param partitionType The partition type identifier + * @return A {@link SourcePartition} instance + */ + Optional acquireAvailablePartition(String partitionType); + + /** + * This method is used to update progress state for a partition in the coordination store. + * It will also extend the timeout for ownership. + * + * @param partition The partition to be updated. + * @param The progress state class + * @throws org.opensearch.dataprepper.model.source.coordinator.exceptions.PartitionUpdateException when the partition was not updated successfully + */ + void saveProgressStateForPartition(SourcePartition partition); + + /** + * This method is used to release the lease of a partition in the coordination store. + * The progress state will also be updated. + * + * @param partition The partition to be updated. + * @param The progress state class + * @throws org.opensearch.dataprepper.model.source.coordinator.exceptions.PartitionUpdateException when the partition was not updated successfully + */ + void giveUpPartition(SourcePartition partition); + + /** + * This method is used to mark a partition as COMPLETED in the coordination store. + * The progress state will also be updated. + * + * @param partition The partition to be updated. + * @param The progress state class + * @throws org.opensearch.dataprepper.model.source.coordinator.exceptions.PartitionUpdateException when the partition was not updated successfully + */ + void completePartition(SourcePartition partition); + + /** + * This method is used to mark a partition as CLOSED in the coordination store. + * The closed partition will not be available for lease until reopen. + * The progress state will also be updated. + * + * @param partition The partition to be updated + * @param reopenAfter The duration from the current time to wait before this partition should be processed further at a later date + * @param maxClosedCount The number of times to allow this partition to be closed. Will mark the partition as completed if the partition has been closed this many times or more + * in the past + * @param The progress state class + * @throws org.opensearch.dataprepper.model.source.coordinator.exceptions.PartitionUpdateException when the partition was not updated successfully + */ + void closePartition(SourcePartition partition, final Duration reopenAfter, final int maxClosedCount); + + + /** + * This method is used to get a partition from the coordination store. + * Unlike acquiring, this does not add ownership to the item. + * Hence, it's designed to be used as a "Global State" which can be read whenever needed. + * + * @param partitionKey A unique key for that partition + * @return A {@link SourcePartition} instance + */ + Optional getPartition(String partitionKey); + + /** + * This method is to perform initialization for the coordinator + */ + void initialize(); + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/Partition.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/Partition.java new file mode 100644 index 0000000000..2f6f51c8af --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/Partition.java @@ -0,0 +1,22 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination; + +import java.util.Optional; + +/** + * A Partition Interface represents an item in the coordination store. + */ +public interface Partition { + + + String getPartitionType(); + + String getPartitionKey(); + + Optional getProgressState(); + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/PartitionFactory.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/PartitionFactory.java new file mode 100644 index 0000000000..1d154eb17e --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/PartitionFactory.java @@ -0,0 +1,43 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.InitPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.StreamPartition; + +import java.util.function.Function; + +/** + * Special partition factory just for this DynamoDB source. + */ +public class PartitionFactory implements Function { + + + @Override + public SourcePartition apply(SourcePartitionStoreItem partitionStoreItem) { + String sourceIdentifier = partitionStoreItem.getSourceIdentifier(); + String partitionType = sourceIdentifier.substring(sourceIdentifier.lastIndexOf('|') + 1); + + if (ExportPartition.PARTITION_TYPE.equals(partitionType)) { + return new ExportPartition(partitionStoreItem); + } else if (StreamPartition.PARTITION_TYPE.equals(partitionType)) { + return new StreamPartition(partitionStoreItem); + } else if (DataFilePartition.PARTITION_TYPE.equals(partitionType)) { + return new DataFilePartition(partitionStoreItem); + } else if (InitPartition.PARTITION_TYPE.equals(partitionType)) { + return new InitPartition(partitionStoreItem); + } else { + // Unable to acquire other partitions. + return new GlobalState(partitionStoreItem); + } + } + + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/SourcePartition.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/SourcePartition.java new file mode 100644 index 0000000000..d26320666e --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/SourcePartition.java @@ -0,0 +1,86 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.Optional; + +/** + *

+ * A base definition of a {@link Partition} in the coordination store. + * All partitions must extend this. + *

+ * We store the {SourcePartitionStoreItem} in the partition. + * The benefits are: + *
    + *
  • Don't have to query again before each updates
  • + *
  • Can perform Optimistic locking on updates.
  • + *
+ * Future improvement may be made for this. As we don't have access to Version Number. + * + * @param The progress state class + */ +public abstract class SourcePartition implements Partition { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultEnhancedSourceCoordinator.class); + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private SourcePartitionStoreItem sourcePartitionStoreItem; + + protected SourcePartitionStoreItem getSourcePartitionStoreItem() { + return sourcePartitionStoreItem; + } + + + protected void setSourcePartitionStoreItem(SourcePartitionStoreItem sourcePartitionStoreItem) { + this.sourcePartitionStoreItem = sourcePartitionStoreItem; + } + + + /** + * Helper method to convert progress state. + * This is because the state is currently stored as a String in the coordination store. + */ + protected T convertStringToPartitionProgressState(Class progressStateClass, final String serializedPartitionProgressState) { + if (Objects.isNull(serializedPartitionProgressState)) { + return null; + } + + try { + if (progressStateClass != null) { + return MAPPER.readValue(serializedPartitionProgressState, progressStateClass); + } + return MAPPER.readValue(serializedPartitionProgressState, new TypeReference<>() { + }); + } catch (final JsonProcessingException e) { + LOG.error("Unable to convert string to partition progress state class {}: {}", progressStateClass.getName(), e); + return null; + } + } + + /** + * Helper method to convert progress state to String + * This is because the state is currently stored as a String in the coordination store. + */ + protected String convertPartitionProgressStatetoString(Optional partitionProgressState) { + if (partitionProgressState.isEmpty()) { + return null; + } + try { + return MAPPER.writeValueAsString(partitionProgressState.get()); + } catch (final JsonProcessingException e) { + LOG.error("Unable to convert partition progress state class to string: {}", e); + return null; + } + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/DataFilePartition.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/DataFilePartition.java new file mode 100644 index 0000000000..bbf35374f4 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/DataFilePartition.java @@ -0,0 +1,74 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.DataFileProgressState; + +import java.util.Optional; + +/** + * An DataFilePartition represents an export data file needs to be loaded. + * The source identifier contains keyword 'DATAFILE' + */ +public class DataFilePartition extends SourcePartition { + + public static final String PARTITION_TYPE = "DATAFILE"; + + private final String exportArn; + private final String bucket; + private final String key; + + private final DataFileProgressState state; + + public DataFilePartition(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + String[] keySplits = sourcePartitionStoreItem.getSourcePartitionKey().split("\\|"); + exportArn = keySplits[0]; + bucket = keySplits[1]; + key = keySplits[2]; + this.state = convertStringToPartitionProgressState(DataFileProgressState.class, sourcePartitionStoreItem.getPartitionProgressState()); + + } + + public DataFilePartition(String exportArn, String bucket, String key, Optional state) { + this.exportArn = exportArn; + this.bucket = bucket; + this.key = key; + this.state = state.orElse(null); + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return exportArn + "|" + bucket + "|" + key; + } + + @Override + public Optional getProgressState() { + if (state != null) { + return Optional.of(state); + } + return Optional.empty(); + } + + public String getExportArn() { + return exportArn; + } + + public String getBucket() { + return bucket; + } + + public String getKey() { + return key; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/ExportPartition.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/ExportPartition.java new file mode 100644 index 0000000000..a7da8ac225 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/ExportPartition.java @@ -0,0 +1,74 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.ExportProgressState; + +import java.time.Instant; +import java.util.Optional; + +/** + * An ExportPartition represents an export job needs to be run for a table. + * Each table may have multiple export jobs, each export job has an export time associate with it. + * Each job maintains the state such as total files/records etc. independently. + * The source identifier contains keyword 'EXPORT' + */ +public class ExportPartition extends SourcePartition { + + public static final String PARTITION_TYPE = "EXPORT"; + private final String tableArn; + + private final Instant exportTime; + + private final ExportProgressState state; + + public ExportPartition(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + String[] keySplits = sourcePartitionStoreItem.getSourcePartitionKey().split("\\|"); + tableArn = keySplits[0]; + exportTime = Instant.ofEpochMilli(Long.valueOf(keySplits[1])); + this.state = convertStringToPartitionProgressState(ExportProgressState.class, sourcePartitionStoreItem.getPartitionProgressState()); + + } + + public ExportPartition(String tableArn, Instant exportTime, Optional state) { + this.tableArn = tableArn; + this.exportTime = exportTime; + this.state = state.orElse(null); + + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return tableArn + "|" + exportTime.toEpochMilli(); + } + + @Override + public Optional getProgressState() { + if (state != null) { + return Optional.of(state); + } + return Optional.empty(); + } + + + public String getTableArn() { + return tableArn; + } + + public Instant getExportTime() { + return exportTime; + } + + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/GlobalState.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/GlobalState.java new file mode 100644 index 0000000000..4b45dd10f4 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/GlobalState.java @@ -0,0 +1,62 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; + +import java.util.Map; +import java.util.Optional; + +/** + * Global State is a special type of partition. The partition type is null. + * You can't acquire (own) a Global State. + * However, you can read and update Global State whenever required. + * The progress state is a Map object. + */ +public class GlobalState extends SourcePartition> { + + private final String stateName; + + private Map state; + + public GlobalState(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + this.stateName = sourcePartitionStoreItem.getSourcePartitionKey(); + this.state = convertStringToPartitionProgressState(null, sourcePartitionStoreItem.getPartitionProgressState()); + + } + + public GlobalState(String stateName, Optional> state) { + this.stateName = stateName; + this.state = state.orElse(null); + + } + + @Override + public String getPartitionType() { + return null; + } + + @Override + public String getPartitionKey() { + return stateName; + } + + @Override + public Optional> getProgressState() { + if (state != null) { + return Optional.of(state); + } + return Optional.empty(); + } + + public void setProgressState(Map state) { + this.state = state; + } + + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/InitPartition.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/InitPartition.java new file mode 100644 index 0000000000..c2c7008bee --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/InitPartition.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.EmptyProgressState; + +import java.util.Optional; + +/** + * Use a partition to track whether initialization has happened or not. + * This is to ensure that initialization is triggered once. + * The downside is that after initialization, changes to configuration will be ignored. + * The source identifier contains keyword 'INIT' + */ +public class InitPartition extends SourcePartition { + public static final String PARTITION_TYPE = "INIT"; + + private static final String DEFAULT_PARTITION_KEY = "GLOBAL"; + + public InitPartition() { + } + + public InitPartition(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return DEFAULT_PARTITION_KEY; + } + + @Override + public Optional getProgressState() { + return Optional.empty(); + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/StreamPartition.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/StreamPartition.java new file mode 100644 index 0000000000..846b33d573 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/partition/StreamPartition.java @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.StreamProgressState; + +import java.util.Optional; + +public class StreamPartition extends SourcePartition { + + public static final String PARTITION_TYPE = "STREAM"; + + private final String streamArn; + + private final String shardId; + + private final StreamProgressState state; + + public StreamPartition(String streamArn, String shardId, Optional state) { + this.streamArn = streamArn; + this.shardId = shardId; + this.state = state.orElse(null); + } + + public StreamPartition(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + String[] keySplits = sourcePartitionStoreItem.getSourcePartitionKey().split("\\|"); + streamArn = keySplits[0]; + shardId = keySplits[1]; + this.state = convertStringToPartitionProgressState(StreamProgressState.class, sourcePartitionStoreItem.getPartitionProgressState()); + + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return streamArn + "|" + shardId; + } + + @Override + public Optional getProgressState() { + if (state != null) { + return Optional.of(state); + } + return Optional.empty(); + } + + public String getStreamArn() { + return streamArn; + } + + public String getShardId() { + return shardId; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/DataFileProgressState.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/DataFileProgressState.java new file mode 100644 index 0000000000..3a7780fec6 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/DataFileProgressState.java @@ -0,0 +1,34 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DataFileProgressState { + + + @JsonProperty("totalRecords") + private int total; + + @JsonProperty("loadedRecords") + private int loaded; + + public int getTotal() { + return total; + } + + public int getLoaded() { + return loaded; + } + + public void setTotal(int total) { + this.total = total; + } + + public void setLoaded(int loaded) { + this.loaded = loaded; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/EmptyProgressState.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/EmptyProgressState.java new file mode 100644 index 0000000000..c49c0773c8 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/EmptyProgressState.java @@ -0,0 +1,9 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state; + +public class EmptyProgressState { +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/ExportProgressState.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/ExportProgressState.java new file mode 100644 index 0000000000..4aa15cacf5 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/ExportProgressState.java @@ -0,0 +1,67 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class ExportProgressState { + + @JsonProperty("exportArn") + private String exportArn; + + @JsonProperty("status") + private String status; + + @JsonProperty("bucket") + private String bucket; + + @JsonProperty("prefix") + private String prefix; + + @JsonProperty("exportTime") + private String exportTime; + + + public String getExportArn() { + return exportArn; + } + + public void setExportArn(String exportArn) { + this.exportArn = exportArn; + } + + public String getBucket() { + return bucket; + } + + public void setBucket(String bucket) { + this.bucket = bucket; + } + + public String getPrefix() { + return prefix; + } + + public void setPrefix(String prefix) { + this.prefix = prefix; + } + + public String getExportTime() { + return exportTime; + } + + public void setExportTime(String exportTime) { + this.exportTime = exportTime; + } + + public String getStatus() { + return status; + } + + public void setStatus(String status) { + this.status = status; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/StreamProgressState.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/StreamProgressState.java new file mode 100644 index 0000000000..acffba8314 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/state/StreamProgressState.java @@ -0,0 +1,45 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class StreamProgressState { + + @JsonProperty("startTime") + private long startTime; + + @JsonProperty("sequenceNumber") + private String sequenceNumber; + + + @JsonProperty("waitForExport") + private boolean waitForExport = false; + + public long getStartTime() { + return startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + public String getSequenceNumber() { + return sequenceNumber; + } + + public void setSequenceNumber(String sequenceNumber) { + this.sequenceNumber = sequenceNumber; + } + + public boolean shouldWaitForExport() { + return waitForExport; + } + + public void setWaitForExport(boolean waitForExport) { + this.waitForExport = waitForExport; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileCheckpointer.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileCheckpointer.java new file mode 100644 index 0000000000..bb681c8a20 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileCheckpointer.java @@ -0,0 +1,78 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.DataFileProgressState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + +/** + * A helper class to handle the data file partition status and the progress state + * It will use coordinator APIs under the hood. + */ +public class DataFileCheckpointer { + private static final Logger LOG = LoggerFactory.getLogger(DataFileCheckpointer.class); + + + private final EnhancedSourceCoordinator enhancedSourceCoordinator; + + private final DataFilePartition dataFilePartition; + + + public DataFileCheckpointer(EnhancedSourceCoordinator enhancedSourceCoordinator, DataFilePartition dataFilePartition) { + this.enhancedSourceCoordinator = enhancedSourceCoordinator; + this.dataFilePartition = dataFilePartition; + } + + private void setProgressState(int lineNumber) { + //Always has a state. + Optional progressState = dataFilePartition.getProgressState(); + progressState.get().setLoaded(lineNumber); + } + + /** + * This method is to do a checkpoint with latest sequence number processed. + * Note that this should be called on a regular basis even there are no changes to sequence number + * As the checkpoint will also extend the timeout for the lease + * + * @param lineNumber The last line number + */ + public void checkpoint(int lineNumber) { + LOG.debug("Checkpoint data file " + dataFilePartition.getKey() + " with line number " + lineNumber); + setProgressState(lineNumber); + enhancedSourceCoordinator.saveProgressStateForPartition(dataFilePartition); + } + + /** + * This method is to mark the shard partition as COMPLETED with the final sequence number + * Note that this should be called when reaching the end of shard. + * + * @param lineNumber The last line number + */ + public void complete(int lineNumber) { + LOG.debug("Complete the read of data file " + dataFilePartition.getKey() + " with line number " + lineNumber); + setProgressState(lineNumber); + enhancedSourceCoordinator.completePartition(dataFilePartition); + } + + /** + * This method is to release the lease of the data file partition. + * Normally this should only be called due to failures or interruption. + * + * @param lineNumber The last line number + */ + public void release(int lineNumber) { + LOG.debug("Release the ownership of data file " + dataFilePartition.getKey() + " with line number " + lineNumber); + setProgressState(lineNumber); + enhancedSourceCoordinator.giveUpPartition(dataFilePartition); + } + + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoader.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoader.java new file mode 100644 index 0000000000..58337f871a --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoader.java @@ -0,0 +1,188 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import org.opensearch.dataprepper.plugins.source.dynamodb.converter.ExportRecordConverter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedReader; +import java.io.InputStreamReader; +import java.util.ArrayList; +import java.util.List; +import java.util.zip.GZIPInputStream; + +/** + * One loader per file. + */ +public class DataFileLoader implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(DataFileLoader.class); + + /** + * A flag to interrupt the process + */ + private static volatile boolean shouldStop = false; + + /** + * Number of lines to be read in a batch + */ + private static final int DEFAULT_BATCH_SIZE = 1000; + + /** + * Default regular checkpoint interval + */ + private static final int DEFAULT_CHECKPOINT_INTERVAL_MILLS = 2 * 60_000; + + private final String bucketName; + + private final String key; + + private final ExportRecordConverter recordConverter; + + private final S3ObjectReader s3ObjectReader; + + private final DataFileCheckpointer checkpointer; + + // Start Line is the checkpoint + private final int startLine; + + private DataFileLoader(Builder builder) { + this.s3ObjectReader = builder.s3ObjectReader; + this.recordConverter = builder.recordConverter; + this.bucketName = builder.bucketName; + this.key = builder.key; + this.checkpointer = builder.checkpointer; + this.startLine = builder.startLine; + } + + public static Builder builder() { + return new Builder(); + } + + + /** + * Default Builder for DataFileLoader + */ + static class Builder { + + private S3ObjectReader s3ObjectReader; + + private ExportRecordConverter recordConverter; + + private DataFileCheckpointer checkpointer; + + private String bucketName; + + private String key; + + private int startLine; + + public Builder s3ObjectReader(S3ObjectReader s3ObjectReader) { + this.s3ObjectReader = s3ObjectReader; + return this; + } + + public Builder recordConverter(ExportRecordConverter recordConverter) { + this.recordConverter = recordConverter; + return this; + } + + public Builder checkpointer(DataFileCheckpointer checkpointer) { + this.checkpointer = checkpointer; + return this; + } + + public Builder bucketName(String bucketName) { + this.bucketName = bucketName; + return this; + } + + public Builder key(String key) { + this.key = key; + return this; + } + + public Builder startLine(int startLine) { + this.startLine = startLine; + return this; + } + + public DataFileLoader build() { + return new DataFileLoader(this); + } + + } + + + @Override + public void run() { + LOG.debug("Read export data from s3://" + bucketName + "/" + key + " with start line " + startLine); + long lastCheckpointTime = System.currentTimeMillis(); + List lines = new ArrayList<>(); + + // line count regardless the start line number + int lineCount = 0; + int lastLineProcessed = 0; + + try (GZIPInputStream gzipInputStream = new GZIPInputStream(s3ObjectReader.readFile(bucketName, key))) { + BufferedReader reader = new BufferedReader(new InputStreamReader(gzipInputStream)); + + String line; + while ((line = reader.readLine()) != null) { + + if (shouldStop) { + checkpointer.checkpoint(lastLineProcessed); + LOG.debug("Should Stop flag is set to True, looks like shutdown has triggered"); + throw new RuntimeException("Load is interrupted"); + } + + lineCount += 1; + + // process each line + if (lineCount <= startLine) { + continue; + } + lines.add(line); + + if ((lineCount - startLine) % DEFAULT_BATCH_SIZE == 0) { + // LOG.debug("Write to buffer for line " + (lineCount - DEFAULT_BATCH_SIZE) + " to " + lineCount); + recordConverter.writeToBuffer(lines); + lines.clear(); + lastLineProcessed = lineCount; + } + + if (System.currentTimeMillis() - lastCheckpointTime > DEFAULT_CHECKPOINT_INTERVAL_MILLS) { + LOG.debug("Perform regular checkpointing for Data File Loader"); + checkpointer.checkpoint(lastLineProcessed); + lastCheckpointTime = System.currentTimeMillis(); + + } + + } + if (!lines.isEmpty()) { + // Do final checkpoint. + recordConverter.writeToBuffer(lines); + checkpointer.checkpoint(lineCount); + } + + lines.clear(); + reader.close(); + LOG.debug("Data Loader completed successfully"); + } catch (Exception e) { + checkpointer.checkpoint(lineCount); + throw new RuntimeException("Data Loader completed with Exception: " + e.getMessage()); + } + + } + + /** + * Currently, this is to stop all consumers. + */ + public static void stopAll() { + shouldStop = true; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoaderFactory.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoaderFactory.java new file mode 100644 index 0000000000..07d5a6efa7 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoaderFactory.java @@ -0,0 +1,56 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.converter.ExportRecordConverter; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import software.amazon.awssdk.services.s3.S3Client; + +/** + * Factory class for DataFileLoader thread. + */ +public class DataFileLoaderFactory { + + private final EnhancedSourceCoordinator coordinator; + + private final S3ObjectReader fileReader; + + private final PluginMetrics pluginMetrics; + + private final Buffer> buffer; + + public DataFileLoaderFactory(EnhancedSourceCoordinator coordinator, S3Client s3Client, PluginMetrics pluginMetrics, Buffer> buffer) { + this.coordinator = coordinator; + this.pluginMetrics = pluginMetrics; + this.buffer = buffer; + fileReader = new S3ObjectReader(s3Client); + } + + public Runnable createDataFileLoader(DataFilePartition dataFilePartition, TableInfo tableInfo) { + ExportRecordConverter recordProcessor = new ExportRecordConverter(buffer, tableInfo, pluginMetrics); + + DataFileCheckpointer checkpointer = new DataFileCheckpointer(coordinator, dataFilePartition); + + // Start a data loader thread. + DataFileLoader loader = DataFileLoader.builder() + .s3ObjectReader(fileReader) + .bucketName(dataFilePartition.getBucket()) + .key(dataFilePartition.getKey()) + .recordConverter(recordProcessor) + .checkpointer(checkpointer) + .startLine(dataFilePartition.getProgressState().get().getLoaded()) + .build(); + + return loader; + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileScheduler.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileScheduler.java new file mode 100644 index 0000000000..181300c9c3 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileScheduler.java @@ -0,0 +1,185 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.LoadStatus; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; + + +public class DataFileScheduler implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(DataFileScheduler.class); + + private final AtomicInteger numOfWorkers = new AtomicInteger(0); + + private static final int MAX_JOB_COUNT = 2; + private static final int DEFAULT_LEASE_INTERVAL_MILLIS = 30_000; + + static final String EXPORT_FILE_SUCCESS_COUNT = "exportFileSuccess"; + + + private final EnhancedSourceCoordinator coordinator; + + private final ExecutorService executor; + + private final DataFileLoaderFactory loaderFactory; + + private final PluginMetrics pluginMetrics; + + + private final Counter exportFileSuccessCounter; + + + public DataFileScheduler(EnhancedSourceCoordinator coordinator, DataFileLoaderFactory loaderFactory, PluginMetrics pluginMetrics) { + this.coordinator = coordinator; + this.pluginMetrics = pluginMetrics; + this.loaderFactory = loaderFactory; + + + executor = Executors.newFixedThreadPool(MAX_JOB_COUNT); + + this.exportFileSuccessCounter = pluginMetrics.counter(EXPORT_FILE_SUCCESS_COUNT); + } + + private void processDataFilePartition(DataFilePartition dataFilePartition) { + String exportArn = dataFilePartition.getExportArn(); + String tableArn = getTableArn(exportArn); + + TableInfo tableInfo = getTableInfo(tableArn); + + Runnable loader = loaderFactory.createDataFileLoader(dataFilePartition, tableInfo); + CompletableFuture runLoader = CompletableFuture.runAsync(loader, executor); + runLoader.whenComplete(completeDataLoader(dataFilePartition)); + numOfWorkers.incrementAndGet(); + } + + @Override + public void run() { + LOG.debug("Start running Data file Scheduler"); + + while (!Thread.interrupted()) { + if (numOfWorkers.get() < MAX_JOB_COUNT) { + final Optional sourcePartition = coordinator.acquireAvailablePartition(DataFilePartition.PARTITION_TYPE); + + if (sourcePartition.isPresent()) { + DataFilePartition dataFilePartition = (DataFilePartition) sourcePartition.get(); + processDataFilePartition(dataFilePartition); + } + } + try { + Thread.sleep(DEFAULT_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException e) { + LOG.info("InterruptedException occurred"); + break; + } + + } + LOG.debug("Data file scheduler is interrupted, Stop all data file loaders..."); + // Cannot call executor.shutdownNow() here + // Otherwise the final checkpoint will fail due to SDK interruption. + executor.shutdown(); + DataFileLoader.stopAll(); + } + + private TableInfo getTableInfo(String tableArn) { + GlobalState tableState = (GlobalState) coordinator.getPartition(tableArn).get(); + TableInfo tableInfo = new TableInfo(tableArn, TableMetadata.fromMap(tableState.getProgressState().get())); + return tableInfo; + } + + private String getTableArn(String exportArn) { + // e.g. given export arn:arn:aws:dynamodb:us-west-2:123456789012:table/Thread/export/01693291918297-bfeccbea + // returns: arn:aws:dynamodb:us-west-2:123456789012:table/Thread + return exportArn.substring(0, exportArn.lastIndexOf("/export/")); + } + + private String getStreamArn(String exportArn) { + String tableArn = getTableArn(exportArn); + TableInfo tableInfo = getTableInfo(tableArn); + + if (tableInfo.getMetadata().isStreamRequired()) { + return tableInfo.getMetadata().getStreamArn(); + } + return null; + } + + + private BiConsumer completeDataLoader(DataFilePartition dataFilePartition) { + return (v, ex) -> { + numOfWorkers.decrementAndGet(); + if (ex == null) { + exportFileSuccessCounter.increment(); + // Update global state + updateState(dataFilePartition.getExportArn(), dataFilePartition.getProgressState().get().getLoaded()); + // After global state is updated, mask the partition as completed. + coordinator.completePartition(dataFilePartition); + + } else { + // The data loader must have already done one last checkpointing. + LOG.debug("Data Loader completed with exception"); + LOG.error("{}", ex); + // Release the ownership + coordinator.giveUpPartition(dataFilePartition); + } + + }; + } + + private void updateState(String exportArn, int loaded) { + + String streamArn = getStreamArn(exportArn); + + // Unlimited retries + // The state be out of dated when updating. + while (true) { + Optional globalPartition = coordinator.getPartition(exportArn); + if (globalPartition.isEmpty()) { + LOG.error("Failed to get load status for " + exportArn); + return; + } + + GlobalState globalState = (GlobalState) globalPartition.get(); + LoadStatus loadStatus = LoadStatus.fromMap(globalState.getProgressState().get()); + LOG.debug("Current status: total {} loaded {}", loadStatus.getTotalFiles(), loadStatus.getLoadedFiles()); + + loadStatus.setLoadedFiles(loadStatus.getLoadedFiles() + 1); + loadStatus.setLoadedRecords(loadStatus.getLoadedRecords() + loaded); + globalState.setProgressState(loadStatus.toMap()); + + try { + coordinator.saveProgressStateForPartition(globalState); + // if all load are completed. + if (streamArn != null && loadStatus.getLoadedFiles() == loadStatus.getTotalFiles()) { + LOG.debug("All Exports are done, streaming can continue..."); + coordinator.createPartition(new GlobalState(streamArn, Optional.empty())); + } + break; + } catch (Exception e) { + LOG.error("Failed to update the global status, looks like the status was out of dated, will retry.."); + } + + } + + + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ExportScheduler.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ExportScheduler.java new file mode 100644 index 0000000000..34fb702cc1 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ExportScheduler.java @@ -0,0 +1,265 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.DataFileProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.ExportProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.ExportSummary; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.LoadStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; + +import java.time.Duration; +import java.util.Map; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; + +/** + * A scheduler to manage all the export related work in one place + */ +public class ExportScheduler implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(ExportScheduler.class); + + private static final int DEFAULT_TAKE_LEASE_INTERVAL_MILLIS = 60_000; + + private static final Duration DEFAULT_CLOSE_DURATION = Duration.ofMinutes(1); + + private static final int DEFAULT_MAX_CLOSE_COUNT = 6; + + private static final int DEFAULT_CHECKPOINT_INTERVAL_MILLS = 5 * 60_000; + + private static final int DEFAULT_CHECK_STATUS_INTERVAL_MILLS = 30 * 1000; + + + private static final String COMPLETED_STATUS = "Completed"; + private static final String FAILED_STATUS = "Failed"; + + static final String EXPORT_JOB_SUCCESS_COUNT = "exportJobSuccess"; + static final String EXPORT_JOB_ERROR_COUNT = "exportJobErrors"; + static final String EXPORT_FILES_TOTAL_COUNT = "exportFilesTotal"; + static final String EXPORT_RECORDS_TOTAL_COUNT = "exportRecordsTotal"; + + private final PluginMetrics pluginMetrics; + + private final EnhancedSourceCoordinator enhancedSourceCoordinator; + + private final DynamoDbClient dynamoDBClient; + + private final ExecutorService executor; + + private final ManifestFileReader manifestFileReader; + + private final ExportTaskManager exportTaskManager; + + private final Counter exportJobSuccessCounter; + private final Counter exportJobErrorCounter; + + private final Counter exportFilesTotalCounter; + private final Counter exportRecordsTotalCounter; + + public ExportScheduler(EnhancedSourceCoordinator enhancedSourceCoordinator, DynamoDbClient dynamoDBClient, ManifestFileReader manifestFileReader, PluginMetrics pluginMetrics) { + this.enhancedSourceCoordinator = enhancedSourceCoordinator; + this.dynamoDBClient = dynamoDBClient; + this.pluginMetrics = pluginMetrics; + this.exportTaskManager = new ExportTaskManager(dynamoDBClient); + + this.manifestFileReader = manifestFileReader; + executor = Executors.newCachedThreadPool(); + + exportJobSuccessCounter = pluginMetrics.counter(EXPORT_JOB_SUCCESS_COUNT); + exportJobErrorCounter = pluginMetrics.counter(EXPORT_JOB_ERROR_COUNT); + exportFilesTotalCounter = pluginMetrics.counter(EXPORT_FILES_TOTAL_COUNT); + exportRecordsTotalCounter = pluginMetrics.counter(EXPORT_RECORDS_TOTAL_COUNT); + + + } + + @Override + public void run() { + LOG.debug("Start running Export Scheduler"); + while (!Thread.interrupted()) { + // Does not have limit on max leases + // As most of the time it's just to wait + final Optional sourcePartition = enhancedSourceCoordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE); + + if (sourcePartition.isPresent()) { + + ExportPartition exportPartition = (ExportPartition) sourcePartition.get(); + LOG.debug("Acquired an export partition: " + exportPartition.getPartitionKey()); + + String exportArn = getOrCreateExportArn(exportPartition); + + if (exportArn == null) { + closeExportPartitionWithError(exportPartition); + } else { + CompletableFuture checkStatus = CompletableFuture.supplyAsync(() -> checkExportStatus(exportPartition), executor); + checkStatus.whenComplete(completeExport(exportPartition)); + } + + } + try { + Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException e) { + LOG.info("InterruptedException occurred"); + break; + } + + } + LOG.debug("Export scheduler interrupted, looks like shutdown has triggered"); + executor.shutdownNow(); + + } + + + private BiConsumer completeExport(ExportPartition exportPartition) { + return (status, ex) -> { + if (ex != null) { + LOG.debug("Check export status for {} failed with error {}", exportPartition.getPartitionKey(), ex.getMessage()); +// closeExportPartitionWithError(exportPartition); + enhancedSourceCoordinator.giveUpPartition(exportPartition); + } else { + // check status + LOG.debug("Check export status completed successfully"); + + if (!"COMPLETED".equals(status)) { + closeExportPartitionWithError(exportPartition); + return; + } + LOG.debug("Start reading the manifest files"); + + // Always has a state + ExportProgressState state = exportPartition.getProgressState().get(); + String bucketName = state.getBucket(); + String exportArn = state.getExportArn(); + + + String manifestKey = exportTaskManager.getExportManifest(exportArn); + LOG.debug("Export manifest summary file is " + manifestKey); + + // Extract the info in the manifest summary file + // We may need to store the info + ExportSummary summaryInfo = manifestFileReader.parseSummaryFile(bucketName, manifestKey); + + // Get the manifest data path + // We don't really need to use the summary info to get the path + Map dataFileInfo = manifestFileReader.parseDataFile(summaryInfo.getS3Bucket(), summaryInfo.getManifestFilesS3Key()); + + // Create a data file partition for each + createDataFilePartitions(exportArn, bucketName, dataFileInfo); + + // Finally close the export partition + completeExportPartition(exportPartition); + + } + + }; + } + + + private void createDataFilePartitions(String exportArn, String bucketName, Map dataFileInfo) { + LOG.debug("Totally {} data files generated for export {}", dataFileInfo.size(), exportArn); + AtomicInteger totalRecords = new AtomicInteger(); + AtomicInteger totalFiles = new AtomicInteger(); + dataFileInfo.forEach((key, size) -> { + DataFileProgressState progressState = new DataFileProgressState(); + progressState.setTotal(size); + progressState.setLoaded(0); + totalFiles.addAndGet(1); + totalRecords.addAndGet(size); + DataFilePartition partition = new DataFilePartition(exportArn, bucketName, key, Optional.of(progressState)); + enhancedSourceCoordinator.createPartition(partition); + }); + + exportFilesTotalCounter.increment(totalFiles.get()); + exportRecordsTotalCounter.increment(totalRecords.get()); + + // Currently, we need to maintain a global state to track the overall progress. + // So that we can easily tell if all the export files are loaded + LoadStatus loadStatus = new LoadStatus(totalFiles.get(), 0, totalRecords.get(), 0); + enhancedSourceCoordinator.createPartition(new GlobalState(exportArn, Optional.of(loadStatus.toMap()))); + } + + + private void closeExportPartitionWithError(ExportPartition exportPartition) { + exportJobErrorCounter.increment(1); + ExportProgressState exportProgressState = exportPartition.getProgressState().get(); + // Clear current Arn, so that a new export can be submitted. + exportProgressState.setExportArn(null); + exportProgressState.setStatus(FAILED_STATUS); + enhancedSourceCoordinator.closePartition(exportPartition, DEFAULT_CLOSE_DURATION, DEFAULT_MAX_CLOSE_COUNT); + } + + private void completeExportPartition(ExportPartition exportPartition) { + exportJobSuccessCounter.increment(); + ExportProgressState state = exportPartition.getProgressState().get(); + state.setStatus(COMPLETED_STATUS); + enhancedSourceCoordinator.completePartition(exportPartition); + } + + private String checkExportStatus(ExportPartition exportPartition) { + long lastCheckpointTime = System.currentTimeMillis(); + String exportArn = exportPartition.getProgressState().get().getExportArn(); + + LOG.debug("Start Checking the status of export " + exportArn); + while (true) { + if (System.currentTimeMillis() - lastCheckpointTime > DEFAULT_CHECKPOINT_INTERVAL_MILLS) { + enhancedSourceCoordinator.saveProgressStateForPartition(exportPartition); + lastCheckpointTime = System.currentTimeMillis(); + } + + String status = exportTaskManager.checkExportStatus(exportArn); + if (!"IN_PROGRESS".equals(status)) { + LOG.debug("Export {} is completed with final status {}", exportArn, status); + return status; + } + LOG.debug("Export {} is still running in progress, sleep and recheck later", exportArn); + try { + Thread.sleep(DEFAULT_CHECK_STATUS_INTERVAL_MILLS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + } + + private String getOrCreateExportArn(ExportPartition exportPartition) { + // State cannot be empty + ExportProgressState state = exportPartition.getProgressState().get(); + // Check the progress state + if (state.getExportArn() != null) { + LOG.debug("Export Job has already submitted for table {} with export time {}", exportPartition.getTableArn(), exportPartition.getExportTime()); + // Export job already submitted + return state.getExportArn(); + } + + LOG.debug("Try to submit a new export job for table {} with export time {}", exportPartition.getTableArn(), exportPartition.getExportTime()); + // submit a new export request + String exportArn = exportTaskManager.submitExportJob(exportPartition.getTableArn(), state.getBucket(), state.getPrefix(), exportPartition.getExportTime()); + + // Update state with export Arn in the coordination table. + // So that it won't be submitted again after a restart. + if (exportArn != null) { + LOG.debug("Export arn is " + exportArn); + state.setExportArn(exportArn); + enhancedSourceCoordinator.saveProgressStateForPartition(exportPartition); + } + return exportArn; + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ExportTaskManager.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ExportTaskManager.java new file mode 100644 index 0000000000..c0b43009a0 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ExportTaskManager.java @@ -0,0 +1,88 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.dynamodb.model.DescribeExportRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeExportResponse; +import software.amazon.awssdk.services.dynamodb.model.ExportFormat; +import software.amazon.awssdk.services.dynamodb.model.ExportTableToPointInTimeRequest; +import software.amazon.awssdk.services.dynamodb.model.ExportTableToPointInTimeResponse; + +import java.time.Instant; + +public class ExportTaskManager { + + private static final Logger LOG = LoggerFactory.getLogger(ExportTaskManager.class); + + private static final ExportFormat DEFAULT_EXPORT_FORMAT = ExportFormat.ION; + + private final DynamoDbClient dynamoDBClient; + + + public ExportTaskManager(DynamoDbClient dynamoDBClient) { + this.dynamoDBClient = dynamoDBClient; + } + + public String submitExportJob(String tableArn, String bucketName, String prefix, Instant exportTime) { + // No needs to use a client token here. + ExportTableToPointInTimeRequest req = ExportTableToPointInTimeRequest.builder() + .tableArn(tableArn) + .s3Bucket(bucketName) + .s3Prefix(prefix) + .exportFormat(DEFAULT_EXPORT_FORMAT) + .exportTime(exportTime) + .build(); + + + try { + ExportTableToPointInTimeResponse response = dynamoDBClient.exportTableToPointInTime(req); + + String exportArn = response.exportDescription().exportArn(); + String status = response.exportDescription().exportStatusAsString(); + + LOG.debug("Export Job submitted with ARN {} and status {}", exportArn, status); + return exportArn; + } catch (SdkException e) { + LOG.error("Failed to submit an export job with error " + e.getMessage()); + return null; + } + + } + + public String getExportManifest(String exportArn) { + DescribeExportRequest request = DescribeExportRequest.builder().exportArn(exportArn).build(); + + String manifestKey = null; + try { + DescribeExportResponse resp = dynamoDBClient.describeExport(request); + manifestKey = resp.exportDescription().exportManifest(); + + } catch (SdkException e) { + LOG.error("Unable to get manifest file for export " + exportArn); + } + return manifestKey; + } + + public String checkExportStatus(String exportArn) { + DescribeExportRequest request = DescribeExportRequest.builder().exportArn(exportArn).build(); + + // Not need to catch exception here. + DescribeExportResponse resp = dynamoDBClient.describeExport(request); + String status = resp.exportDescription().exportStatusAsString(); + + if (resp.exportDescription().failureCode() != null) { + LOG.error("Export failed with error: " + resp.exportDescription().failureMessage()); + } + + return status; + } + + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ManifestFileReader.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ManifestFileReader.java new file mode 100644 index 0000000000..4e8468114b --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ManifestFileReader.java @@ -0,0 +1,85 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.ExportSummary; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.util.HashMap; +import java.util.Map; + +public class ManifestFileReader { + + private static final Logger LOG = LoggerFactory.getLogger(ManifestFileReader.class); + + private static final TypeReference> MAP_TYPE_REFERENCE = new TypeReference<>() { + }; + + private static final String DATA_FILE_S3_KEY = "dataFileS3Key"; + private static final String DATA_FILE_ITEM_COUNT_KEY = "itemCount"; + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private final S3ObjectReader objectReader; + + public ManifestFileReader(S3ObjectReader objectReader) { + this.objectReader = objectReader; + } + + public ExportSummary parseSummaryFile(String bucket, String key) { + LOG.debug("Try to read the manifest summary file"); + InputStream object = objectReader.readFile(bucket, key); + + BufferedReader reader = new BufferedReader(new InputStreamReader(object)); + try { + String line = reader.readLine(); + LOG.debug("Manifest summary: {}", line); + ExportSummary summaryInfo = MAPPER.readValue(line, ExportSummary.class); + return summaryInfo; + + } catch (JsonProcessingException e) { + LOG.error("Failed to parse the summary info due to {}", e.getMessage()); + throw new RuntimeException(e); + + } catch (IOException e) { + LOG.error("IO Exception due to {}", e.getMessage()); + throw new RuntimeException(e); + } + + } + + public Map parseDataFile(String bucket, String key) { + LOG.debug("Try to read the manifest data file"); + + Map result = new HashMap<>(); + InputStream object = objectReader.readFile(bucket, key); + BufferedReader reader = new BufferedReader(new InputStreamReader(object)); + + String line; + try { + while ((line = reader.readLine()) != null) { + // An example line as below: + // {"itemCount":46331,"md5Checksum":"a0k21IY3eelgr2PuWJLjJw==","etag":"51f9f394903c5d682321c6211aae8b6a-1","dataFileS3Key":"test-table-export/AWSDynamoDB/01692350182719-6de2c037/data/fpgzwz7ome3s7a5gqn2mu3ogtq.json.gz"} + Map map = MAPPER.readValue(line, MAP_TYPE_REFERENCE); + LOG.debug("Get a file {} with item count {}", map.get(DATA_FILE_S3_KEY), map.get(DATA_FILE_ITEM_COUNT_KEY)); + result.put(map.get(DATA_FILE_S3_KEY), Integer.valueOf(map.get(DATA_FILE_ITEM_COUNT_KEY))); + + } + } catch (IOException e) { + LOG.error("IO Exception due to " + e.getMessage()); + } + + return result; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/S3ObjectReader.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/S3ObjectReader.java new file mode 100644 index 0000000000..424c934323 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/S3ObjectReader.java @@ -0,0 +1,41 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + +import java.io.InputStream; + +public class S3ObjectReader { + + private static final Logger LOG = LoggerFactory.getLogger(S3ObjectReader.class); + + private final S3Client s3Client; + + public S3ObjectReader(S3Client s3Client) { + this.s3Client = s3Client; + } + + public InputStream readFile(String bucketName, String s3Key) { + LOG.debug("Read file from s3://" + bucketName + "/" + s3Key); + + GetObjectRequest objectRequest = GetObjectRequest + .builder() + .bucket(bucketName) + .key(s3Key) + .build(); + + ResponseInputStream object = s3Client.getObject(objectRequest); + + return object; + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/ExportSummary.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/ExportSummary.java new file mode 100644 index 0000000000..881f59f605 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/ExportSummary.java @@ -0,0 +1,125 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.model; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.JsonProperty; + +/** + * Model represents the summary manifest information of the export. + *

+ * Check more details from + * https://docs.aws.amazon.com/amazondynamodb/latest/developerguide/S3DataExport.Output.html#S3DataExport.Output_Manifest + *

+ */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class ExportSummary { + @JsonProperty("version") + private String version; + + @JsonProperty("exportArn") + private String exportArn; + + @JsonProperty("startTime") + private String startTime; + + @JsonProperty("endTime") + private String endTime; + + @JsonProperty("tableArn") + private String tableArn; + + @JsonProperty("tableId") + private String tableId; + + @JsonProperty("exportTime") + private String exportTime; + + @JsonProperty("s3Bucket") + private String s3Bucket; + + @JsonProperty("s3Prefix") + private String s3Prefix; + + @JsonProperty("s3SseAlgorithm") + private String s3SseAlgorithm; + + @JsonProperty("s3SseKmsKeyId") + private String s3SseKmsKeyId; + + @JsonProperty("manifestFilesS3Key") + private String manifestFilesS3Key; + + @JsonProperty("billedSizeBytes") + private long billedSizeBytes; + + @JsonProperty("itemCount") + private int itemCount; + + @JsonProperty("outputFormat") + private String outputFormat; + + + public String getVersion() { + return version; + } + + public String getExportArn() { + return exportArn; + } + + public String getStartTime() { + return startTime; + } + + public String getEndTime() { + return endTime; + } + + public String getTableArn() { + return tableArn; + } + + public String getTableId() { + return tableId; + } + + public String getExportTime() { + return exportTime; + } + + public String getS3Bucket() { + return s3Bucket; + } + + public String getS3Prefix() { + return s3Prefix; + } + + public String getS3SseAlgorithm() { + return s3SseAlgorithm; + } + + public String getS3SseKmsKeyId() { + return s3SseKmsKeyId; + } + + public String getManifestFilesS3Key() { + return manifestFilesS3Key; + } + + public long getBilledSizeBytes() { + return billedSizeBytes; + } + + public int getItemCount() { + return itemCount; + } + + public String getOutputFormat() { + return outputFormat; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/LoadStatus.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/LoadStatus.java new file mode 100644 index 0000000000..fd84c87e98 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/LoadStatus.java @@ -0,0 +1,81 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.model; + +import java.util.Map; + +public class LoadStatus { + + private static final String TOTAL_FILES = "totalFiles"; + private static final String LOADED_FILES = "loadedFiles"; + private static final String TOTAL_RECORDS = "totalRecords"; + private static final String LOADED_RECORDS = "loadedRecords"; + + private int totalFiles; + + private int loadedFiles; + + private int totalRecords; + + private int loadedRecords; + + public LoadStatus(int totalFiles, int loadedFiles, int totalRecords, int loadedRecords) { + this.totalFiles = totalFiles; + this.loadedFiles = loadedFiles; + this.totalRecords = totalRecords; + this.loadedRecords = loadedRecords; + } + + public int getTotalFiles() { + return totalFiles; + } + + public void setTotalFiles(int totalFiles) { + this.totalFiles = totalFiles; + } + + public int getLoadedFiles() { + return loadedFiles; + } + + public void setLoadedFiles(int loadedFiles) { + this.loadedFiles = loadedFiles; + } + + public int getTotalRecords() { + return totalRecords; + } + + public void setTotalRecords(int totalRecords) { + this.totalRecords = totalRecords; + } + + public int getLoadedRecords() { + return loadedRecords; + } + + public void setLoadedRecords(int loadedRecords) { + this.loadedRecords = loadedRecords; + } + + public Map toMap() { + return Map.of( + TOTAL_FILES, totalFiles, + LOADED_FILES, loadedFiles, + TOTAL_RECORDS, totalRecords, + LOADED_RECORDS, loadedRecords + ); + } + + public static LoadStatus fromMap(Map map) { + return new LoadStatus( + (int) map.get(TOTAL_FILES), + (int) map.get(LOADED_FILES), + (int) map.get(TOTAL_RECORDS), + (int) map.get(LOADED_RECORDS) + ); + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/TableInfo.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/TableInfo.java new file mode 100644 index 0000000000..5724c75ee1 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/TableInfo.java @@ -0,0 +1,27 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.model; + +public class TableInfo { + + private final String tableArn; + + private final TableMetadata metadata; + + public TableInfo(String tableArn, TableMetadata metadata) { + this.tableArn = tableArn; + this.metadata = metadata; + } + + public String getTableArn() { + return tableArn; + } + + public TableMetadata getMetadata() { + return metadata; + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/TableMetadata.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/TableMetadata.java new file mode 100644 index 0000000000..0f2983639b --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/model/TableMetadata.java @@ -0,0 +1,174 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.model; + +import java.util.HashMap; +import java.util.Map; + +public class TableMetadata { + + private static final String PARTITION_KEY = "partitionKey"; + private static final String SORT_KEY = "sortKey"; + + private static final String STREAM_ARN_KEY = "streamArn"; + private static final String REQUIRE_EXPORT_KEY = "export"; + private static final String REQUIRE_STREAM_KEY = "stream"; + + + private final String partitionKeyAttributeName; + + private final String sortKeyAttributeName; + + private final String streamStartPosition; + + private final String streamArn; + + private final boolean streamRequired; + + private final boolean exportRequired; + + private final String exportBucket; + + private final String exportPrefix; + + private TableMetadata(Builder builder) { + this.partitionKeyAttributeName = builder.partitionKeyAttributeName; + this.sortKeyAttributeName = builder.sortKeyAttributeName; + this.streamArn = builder.streamArn; + this.streamRequired = builder.streamRequired; + this.exportRequired = builder.exportRequired; + this.exportBucket = builder.exportBucket; + this.exportPrefix = builder.exportPrefix; + this.streamStartPosition = builder.streamStartPosition; + + } + + public static Builder builder() { + return new Builder(); + } + + + public static class Builder { + + + private String partitionKeyAttributeName; + + private String sortKeyAttributeName; + + private boolean streamRequired; + + private boolean exportRequired; + + private String streamArn; + + private String exportBucket; + + private String exportPrefix; + + private String streamStartPosition; + + + public Builder partitionKeyAttributeName(String partitionKeyAttributeName) { + this.partitionKeyAttributeName = partitionKeyAttributeName; + return this; + } + + public Builder sortKeyAttributeName(String sortKeyAttributeName) { + this.sortKeyAttributeName = sortKeyAttributeName; + return this; + } + + public Builder streamArn(String streamArn) { + this.streamArn = streamArn; + return this; + } + + public Builder streamRequired(boolean streamRequired) { + this.streamRequired = streamRequired; + return this; + } + + public Builder exportRequired(boolean exportRequired) { + this.exportRequired = exportRequired; + return this; + } + + public Builder exportBucket(String exportBucket) { + this.exportBucket = exportBucket; + return this; + } + + public Builder exportPrefix(String exportPrefix) { + this.exportPrefix = exportPrefix; + return this; + } + + public Builder streamStartPosition(String streamStartPosition) { + this.streamStartPosition = streamStartPosition; + return this; + } + + public TableMetadata build() { + return new TableMetadata(this); + } + + } + + + public Map toMap() { + Map map = new HashMap<>(); + map.put(PARTITION_KEY, partitionKeyAttributeName); + map.put(SORT_KEY, sortKeyAttributeName); + map.put(STREAM_ARN_KEY, streamArn); + map.put(REQUIRE_EXPORT_KEY, exportRequired); + map.put(REQUIRE_STREAM_KEY, streamRequired); + return map; + + } + + public static TableMetadata fromMap(Map map) { + return TableMetadata.builder() + .partitionKeyAttributeName((String) map.get(PARTITION_KEY)) + .sortKeyAttributeName((String) map.get(SORT_KEY)) + .streamArn((String) map.get(STREAM_ARN_KEY)) + .exportRequired((boolean) map.getOrDefault(REQUIRE_EXPORT_KEY, false)) + .streamRequired((boolean) map.getOrDefault(REQUIRE_STREAM_KEY, false)) + .build(); + } + + + public String getPartitionKeyAttributeName() { + return partitionKeyAttributeName; + } + + public String getSortKeyAttributeName() { + return sortKeyAttributeName; + } + + public String getStreamArn() { + return streamArn; + } + + public boolean isStreamRequired() { + return streamRequired; + } + + public boolean isExportRequired() { + return exportRequired; + } + + public String getStreamStartPosition() { + return streamStartPosition; + } + + public String getExportBucket() { + return exportBucket; + } + + public String getExportPrefix() { + return exportPrefix; + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumer.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumer.java new file mode 100644 index 0000000000..fd9855547b --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumer.java @@ -0,0 +1,233 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.stream; + + +import org.opensearch.dataprepper.plugins.source.dynamodb.converter.StreamRecordConverter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.dynamodb.model.GetRecordsRequest; +import software.amazon.awssdk.services.dynamodb.model.GetRecordsResponse; +import software.amazon.awssdk.services.dynamodb.model.Record; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; + +import java.time.Instant; +import java.util.List; +import java.util.stream.Collectors; + +/** + * A basic data consumer to read from one shard + */ +public class ShardConsumer implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(ShardConsumer.class); + + /** + * A flag to interrupt the process + */ + private static volatile boolean shouldStop = false; + + /** + * Max number of items to return per GetRecords call, maximum 1000. + */ + private static final int MAX_GET_RECORD_ITEM_COUNT = 1000; + + /** + * Idle Time between GetRecords Reads + */ + private static final int GET_RECORD_INTERVAL_MILLS = 200; + + /** + * Default interval to check if export is completed. + */ + private static final int DEFAULT_WAIT_FOR_EXPORT_INTERVAL_MILLS = 60_000; + + /** + * Default regular checkpoint interval + */ + private static final int DEFAULT_CHECKPOINT_INTERVAL_MILLS = 2 * 60_000; + + private final DynamoDbStreamsClient dynamoDbStreamsClient; + + private final StreamRecordConverter recordConverter; + + private final StreamCheckpointer checkpointer; + + private String shardIterator; + + private final Instant startTime; + + private boolean waitForExport; + + private ShardConsumer(Builder builder) { + this.dynamoDbStreamsClient = builder.dynamoDbStreamsClient; + this.recordConverter = builder.recordConverter; + this.checkpointer = builder.checkpointer; + this.shardIterator = builder.shardIterator; + this.startTime = builder.startTime; + this.waitForExport = builder.waitForExport; + } + + public static Builder builder(DynamoDbStreamsClient streamsClient) { + return new Builder(streamsClient); + } + + + static class Builder { + + private final DynamoDbStreamsClient dynamoDbStreamsClient; + + + private StreamRecordConverter recordConverter; + + private StreamCheckpointer checkpointer; + + private String shardIterator; + + + private Instant startTime; + + private boolean waitForExport; + + public Builder(DynamoDbStreamsClient dynamoDbStreamsClient) { + this.dynamoDbStreamsClient = dynamoDbStreamsClient; + } + + public Builder recordConverter(StreamRecordConverter recordConverter) { + this.recordConverter = recordConverter; + return this; + } + + public Builder checkpointer(StreamCheckpointer checkpointer) { + this.checkpointer = checkpointer; + return this; + } + + public Builder shardIterator(String shardIterator) { + this.shardIterator = shardIterator; + return this; + } + + public Builder startTime(Instant startTime) { + this.startTime = startTime; + return this; + } + + public Builder waitForExport(boolean waitForExport) { + this.waitForExport = waitForExport; + return this; + } + + public ShardConsumer build() { + return new ShardConsumer(this); + } + + } + + + @Override + public void run() { + LOG.debug("Shard Consumer start to run..."); + + long lastCheckpointTime = System.currentTimeMillis(); + String sequenceNumber = ""; + + while (!shouldStop) { + if (shardIterator == null) { + // End of Shard + LOG.debug("Reach end of shard"); + checkpointer.checkpoint(sequenceNumber); + break; + } + + if (System.currentTimeMillis() - lastCheckpointTime > DEFAULT_CHECKPOINT_INTERVAL_MILLS) { + LOG.debug("Perform regular checkpointing for Shard Consumer"); + checkpointer.checkpoint(sequenceNumber); + lastCheckpointTime = System.currentTimeMillis(); + } + + // Use the shard iterator to read the stream records + GetRecordsRequest req = GetRecordsRequest.builder() + .shardIterator(shardIterator) + .limit(MAX_GET_RECORD_ITEM_COUNT) + .build(); + + + List records; + GetRecordsResponse response; + try { + response = dynamoDbStreamsClient.getRecords(req); + } catch (SdkException e) { + checkpointer.checkpoint(sequenceNumber); + throw e; + } + + shardIterator = response.nextShardIterator(); + + if (!response.records().isEmpty()) { + // Always use the last sequence number for checkpoint + sequenceNumber = response.records().get(response.records().size() - 1).dynamodb().sequenceNumber(); + + if (waitForExport) { + Instant lastEventTime = response.records().get(response.records().size() - 1).dynamodb().approximateCreationDateTime(); + if (lastEventTime.compareTo(startTime) <= 0) { + LOG.debug("Get {} events before start time, ignore...", response.records().size()); + continue; + } + checkpointer.checkpoint(sequenceNumber); + waitForExport(); + waitForExport = false; + + records = response.records().stream() + .filter(record -> record.dynamodb().approximateCreationDateTime().compareTo(startTime) > 0) + .collect(Collectors.toList()); + } else { + records = response.records(); + } + recordConverter.writeToBuffer(records); + } + try { + // Idle between get records call. + Thread.sleep(GET_RECORD_INTERVAL_MILLS); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + // interrupted + if (shouldStop) { + // Do last checkpoint and then quit + LOG.error("Should Stop flag is set to True, looks like shutdown has triggered"); + checkpointer.checkpoint(sequenceNumber); + throw new RuntimeException("Shard Consumer is interrupted"); + } + } + + private void waitForExport() { + LOG.debug("Start waiting for export to be done and loaded"); + while (!checkpointer.isExportDone()) { + LOG.debug("Export is in progress, wait..."); + try { + Thread.sleep(DEFAULT_WAIT_FOR_EXPORT_INTERVAL_MILLS); + } catch (InterruptedException e) { + LOG.error("Wait for export is interrupted ({})", e.getMessage()); + // Directly quit the process + throw new RuntimeException("Wait for export is interrupted."); + } + } + } + + + /** + * Currently, this is to stop all consumers. + */ + public static void stopAll() { + shouldStop = true; + } + + +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumerFactory.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumerFactory.java new file mode 100644 index 0000000000..6e141c6e27 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumerFactory.java @@ -0,0 +1,100 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.stream; + +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.converter.StreamRecordConverter; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; + +import java.time.Instant; +import java.util.Optional; + +/** + * Factory class to create shard consumers + */ +public class ShardConsumerFactory { + private static final Logger LOG = LoggerFactory.getLogger(ShardManager.class); + + private static final int STREAM_TO_TABLE_OFFSET = "stream/".length(); + + private final DynamoDbStreamsClient streamsClient; + + private final EnhancedSourceCoordinator enhancedSourceCoordinator; + private final PluginMetrics pluginMetrics; + private final ShardManager shardManager; + + private final Buffer> buffer; + + public ShardConsumerFactory(EnhancedSourceCoordinator enhancedSourceCoordinator, DynamoDbStreamsClient streamsClient, PluginMetrics pluginMetrics, ShardManager shardManager, Buffer> buffer) { + this.streamsClient = streamsClient; + this.enhancedSourceCoordinator = enhancedSourceCoordinator; + this.pluginMetrics = pluginMetrics; + this.shardManager = shardManager; + this.buffer = buffer; + + } + + public Runnable createConsumer(StreamPartition streamPartition) { + + LOG.debug("Try to create a thread for shard " + streamPartition.getShardId()); + + // Check and get the current state. + Optional progressState = streamPartition.getProgressState(); + String sequenceNumber = null; + Instant startTime = null; + boolean waitForExport = false; + if (progressState.isPresent()) { + sequenceNumber = progressState.get().getSequenceNumber(); + waitForExport = progressState.get().shouldWaitForExport(); + if (progressState.get().getStartTime() != 0) { + startTime = Instant.ofEpochMilli(progressState.get().getStartTime()); + } + } + + String shardIter = shardManager.getShardIterator(streamPartition.getStreamArn(), streamPartition.getShardId(), sequenceNumber); + if (shardIter == null) { + LOG.error("Unable to get a shard iterator, looks like the shard has expired"); + return null; + } + + StreamCheckpointer checkpointer = new StreamCheckpointer(enhancedSourceCoordinator, streamPartition); + String tableArn = getTableArn(streamPartition.getStreamArn()); + TableInfo tableInfo = getTableInfo(tableArn); + StreamRecordConverter recordConverter = new StreamRecordConverter(buffer, tableInfo, pluginMetrics); + + ShardConsumer shardConsumer = ShardConsumer.builder(streamsClient) + .recordConverter(recordConverter) + .checkpointer(checkpointer) + .shardIterator(shardIter) + .startTime(startTime) + .waitForExport(waitForExport) + .build(); + return shardConsumer; + } + + private TableInfo getTableInfo(String tableArn) { + GlobalState tableState = (GlobalState) enhancedSourceCoordinator.getPartition(tableArn).get(); + TableInfo tableInfo = new TableInfo(tableArn, TableMetadata.fromMap(tableState.getProgressState().get())); + return tableInfo; + } + + private String getTableArn(String streamArn) { + // e.g. Given a stream arn: arn:aws:dynamodb:us-west-2:xxx:table/test-table/stream/2023-07-31T04:59:58.190 + // Returns arn:aws:dynamodb:us-west-2:xxx:table/test-table + return streamArn.substring(0, streamArn.lastIndexOf('/') - STREAM_TO_TABLE_OFFSET); + } +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardManager.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardManager.java new file mode 100644 index 0000000000..c4383d9c5a --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardManager.java @@ -0,0 +1,158 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.stream; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.exception.SdkException; +import software.amazon.awssdk.services.dynamodb.model.DescribeStreamRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeStreamResponse; +import software.amazon.awssdk.services.dynamodb.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.dynamodb.model.GetShardIteratorResponse; +import software.amazon.awssdk.services.dynamodb.model.Shard; +import software.amazon.awssdk.services.dynamodb.model.ShardIteratorType; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; + +import java.util.List; +import java.util.stream.Collectors; + +/** + * Utility for manage shards. + */ +public class ShardManager { + + private static final Logger LOG = LoggerFactory.getLogger(ShardManager.class); + + private final DynamoDbStreamsClient streamsClient; + + public ShardManager(DynamoDbStreamsClient streamsClient) { + this.streamsClient = streamsClient; + } + + + private List listShards(String streamArn) { + // Get all the shard IDs from the stream. + List shards; + String lastEvaluatedShardId = null; + do { + DescribeStreamRequest req = DescribeStreamRequest.builder() + .streamArn(streamArn) + .exclusiveStartShardId(lastEvaluatedShardId) + .build(); + + DescribeStreamResponse describeStreamResult = streamsClient.describeStream(req); + shards = describeStreamResult.streamDescription().shards(); + + // If LastEvaluatedShardId is set, + // at least one more page of shard IDs to retrieve + lastEvaluatedShardId = describeStreamResult.streamDescription().lastEvaluatedShardId(); + } while (lastEvaluatedShardId != null); + + LOG.debug("Stream {} has {} shards found", streamArn, shards.size()); + return shards; + } + + /** + * Get a list of Child Shard Ids based on a parent shard id provided. + * + * @param streamArn Stream Arn + * @param shardId Parent Shard Id + * @return A list of child shard Ids. + */ + public List getChildShardIds(String streamArn, String shardId) { + LOG.debug("Getting child ids for " + shardId); + List shards = listShards(streamArn); + return shards.stream() + .filter(s -> shardId.equals(s.parentShardId())) + .map(s -> s.shardId()) + .collect(Collectors.toList()); + } + + /** + * Get a list of active/open shards for a Stream. + * They don't have an ending sequence number and is currently active for write. + * + * @param streamArn Stream Arn + * @return A list of shard Ids + */ + public List getActiveShards(String streamArn) { + List shards = listShards(streamArn); + return shards.stream() + .filter(s -> s.sequenceNumberRange().endingSequenceNumber() == null) + .map(s -> s.shardId()) + .collect(Collectors.toList()); + } + + + /** + * Get a shard iterator to start reading stream records from a shard. + * If sequence number is provided, use AFTER_SEQUENCE_NUMBER to retrieve the iterator, + * otherwise use TRIM_HORIZON to retrieve the iterator. + *

+ * Note that the shard may be expired, if so, null will be returned. + *

+ * + * @param streamArn Stream Arn + * @param shardId Shard Id + * @param sequenceNumber The last Sequence Number processed if any + * @return A shard iterator. + */ + public String getShardIterator(String streamArn, String shardId, String sequenceNumber) { + LOG.debug("Get Initial Shard Iter for " + shardId); + GetShardIteratorRequest getShardIteratorRequest; + + if (sequenceNumber != null && !sequenceNumber.isEmpty()) { + LOG.debug("Get Shard Iterator after " + sequenceNumber); + getShardIteratorRequest = GetShardIteratorRequest.builder() + .shardId(shardId) + .streamArn(streamArn) + .shardIteratorType(ShardIteratorType.AFTER_SEQUENCE_NUMBER) + .sequenceNumber(sequenceNumber) + .build(); + + } else { + LOG.debug("Get Shard Iterator from beginning (TRIM_HORIZON)"); + getShardIteratorRequest = GetShardIteratorRequest.builder() + .shardId(shardId) + .streamArn(streamArn) + .shardIteratorType(ShardIteratorType.TRIM_HORIZON) + .build(); + } + + + try { + GetShardIteratorResponse getShardIteratorResult = streamsClient.getShardIterator(getShardIteratorRequest); + String currentShardIter = getShardIteratorResult.shardIterator(); + return currentShardIter; + } catch (SdkException e) { + LOG.error("Exception when trying to get the shard iterator"); + LOG.error(e.getMessage()); + return null; + } + + + } + + /** + * Get a list of root shard Ids. + * A root shard is a shard whose parent shard is not in the list or whose parent id is null. + * + * @param streamArn Stream Arn + * @return A list of root shard Ids + */ + public List getRootShardIds(String streamArn) { + List shards = listShards(streamArn); + + List childIds = shards.stream().map(shard -> shard.shardId()).collect(Collectors.toList()); + List rootIds = shards.stream() + .filter(shard -> shard.parentShardId() == null || !childIds.contains(shard.parentShardId())) + .map(shard -> shard.shardId()) + .collect(Collectors.toList()); + + return rootIds; + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/StreamCheckpointer.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/StreamCheckpointer.java new file mode 100644 index 0000000000..9407576b83 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/StreamCheckpointer.java @@ -0,0 +1,94 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.stream; + +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.StreamProgressState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Optional; + +/** + * A helper class to handle the stream partition status and the progress state + * It will use coordinator APIs under the hood. + */ +public class StreamCheckpointer { + private static final Logger LOG = LoggerFactory.getLogger(StreamCheckpointer.class); + + private final EnhancedSourceCoordinator coordinator; + + private final StreamPartition streamPartition; + + public StreamCheckpointer(EnhancedSourceCoordinator coordinator, StreamPartition streamPartition) { + this.coordinator = coordinator; + this.streamPartition = streamPartition; + } + + private void setSequenceNumber(String sequenceNumber) { + // Must only update progress if sequence number is not empty + // A blank sequence number means the current sequence number in the progress state has not changed + if (sequenceNumber != null && !sequenceNumber.isEmpty()) { + Optional progressState = streamPartition.getProgressState(); + if (progressState.isPresent()) { + progressState.get().setSequenceNumber(sequenceNumber); + } else { + + } + + } + } + + /** + * This method is to do a checkpoint with latest sequence number processed. + * Note that this should be called on a regular basis even there are no changes to sequence number + * As the checkpoint will also extend the timeout for the lease + * + * @param sequenceNumber The last sequence number + */ + + public void checkpoint(String sequenceNumber) { + LOG.debug("Checkpoint shard " + streamPartition.getShardId() + " with sequenceNumber " + sequenceNumber); + setSequenceNumber(sequenceNumber); + coordinator.saveProgressStateForPartition(streamPartition); + + } + + /** + * This method is to mark the shard partition as COMPLETED with the final sequence number + * Note that this should be called when reaching the end of shard. + * + * @param sequenceNumber The last sequence number + */ + + public void complete(String sequenceNumber) { + LOG.debug("Complete the read of shard " + streamPartition.getShardId() + " with final sequenceNumber " + sequenceNumber); + setSequenceNumber(sequenceNumber); + coordinator.completePartition(streamPartition); + + } + + /** + * This method is to release the lease of the stream partition. + * Normally this should only be called due to failures or interruption. + * + * @param sequenceNumber The last sequence number + */ + public void release(String sequenceNumber) { + LOG.debug("Release the ownership of shard " + streamPartition.getShardId() + " with final sequenceNumber " + sequenceNumber); + setSequenceNumber(sequenceNumber); + coordinator.giveUpPartition(streamPartition); + + } + + public boolean isExportDone() { + Optional globalPartition = coordinator.getPartition(streamPartition.getStreamArn()); + return globalPartition.isPresent(); + } + +} diff --git a/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/StreamScheduler.java b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/StreamScheduler.java new file mode 100644 index 0000000000..cd7775c5ba --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/main/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/StreamScheduler.java @@ -0,0 +1,125 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.stream; + +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.StreamProgressState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; + +/** + * A scheduler to manage all the stream related work in one place + */ +public class StreamScheduler implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(StreamScheduler.class); + + private static final int MAX_JOB_COUNT = 50; + private static final int DEFAULT_LEASE_INTERVAL_MILLIS = 30_000; + + private final AtomicInteger numOfWorkers = new AtomicInteger(0); + private final EnhancedSourceCoordinator coordinator; + private final ShardConsumerFactory consumerFactory; + private final ExecutorService executor; + private final ShardManager shardManager; + + + public StreamScheduler(EnhancedSourceCoordinator coordinator, ShardConsumerFactory consumerFactory, ShardManager shardManager) { + this.coordinator = coordinator; + this.shardManager = shardManager; + this.consumerFactory = consumerFactory; + + executor = Executors.newFixedThreadPool(MAX_JOB_COUNT); + + } + + private void processStreamPartition(StreamPartition streamPartition) { + Runnable shardConsumer = consumerFactory.createConsumer(streamPartition); + if (shardConsumer != null) { + CompletableFuture runConsumer = CompletableFuture.runAsync(shardConsumer, executor); + runConsumer.whenComplete(completeConsumer(streamPartition)); + numOfWorkers.incrementAndGet(); + } else { + // If failed to create a new consumer. + coordinator.completePartition(streamPartition); + } + } + + @Override + public void run() { + LOG.debug("Stream Scheduler start to run..."); + while (!Thread.interrupted()) { + if (numOfWorkers.get() < MAX_JOB_COUNT) { + final Optional sourcePartition = coordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE); + if (sourcePartition.isPresent()) { + StreamPartition streamPartition = (StreamPartition) sourcePartition.get(); + processStreamPartition(streamPartition); + } + } + + try { + Thread.sleep(DEFAULT_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException e) { + LOG.info("InterruptedException occurred"); + break; + } + } + // Should Stop + LOG.debug("Stream Scheduler is interrupted, looks like shutdown has triggered"); + + // Cannot call executor.shutdownNow() here + // Otherwise the final checkpoint will fail due to SDK interruption. + ShardConsumer.stopAll(); + executor.shutdown(); + } + + private BiConsumer completeConsumer(StreamPartition streamPartition) { + return (v, ex) -> { + numOfWorkers.decrementAndGet(); + if (ex == null) { + LOG.debug("Shard consumer is completed"); + LOG.debug("Start creating new stream partitions for Child Shards"); + + List childShardIds = shardManager.getChildShardIds(streamPartition.getStreamArn(), streamPartition.getShardId()); + LOG.debug("Child Ids Retrieved: {}", childShardIds); + + createStreamPartitions(streamPartition.getStreamArn(), childShardIds); + LOG.debug("Create child shard completed"); + // Finally mask the partition as completed. + coordinator.completePartition(streamPartition); + + } else { + // Do nothing + // The consumer must have already done one last checkpointing. + LOG.debug("Shard consumer completed with exception"); + LOG.error(ex.toString()); + coordinator.giveUpPartition(streamPartition); + + + } + }; + } + + private void createStreamPartitions(String streamArn, List shardIds) { + shardIds.forEach( + shardId -> { + StreamPartition partition = new StreamPartition(streamArn, shardId, Optional.of(new StreamProgressState())); + coordinator.createPartition(partition); + } + ); + } + + +} diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBServiceTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBServiceTest.java new file mode 100644 index 0000000000..7b6a9b7531 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBServiceTest.java @@ -0,0 +1,312 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.InvalidPluginConfigurationException; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.ExportConfig; +import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.StreamConfig; +import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.TableConfig; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.InitPartition; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.dynamodb.model.ContinuousBackupsDescription; +import software.amazon.awssdk.services.dynamodb.model.ContinuousBackupsStatus; +import software.amazon.awssdk.services.dynamodb.model.DescribeContinuousBackupsRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeContinuousBackupsResponse; +import software.amazon.awssdk.services.dynamodb.model.DescribeStreamRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeStreamResponse; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeTableResponse; +import software.amazon.awssdk.services.dynamodb.model.KeySchemaElement; +import software.amazon.awssdk.services.dynamodb.model.KeyType; +import software.amazon.awssdk.services.dynamodb.model.PointInTimeRecoveryDescription; +import software.amazon.awssdk.services.dynamodb.model.PointInTimeRecoveryStatus; +import software.amazon.awssdk.services.dynamodb.model.SequenceNumberRange; +import software.amazon.awssdk.services.dynamodb.model.Shard; +import software.amazon.awssdk.services.dynamodb.model.StreamDescription; +import software.amazon.awssdk.services.dynamodb.model.StreamSpecification; +import software.amazon.awssdk.services.dynamodb.model.StreamViewType; +import software.amazon.awssdk.services.dynamodb.model.TableDescription; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; +import software.amazon.awssdk.services.s3.S3Client; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class DynamoDBServiceTest { + + @Mock + private EnhancedSourceCoordinator coordinator; + + @Mock + private ClientFactory clientFactory; + + @Mock + private DynamoDbClient dynamoDbClient; + + @Mock + private DynamoDbStreamsClient dynamoDbStreamsClient; + + @Mock + private S3Client s3Client; + + @Mock + private DynamoDBSourceConfig sourceConfig; + + @Mock + private TableConfig tableConfig; + + @Mock + private ExportConfig exportConfig; + + @Mock + private StreamConfig streamConfig; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private InitPartition initPartition; + + @Mock + private Buffer> buffer; + + private DynamoDBService dynamoDBService; + + private Collection keySchema; + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String partitionKeyAttrName = "PK"; + private final String sortKeyAttrName = "SK"; + private final String bucketName = UUID.randomUUID().toString(); + private final String prefix = UUID.randomUUID().toString(); + + private final String exportArn = tableArn + "/export/01693291918297-bfeccbea"; + private final String streamArn = tableArn + "/stream/2023-09-14T05:46:45.367"; + + private final String shardId = "shardId-" + UUID.randomUUID(); + + private final long exportTimeMills = 1695021857760L; + private final Instant exportTime = Instant.ofEpochMilli(exportTimeMills); + + + @BeforeEach + void setup() { + + KeySchemaElement pk = KeySchemaElement.builder() + .attributeName(partitionKeyAttrName) + .keyType(KeyType.HASH) + .build(); + KeySchemaElement sk = KeySchemaElement.builder() + .attributeName(sortKeyAttrName) + .keyType(KeyType.RANGE) + .build(); + + keySchema = List.of(pk, sk); + + // Mock Client Factory + lenient().when(clientFactory.buildS3Client()).thenReturn(s3Client); + lenient().when(clientFactory.buildDynamoDBClient()).thenReturn(dynamoDbClient); + lenient().when(clientFactory.buildDynamoDbStreamClient()).thenReturn(dynamoDbStreamsClient); + // Mock configurations + lenient().when(exportConfig.getS3Bucket()).thenReturn(bucketName); + lenient().when(exportConfig.getS3Prefix()).thenReturn(prefix); + lenient().when(streamConfig.getStartPosition()).thenReturn("LATEST"); + lenient().when(tableConfig.getTableArn()).thenReturn(tableArn); + lenient().when(tableConfig.getExportConfig()).thenReturn(exportConfig); + lenient().when(tableConfig.getStreamConfig()).thenReturn(streamConfig); + lenient().when(sourceConfig.getTableConfigs()).thenReturn(List.of(tableConfig)); + + // Mock SDK Calls + lenient().when(dynamoDbStreamsClient.describeStream(any(DescribeStreamRequest.class))).thenReturn(generateDescribeStreamResponse()); + + DescribeTableResponse defaultDescribeTableResponse = generateDescribeTableResponse(StreamViewType.NEW_IMAGE); + lenient().when(dynamoDbClient.describeTable(any(DescribeTableRequest.class))).thenReturn(defaultDescribeTableResponse); + + DescribeContinuousBackupsResponse defaultDescribePITRResponse = generatePITRResponse(true); + lenient().when(dynamoDbClient.describeContinuousBackups(any(DescribeContinuousBackupsRequest.class))).thenReturn(defaultDescribePITRResponse); + + } + + private DynamoDBService createObjectUnderTest() { + DynamoDBService objectUnderTest = new DynamoDBService(coordinator, clientFactory, sourceConfig, pluginMetrics); + return objectUnderTest; + } + + @Test + void test_normal_start() { + dynamoDBService = createObjectUnderTest(); + assertThat(dynamoDBService, notNullValue()); + dynamoDBService.start(buffer); + + } + + + @Test + void test_normal_shutdown() { + dynamoDBService = createObjectUnderTest(); + assertThat(dynamoDBService, notNullValue()); + dynamoDBService.shutdown(); + } + + + @Test + void test_should_init() { + + given(coordinator.acquireAvailablePartition(InitPartition.PARTITION_TYPE)).willReturn(Optional.of(initPartition)).willReturn(Optional.empty()); + + dynamoDBService = createObjectUnderTest(); + dynamoDBService.init(); + // Should call describe table to get basic table info + verify(dynamoDbClient).describeTable(any(DescribeTableRequest.class)); + // Should check PITR enabled or not + verify(dynamoDbClient).describeContinuousBackups(any(DescribeContinuousBackupsRequest.class)); + // Acquire the init partition + verify(coordinator).acquireAvailablePartition(eq(InitPartition.PARTITION_TYPE)); + // Complete the init partition + verify(coordinator).completePartition(any(SourcePartition.class)); + + // Should create 1 export partition + 1 stream partition + 1 global table state + verify(coordinator, times(3)).createPartition(any(SourcePartition.class)); + } + + @Test + void test_already_init() { + given(coordinator.acquireAvailablePartition(InitPartition.PARTITION_TYPE)).willReturn(Optional.empty()); + dynamoDBService = createObjectUnderTest(); + dynamoDBService.init(); + + verifyNoInteractions(dynamoDbClient); + } + + @Test + void test_PITR_not_enabled_should_throw_errors() { + + given(coordinator.acquireAvailablePartition(InitPartition.PARTITION_TYPE)).willReturn(Optional.of(initPartition)).willReturn(Optional.empty()); + + // If PITR is not enabled + DescribeContinuousBackupsResponse response = generatePITRResponse(false); + when(dynamoDbClient.describeContinuousBackups(any(DescribeContinuousBackupsRequest.class))).thenReturn(response); + + dynamoDBService = createObjectUnderTest(); + assertThrows( + InvalidPluginConfigurationException.class, + () -> dynamoDBService.init()); + + } + + @Test + void test_streaming_not_enabled_should_throw_errors() { + + given(coordinator.acquireAvailablePartition(InitPartition.PARTITION_TYPE)).willReturn(Optional.of(initPartition)).willReturn(Optional.empty()); + + // If streaming is not enabled + DescribeTableResponse defaultDescribeTableResponse = generateDescribeTableResponse(null); + lenient().when(dynamoDbClient.describeTable(any(DescribeTableRequest.class))).thenReturn(defaultDescribeTableResponse); + + dynamoDBService = createObjectUnderTest(); + assertThrows( + InvalidPluginConfigurationException.class, + () -> dynamoDBService.init()); + + } + + + /** + * Helper function to mock DescribeContinuousBackupsResponse + */ + private DescribeContinuousBackupsResponse generatePITRResponse(boolean enabled) { + PointInTimeRecoveryDescription pointInTimeRecoveryDescription = PointInTimeRecoveryDescription.builder() + .pointInTimeRecoveryStatus(enabled ? PointInTimeRecoveryStatus.ENABLED : PointInTimeRecoveryStatus.DISABLED) + .build(); + ContinuousBackupsDescription continuousBackupsDescription = ContinuousBackupsDescription.builder() + .continuousBackupsStatus(ContinuousBackupsStatus.ENABLED) + .pointInTimeRecoveryDescription(pointInTimeRecoveryDescription) + .build(); + DescribeContinuousBackupsResponse response = DescribeContinuousBackupsResponse.builder() + .continuousBackupsDescription(continuousBackupsDescription).build(); + + return response; + } + + /** + * Helper function to mock DescribeStreamResponse + */ + private DescribeStreamResponse generateDescribeStreamResponse() { + Shard shard = Shard.builder() + .shardId(shardId) + .parentShardId(null) + .sequenceNumberRange(SequenceNumberRange.builder() + .endingSequenceNumber(null) + .startingSequenceNumber(UUID.randomUUID().toString()) + .build()) + .build(); + + List shardList = new ArrayList<>(); + shardList.add(shard); + + + StreamDescription description = StreamDescription.builder() + .shards(shardList) + .lastEvaluatedShardId(null) + .build(); + DescribeStreamResponse describeStreamResponse = DescribeStreamResponse.builder() + .streamDescription(description) + .build(); + return describeStreamResponse; + } + + /** + * Helper function to mock DescribeTableResponse + */ + private DescribeTableResponse generateDescribeTableResponse(StreamViewType viewType) { + StreamSpecification streamSpecification = StreamSpecification.builder() + .streamEnabled(viewType == null) + .streamViewType(viewType == null ? StreamViewType.UNKNOWN_TO_SDK_VERSION : viewType) + .build(); + + TableDescription tableDescription = TableDescription.builder() + .keySchema(keySchema) + .tableName(tableName) + .tableArn(tableArn) + .latestStreamArn(streamArn) + .streamSpecification(streamSpecification) + .build(); + DescribeTableResponse response = DescribeTableResponse.builder() + .table(tableDescription) + .build(); + return response; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfigTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfigTest.java new file mode 100644 index 0000000000..0e11d82e57 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/DynamoDBSourceConfigTest.java @@ -0,0 +1,86 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.dataformat.yaml.YAMLFactory; +import com.fasterxml.jackson.dataformat.yaml.YAMLGenerator; +import org.junit.jupiter.api.Test; +import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.AwsAuthenticationConfig; +import org.opensearch.dataprepper.plugins.source.dynamodb.configuration.TableConfig; +import software.amazon.awssdk.regions.Region; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertNull; + +class DynamoDBSourceConfigTest { + + private final ObjectMapper objectMapper = new ObjectMapper(new YAMLFactory().enable(YAMLGenerator.Feature.USE_PLATFORM_LINE_BREAKS)); + + @Test + void test_general_config() throws JsonProcessingException { + + final String sourceConfigurationYaml = "tables:\n" + + " - table_arn: \"arn:aws:dynamodb:us-west-2:123456789012:table/table-a\"\n" + + " export:\n" + + " s3_bucket: \"test-bucket\"\n" + + " s3_prefix: \"xxx/\"\n" + + " stream:\n" + + " start_position: \n" + + " - table_arn: \"arn:aws:dynamodb:us-west-2:123456789012:table/table-b\"\n" + + " export:\n" + + " s3_bucket: \"test-bucket\"\n" + + " s3_prefix: \"xxx/\"\n" + + " - table_arn: \"arn:aws:dynamodb:us-west-2:123456789012:table/table-c\"\n" + + " stream:\n" + + " start_position: \"BEGINNING\" \n" + + "aws:\n" + + " region: \"us-west-2\"\n" + + " sts_role_arn: \"arn:aws:iam::123456789012:role/DataPrepperRole\"\n" + + "coordinator:\n" + + " dynamodb:\n" + + " table_name: \"coordinator-table\"\n" + + " region: \"us-west-2\""; + final DynamoDBSourceConfig sourceConfiguration = objectMapper.readValue(sourceConfigurationYaml, DynamoDBSourceConfig.class); + + assertThat(sourceConfiguration.getAwsAuthenticationConfig(), notNullValue()); + assertThat(sourceConfiguration.getCoordinationStoreConfig(), notNullValue()); + assertThat(sourceConfiguration.getTableConfigs(), notNullValue()); + assertThat(sourceConfiguration.getTableConfigs().size(), equalTo(3)); + + TableConfig exportAndStreamConfig = sourceConfiguration.getTableConfigs().get(0); + assertThat(exportAndStreamConfig.getExportConfig(), notNullValue()); + assertThat(exportAndStreamConfig.getExportConfig().getS3Bucket(), equalTo("test-bucket")); + assertThat(exportAndStreamConfig.getExportConfig().getS3Prefix(), equalTo("xxx/")); + assertThat(exportAndStreamConfig.getStreamConfig(), notNullValue()); + assertNull(exportAndStreamConfig.getStreamConfig().getStartPosition()); + + + TableConfig exportOnlyConfig = sourceConfiguration.getTableConfigs().get(1); + assertThat(exportOnlyConfig.getExportConfig(), notNullValue()); + assertThat(exportOnlyConfig.getExportConfig().getS3Bucket(), equalTo("test-bucket")); + assertThat(exportOnlyConfig.getExportConfig().getS3Prefix(), equalTo("xxx/")); + assertNull(exportOnlyConfig.getStreamConfig()); + + + TableConfig streamOnlyConfig = sourceConfiguration.getTableConfigs().get(2); + assertThat(streamOnlyConfig.getStreamConfig(), notNullValue()); + assertThat(streamOnlyConfig.getStreamConfig().getStartPosition(), equalTo("BEGINNING")); + assertNull(streamOnlyConfig.getExportConfig()); + + AwsAuthenticationConfig awsAuthenticationConfig = sourceConfiguration.getAwsAuthenticationConfig(); + assertThat(awsAuthenticationConfig, notNullValue()); + assertThat(awsAuthenticationConfig.getAwsRegion(), equalTo(Region.US_WEST_2)); + assertThat(awsAuthenticationConfig.getAwsStsRoleArn(), equalTo("arn:aws:iam::123456789012:role/DataPrepperRole")); + assertNull(awsAuthenticationConfig.getAwsStsExternalId()); + assertNull(awsAuthenticationConfig.getAwsStsHeaderOverrides()); + + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/ExportRecordConverterTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/ExportRecordConverterTest.java new file mode 100644 index 0000000000..d3f7175867 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/ExportRecordConverterTest.java @@ -0,0 +1,112 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.converter; + +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableMetadata; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.anyDouble; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.opensearch.dataprepper.plugins.source.dynamodb.converter.ExportRecordConverter.EXPORT_RECORD_ERROR_COUNT; +import static org.opensearch.dataprepper.plugins.source.dynamodb.converter.ExportRecordConverter.EXPORT_RECORD_SUCCESS_COUNT; + +@ExtendWith(MockitoExtension.class) +class ExportRecordConverterTest { + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private Buffer> buffer; + + private TableInfo tableInfo; + + @Mock + private Counter exportRecordSuccess; + + @Mock + private Counter exportRecordErrors; + + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String partitionKeyAttrName = "PK"; + private final String sortKeyAttrName = "SK"; + + + @BeforeEach + void setup() { + + TableMetadata metadata = TableMetadata.builder() + .exportRequired(true) + .streamRequired(true) + .partitionKeyAttributeName(partitionKeyAttrName) + .sortKeyAttributeName(sortKeyAttrName) + .build(); + + tableInfo = new TableInfo(tableArn, metadata); + + given(pluginMetrics.counter(EXPORT_RECORD_SUCCESS_COUNT)).willReturn(exportRecordSuccess); + given(pluginMetrics.counter(EXPORT_RECORD_ERROR_COUNT)).willReturn(exportRecordErrors); + + } + + private List generateData(int count) { + List result = new ArrayList<>(); + for (int i = 0; i < count; i++) { + final String pk1 = UUID.randomUUID().toString(); + final String sk1 = UUID.randomUUID().toString(); + + result.add(" $ion_1_0 {Item:{PK:\"" + pk1 + "\",SK:\"" + sk1 + "\"}}"); + } + return result; + } + + @Test + void test_writeToBuffer() throws Exception { + + final Random random = new Random(); + + int numberOfRecords = random.nextInt(10); + + List data = generateData(numberOfRecords); + ExportRecordConverter recordConverter = new ExportRecordConverter(buffer, tableInfo, pluginMetrics); + + final ArgumentCaptor>> writeRequestArgumentCaptor = ArgumentCaptor.forClass(Collection.class); + doNothing().when(buffer).writeAll(writeRequestArgumentCaptor.capture(), anyInt()); + recordConverter.writeToBuffer(data); + + assertThat(writeRequestArgumentCaptor.getValue().size(), equalTo(numberOfRecords)); + verify(exportRecordSuccess).increment(anyDouble()); + + verifyNoInteractions(exportRecordErrors); + + } +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/StreamRecordConverterTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/StreamRecordConverterTest.java new file mode 100644 index 0000000000..00dd67bef0 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/converter/StreamRecordConverterTest.java @@ -0,0 +1,146 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.converter; + +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableMetadata; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.OperationType; +import software.amazon.awssdk.services.dynamodb.model.StreamRecord; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.anyDouble; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.opensearch.dataprepper.plugins.source.dynamodb.converter.StreamRecordConverter.CHANGE_EVENT_ERROR_COUNT; +import static org.opensearch.dataprepper.plugins.source.dynamodb.converter.StreamRecordConverter.CHANGE_EVENT_SUCCESS_COUNT; + +@ExtendWith(MockitoExtension.class) +class StreamRecordConverterTest { + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private Buffer> buffer; + + private TableInfo tableInfo; + + @Mock + private Counter changeEventSuccessCounter; + + @Mock + private Counter changeEventErrorCounter; + + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String partitionKeyAttrName = "PK"; + private final String sortKeyAttrName = "SK"; + + + @BeforeEach + void setup() { + + TableMetadata metadata = TableMetadata.builder() + .exportRequired(true) + .streamRequired(true) + .partitionKeyAttributeName(partitionKeyAttrName) + .sortKeyAttributeName(sortKeyAttrName) + .build(); + + tableInfo = new TableInfo(tableArn, metadata); + + given(pluginMetrics.counter(CHANGE_EVENT_SUCCESS_COUNT)).willReturn(changeEventSuccessCounter); + given(pluginMetrics.counter(CHANGE_EVENT_ERROR_COUNT)).willReturn(changeEventErrorCounter); + + } + + + @Test + void test_writeToBuffer() throws Exception { + +// final String pk1 = UUID.randomUUID().toString(); +// final String sk1 = UUID.randomUUID().toString(); +// final String pk2 = UUID.randomUUID().toString(); +// final String sk2 = UUID.randomUUID().toString(); + + +// final String data = " $ion_1_0 {Item:{PK:\"" + pk1 + "\",SK:\"" + sk1 + "\"}}\n" + +// " $ion_1_0 {Item:{PK:\"" + pk2 + "\",SK:\"" + sk2 + "\"}}\n"; + + final Random random = new Random(); + + int numberOfRecords = random.nextInt(10); + + List data = buildRecords(numberOfRecords); + + StreamRecordConverter recordConverter = new StreamRecordConverter(buffer, tableInfo, pluginMetrics); + + final ArgumentCaptor>> writeRequestArgumentCaptor = ArgumentCaptor.forClass(Collection.class); + doNothing().when(buffer).writeAll(writeRequestArgumentCaptor.capture(), anyInt()); + + + recordConverter.writeToBuffer(data); + +// System.out.println(writeRequestArgumentCaptor.capture()); +// System.out.println(writeRequestArgumentCaptor.getValue()); + +// assertThat(awsAuthenticationConfig.getAwsRegion(), equalTo(Region.US_WEST_2)); + assertThat(writeRequestArgumentCaptor.getValue().size(), equalTo(numberOfRecords)); + verify(changeEventSuccessCounter).increment(anyDouble()); + + verifyNoInteractions(changeEventErrorCounter); + + + } + + private List buildRecords(int count) { + List records = new ArrayList<>(); + for (int i = 0; i < count; i++) { + Map data = Map.of( + partitionKeyAttrName, AttributeValue.builder().s(UUID.randomUUID().toString()).build(), + sortKeyAttrName, AttributeValue.builder().s(UUID.randomUUID().toString()).build()); + + StreamRecord streamRecord = StreamRecord.builder() + .newImage(data) + .sequenceNumber(UUID.randomUUID().toString()) + .approximateCreationDateTime(Instant.now()) + .build(); + software.amazon.awssdk.services.dynamodb.model.Record record = software.amazon.awssdk.services.dynamodb.model.Record.builder() + .dynamodb(streamRecord) + .eventName(OperationType.INSERT) + .build(); + records.add(record); + } + + return records; + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/DefaultCoordinatorTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/DefaultCoordinatorTest.java new file mode 100644 index 0000000000..1883e16252 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/DefaultCoordinatorTest.java @@ -0,0 +1,231 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.source.SourceCoordinationStore; +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStatus; +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; + +import java.time.Duration; +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +@ExtendWith(MockitoExtension.class) +class DefaultCoordinatorTest { + + @Mock + private SourceCoordinationStore sourceCoordinationStore; + + @Mock + private SourcePartitionStoreItem sourcePartitionStoreItem; + + private String sourceIdentifier; + + private DefaultEnhancedSourceCoordinator coordinator; + + private final String DEFAULT_PARTITION_TYPE = "TEST"; + + + @BeforeEach + void setup() { + sourceIdentifier = UUID.randomUUID().toString(); + lenient().when(sourcePartitionStoreItem.getSourcePartitionKey()).thenReturn(UUID.randomUUID().toString()); + lenient().when(sourcePartitionStoreItem.getSourceIdentifier()).thenReturn(sourceIdentifier + "|" + DEFAULT_PARTITION_TYPE); + } + + private DefaultEnhancedSourceCoordinator createObjectUnderTest() { + DefaultEnhancedSourceCoordinator coordinator = new DefaultEnhancedSourceCoordinator(sourceCoordinationStore, sourceIdentifier, sourcePartitionStoreItem -> new TestPartition(sourcePartitionStoreItem)); + return coordinator; + } + + + class TestPartition extends SourcePartition { + + private final String partitionType; + private final String partitionKey; + + public TestPartition(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + + String[] split = sourcePartitionStoreItem.getSourceIdentifier().split("\\|"); + if ("GLOBAL".equals(split[1])) { + this.partitionType = null; + } else { + this.partitionType = split[1]; + } + this.partitionKey = sourcePartitionStoreItem.getSourcePartitionKey(); + } + + public TestPartition(boolean isGlobal) { + this.partitionType = isGlobal ? null : DEFAULT_PARTITION_TYPE; + partitionKey = UUID.randomUUID().toString(); + } + + + @Override + public String getPartitionType() { + return partitionType; + } + + @Override + public String getPartitionKey() { + return partitionKey; + } + + @Override + public Optional getProgressState() { + return Optional.empty(); + } + } + + @Test + void test_initialize_should_run_correctly() { + coordinator = createObjectUnderTest(); + coordinator.initialize(); + // Should call initializeStore + verify(sourceCoordinationStore).initializeStore(); + + } + + @Test + void test_createPartition() { + coordinator = createObjectUnderTest(); + // A normal type. + TestPartition partition = new TestPartition(false); + coordinator.createPartition(partition); + verify(sourceCoordinationStore).tryCreatePartitionItem(eq(sourceIdentifier + "|" + DEFAULT_PARTITION_TYPE), anyString(), eq(SourcePartitionStatus.UNASSIGNED), anyLong(), eq(null)); + + // GlobalState. + TestPartition globalState = new TestPartition(true); + coordinator.createPartition(globalState); + verify(sourceCoordinationStore).tryCreatePartitionItem(eq(sourceIdentifier + "|GLOBAL"), anyString(), eq(null), anyLong(), eq(null)); + + } + + @Test + void test_acquireAvailablePartition_should_run_correctly() { + given(sourceCoordinationStore.tryAcquireAvailablePartition(anyString(), anyString(), any())) + .willReturn(Optional.of(sourcePartitionStoreItem)) + .willReturn(Optional.of(sourcePartitionStoreItem)) + .willReturn(Optional.empty()); + coordinator = createObjectUnderTest(); + + Optional sourcePartition = coordinator.acquireAvailablePartition(DEFAULT_PARTITION_TYPE); + assertThat(sourcePartition.isPresent(), equalTo(true)); + + Optional sourcePartition2 = coordinator.acquireAvailablePartition(DEFAULT_PARTITION_TYPE); + assertThat(sourcePartition2.isPresent(), equalTo(true)); + + Optional sourcePartition3 = coordinator.acquireAvailablePartition(DEFAULT_PARTITION_TYPE); + assertThat(sourcePartition3.isPresent(), equalTo(false)); + + verify(sourceCoordinationStore, times(3)).tryAcquireAvailablePartition(anyString(), anyString(), any(Duration.class)); + } + + + @Test + void test_saveProgressStateForPartition() { + + given(sourceCoordinationStore.tryAcquireAvailablePartition(anyString(), anyString(), any())) + .willReturn(Optional.of(sourcePartitionStoreItem)) + .willReturn(Optional.of(sourcePartitionStoreItem)) + .willReturn(Optional.empty()); + coordinator = createObjectUnderTest(); + + Optional sourcePartition = coordinator.acquireAvailablePartition(DEFAULT_PARTITION_TYPE); + assertThat(sourcePartition.isPresent(), equalTo(true)); + TestPartition partition = (TestPartition) sourcePartition.get(); + coordinator.saveProgressStateForPartition(partition); + + verify(sourceCoordinationStore).tryAcquireAvailablePartition(anyString(), anyString(), any(Duration.class)); + verify(sourceCoordinationStore).tryUpdateSourcePartitionItem(any(SourcePartitionStoreItem.class)); + + } + + @Test + void test_giveUpPartition() { + given(sourceCoordinationStore.tryAcquireAvailablePartition(anyString(), anyString(), any())).willReturn(Optional.of(sourcePartitionStoreItem)); + + coordinator = createObjectUnderTest(); + + Optional sourcePartition = coordinator.acquireAvailablePartition(DEFAULT_PARTITION_TYPE); + assertThat(sourcePartition.isPresent(), equalTo(true)); + TestPartition partition = (TestPartition) sourcePartition.get(); + + coordinator.giveUpPartition(partition); + + verify(sourcePartitionStoreItem).setSourcePartitionStatus(SourcePartitionStatus.UNASSIGNED); + verify(sourcePartitionStoreItem).setPartitionOwnershipTimeout(null); + verify(sourcePartitionStoreItem).setPartitionOwner(null); + + verify(sourceCoordinationStore).tryAcquireAvailablePartition(anyString(), anyString(), any(Duration.class)); + verify(sourceCoordinationStore).tryUpdateSourcePartitionItem(any(SourcePartitionStoreItem.class)); + } + + @Test + void test_completePartition() { + given(sourceCoordinationStore.tryAcquireAvailablePartition(anyString(), anyString(), any())).willReturn(Optional.of(sourcePartitionStoreItem)); + coordinator = createObjectUnderTest(); + + Optional sourcePartition = coordinator.acquireAvailablePartition(DEFAULT_PARTITION_TYPE); + assertThat(sourcePartition.isPresent(), equalTo(true)); + TestPartition partition = (TestPartition) sourcePartition.get(); + + coordinator.completePartition(partition); + + verify(sourcePartitionStoreItem).setSourcePartitionStatus(SourcePartitionStatus.COMPLETED); + verify(sourcePartitionStoreItem).setReOpenAt(null); + verify(sourcePartitionStoreItem).setPartitionOwnershipTimeout(null); + verify(sourcePartitionStoreItem).setPartitionOwner(null); + + verify(sourceCoordinationStore).tryAcquireAvailablePartition(anyString(), anyString(), any(Duration.class)); + verify(sourceCoordinationStore).tryUpdateSourcePartitionItem(any(SourcePartitionStoreItem.class)); + } + + @Test + void test_closePartition() { + given(sourceCoordinationStore.tryAcquireAvailablePartition(anyString(), anyString(), any())).willReturn(Optional.of(sourcePartitionStoreItem)); + coordinator = createObjectUnderTest(); + + Optional sourcePartition = coordinator.acquireAvailablePartition(DEFAULT_PARTITION_TYPE); + assertThat(sourcePartition.isPresent(), equalTo(true)); + TestPartition partition = (TestPartition) sourcePartition.get(); + + coordinator.closePartition(partition, Duration.ofMinutes(10), 1); + verify(sourcePartitionStoreItem).setSourcePartitionStatus(SourcePartitionStatus.CLOSED); + verify(sourcePartitionStoreItem).setPartitionOwnershipTimeout(null); + verify(sourcePartitionStoreItem).setPartitionOwner(null); + + verify(sourceCoordinationStore).tryAcquireAvailablePartition(anyString(), anyString(), any(Duration.class)); + verify(sourceCoordinationStore).tryUpdateSourcePartitionItem(any(SourcePartitionStoreItem.class)); + + } + + @Test + void getPartition() { + String partitionKey = UUID.randomUUID().toString(); + given(sourceCoordinationStore.getSourcePartitionItem(eq(sourceIdentifier + "|GLOBAL"), eq(partitionKey))).willReturn(Optional.of(sourcePartitionStoreItem)); + coordinator = createObjectUnderTest(); + Optional sourcePartition = coordinator.getPartition(partitionKey); + assertThat(sourcePartition.isPresent(), equalTo(true)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/PartitionFactoryTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/PartitionFactoryTest.java new file mode 100644 index 0000000000..749213ae86 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/coordination/PartitionFactoryTest.java @@ -0,0 +1,180 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.coordination; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.InitPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.StreamPartition; + +import java.time.Instant; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class PartitionFactoryTest { + + @Mock + private SourcePartitionStoreItem sourcePartitionStoreItem; + + private final String sourceIdentifier = UUID.randomUUID().toString(); + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + private final String bucketName = UUID.randomUUID().toString(); + + private final String prefix = UUID.randomUUID().toString(); + private final String s3Key = UUID.randomUUID().toString(); + private final String sequenceNumber = UUID.randomUUID().toString(); + + private final String exportArn = tableArn + "/export/01693291918297-bfeccbea"; + private final String streamArn = tableArn + "/stream/2023-09-14T05:46:45.367"; + + private final String shardId = "shardId-" + UUID.randomUUID(); + + private final long exportTimeMills = 1695021857760L; + private final Instant exportTime = Instant.ofEpochMilli(exportTimeMills); + + private final Random random = new Random(); + + private final int totalRecords = random.nextInt(10000); + private final int loadedRecords = random.nextInt(10000); + + + @Test + void testCreateExportPartition() { + String sourceId = sourceIdentifier + "|" + ExportPartition.PARTITION_TYPE; + String partitionKey = tableArn + "|" + exportTimeMills; + when(sourcePartitionStoreItem.getSourceIdentifier()).thenReturn(sourceId); + when(sourcePartitionStoreItem.getSourcePartitionKey()).thenReturn(partitionKey); + + String state = "{\"exportArn\":\"" + exportArn + "\",\"bucket\":\"" + bucketName + "\",\"prefix\":\"" + prefix + "\",\"exportTime\":\"2023-09-20T08:07:17.407353Z\"}"; + + when(sourcePartitionStoreItem.getPartitionProgressState()).thenReturn(state); + + PartitionFactory factory = new PartitionFactory(); + SourcePartition sourcePartition = factory.apply(sourcePartitionStoreItem); + assertThat(sourcePartition, notNullValue()); + ExportPartition exportPartition = (ExportPartition) sourcePartition; + assertThat(exportPartition.getTableArn(), equalTo(tableArn)); + assertThat(exportPartition.getExportTime(), equalTo(exportTime)); + assertThat(exportPartition.getPartitionType(), equalTo(ExportPartition.PARTITION_TYPE)); + assertThat(exportPartition.getPartitionKey(), equalTo(partitionKey)); + assertThat(exportPartition.getProgressState().isPresent(), equalTo(true)); + assertThat(exportPartition.getProgressState().get().getExportArn(), equalTo(exportArn)); + assertThat(exportPartition.getProgressState().get().getBucket(), equalTo(bucketName)); + assertThat(exportPartition.getProgressState().get().getPrefix(), equalTo(prefix)); + + } + + + @Test + void testCreateStreamPartition() { + String sourceId = sourceIdentifier + "|" + StreamPartition.PARTITION_TYPE; + String partitionKey = streamArn + "|" + shardId; + when(sourcePartitionStoreItem.getSourceIdentifier()).thenReturn(sourceId); + when(sourcePartitionStoreItem.getSourcePartitionKey()).thenReturn(partitionKey); + + String state = "{\"startTime\":" + exportTimeMills + ",\"sequenceNumber\":\"" + sequenceNumber + "\",\"waitForExport\":false}"; + when(sourcePartitionStoreItem.getPartitionProgressState()).thenReturn(state); + + + PartitionFactory factory = new PartitionFactory(); + SourcePartition sourcePartition = factory.apply(sourcePartitionStoreItem); + assertThat(sourcePartition, notNullValue()); + StreamPartition streamPartition = (StreamPartition) sourcePartition; + assertThat(streamPartition.getStreamArn(), equalTo(streamArn)); + assertThat(streamPartition.getShardId(), equalTo(shardId)); + assertThat(streamPartition.getPartitionType(), equalTo(StreamPartition.PARTITION_TYPE)); + assertThat(streamPartition.getPartitionKey(), equalTo(partitionKey)); + + assertThat(streamPartition.getProgressState().isPresent(), equalTo(true)); + assertThat(streamPartition.getProgressState().get().getSequenceNumber(), equalTo(sequenceNumber)); + assertThat(streamPartition.getProgressState().get().getStartTime(), equalTo(exportTimeMills)); + assertThat(streamPartition.getProgressState().get().shouldWaitForExport(), equalTo(false)); + + } + + @Test + void testCreateDataFilePartition() { + String sourceId = sourceIdentifier + "|" + DataFilePartition.PARTITION_TYPE; + String partitionKey = exportArn + "|" + bucketName + "|" + s3Key; + when(sourcePartitionStoreItem.getSourceIdentifier()).thenReturn(sourceId); + when(sourcePartitionStoreItem.getSourcePartitionKey()).thenReturn(partitionKey); + + String state = "{\"totalRecords\":" + totalRecords + ",\"loadedRecords\":" + loadedRecords + "}"; + when(sourcePartitionStoreItem.getPartitionProgressState()).thenReturn(state); + + + PartitionFactory factory = new PartitionFactory(); + SourcePartition sourcePartition = factory.apply(sourcePartitionStoreItem); + assertThat(sourcePartition, notNullValue()); + DataFilePartition dataFilePartition = (DataFilePartition) sourcePartition; + assertThat(dataFilePartition.getExportArn(), equalTo(exportArn)); + assertThat(dataFilePartition.getBucket(), equalTo(bucketName)); + assertThat(dataFilePartition.getKey(), equalTo(s3Key)); + assertThat(dataFilePartition.getPartitionType(), equalTo(DataFilePartition.PARTITION_TYPE)); + assertThat(dataFilePartition.getPartitionKey(), equalTo(partitionKey)); + + assertThat(dataFilePartition.getProgressState().isPresent(), equalTo(true)); + assertThat(dataFilePartition.getProgressState().get().getLoaded(), equalTo(loadedRecords)); + assertThat(dataFilePartition.getProgressState().get().getTotal(), equalTo(totalRecords)); + + } + + @Test + void testCreateGlobalState() { + + String sourceId = sourceIdentifier + "|GLOBAL"; + String partitionKey = UUID.randomUUID().toString(); + when(sourcePartitionStoreItem.getSourceIdentifier()).thenReturn(sourceId); + when(sourcePartitionStoreItem.getSourcePartitionKey()).thenReturn(partitionKey); + + String state = "{\"totalRecords\":" + totalRecords + ",\"loadedRecords\":" + loadedRecords + "}"; + when(sourcePartitionStoreItem.getPartitionProgressState()).thenReturn(state); + + + PartitionFactory factory = new PartitionFactory(); + SourcePartition sourcePartition = factory.apply(sourcePartitionStoreItem); + assertThat(sourcePartition, notNullValue()); + GlobalState globalState = (GlobalState) sourcePartition; + assertThat(globalState.getPartitionKey(), equalTo(partitionKey)); + assertNull(globalState.getPartitionType()); + assertThat(globalState.getProgressState().isPresent(), equalTo(true)); + assertThat(globalState.getProgressState().get().size(), equalTo(2)); + assertThat(globalState.getProgressState().get().get("totalRecords"), equalTo(totalRecords)); + assertThat(globalState.getProgressState().get().get("loadedRecords"), equalTo(loadedRecords)); + + + } + + @Test + void testCreateInitPartition() { + String sourceId = sourceIdentifier + "|" + InitPartition.PARTITION_TYPE; + when(sourcePartitionStoreItem.getSourceIdentifier()).thenReturn(sourceId); + + PartitionFactory factory = new PartitionFactory(); + SourcePartition sourcePartition = factory.apply(sourcePartitionStoreItem); + assertThat(sourcePartition, notNullValue()); + InitPartition exportPartition = (InitPartition) sourcePartition; + assertThat(exportPartition.getPartitionKey(), equalTo("GLOBAL")); + assertThat(exportPartition.getPartitionType(), equalTo(InitPartition.PARTITION_TYPE)); + + + } +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoaderFactoryTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoaderFactoryTest.java new file mode 100644 index 0000000000..efcb7be076 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoaderFactoryTest.java @@ -0,0 +1,94 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.DataFileProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableMetadata; +import software.amazon.awssdk.services.s3.S3Client; + +import java.util.Optional; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.notNullValue; + +@ExtendWith(MockitoExtension.class) +class DataFileLoaderFactoryTest { + + @Mock + private EnhancedSourceCoordinator coordinator; + + @Mock + private S3Client s3Client; + @Mock + private PluginMetrics pluginMetrics; + + + private DataFilePartition dataFilePartition; + + @Mock + private Buffer> buffer; + + + private TableInfo tableInfo; + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String partitionKeyAttrName = "PK"; + private final String sortKeyAttrName = "SK"; + + private final String manifestKey = UUID.randomUUID().toString(); + private final String bucketName = UUID.randomUUID().toString(); + private final String prefix = UUID.randomUUID().toString(); + + private final String exportArn = tableArn + "/export/01693291918297-bfeccbea"; + + private final Random random = new Random(); + + private final int total = random.nextInt(10); + + + @BeforeEach + void setup() { + DataFileProgressState state = new DataFileProgressState(); + state.setLoaded(0); + state.setTotal(total); + dataFilePartition = new DataFilePartition(exportArn, bucketName, manifestKey, Optional.of(state)); + + // Mock Global Table Info + TableMetadata metadata = TableMetadata.builder() + .exportRequired(true) + .streamRequired(true) + .partitionKeyAttributeName(partitionKeyAttrName) + .sortKeyAttributeName(sortKeyAttrName) + .build(); + + tableInfo = new TableInfo(tableArn, metadata); + } + + @Test + void test_createDataFileLoader() { + + DataFileLoaderFactory loaderFactory = new DataFileLoaderFactory(coordinator, s3Client, pluginMetrics, buffer); + Runnable loader = loaderFactory.createDataFileLoader(dataFilePartition, tableInfo); + assertThat(loader, notNullValue()); + + } +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoaderTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoaderTest.java new file mode 100644 index 0000000000..5118b700c7 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileLoaderTest.java @@ -0,0 +1,146 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.dynamodb.converter.ExportRecordConverter; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.DataFileProgressState; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.http.AbortableInputStream; +import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.StringJoiner; +import java.util.UUID; +import java.util.zip.GZIPOutputStream; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class DataFileLoaderTest { + + @Mock + private EnhancedSourceCoordinator coordinator; + + @Mock + private S3Client s3Client; + + + private S3ObjectReader s3ObjectReader; + + @Mock + private ExportRecordConverter recordConverter; + + private DataFileCheckpointer checkpointer; + + private DataFilePartition dataFilePartition; + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String manifestKey = UUID.randomUUID().toString(); + private final String bucketName = UUID.randomUUID().toString(); + private final String prefix = UUID.randomUUID().toString(); + + private final String exportArn = tableArn + "/export/01693291918297-bfeccbea"; + + private final Random random = new Random(); + + private final int total = random.nextInt(10); + + @BeforeEach + void setup() throws IOException { + + DataFileProgressState state = new DataFileProgressState(); + state.setLoaded(0); + state.setTotal(total); + + dataFilePartition = new DataFilePartition(exportArn, bucketName, manifestKey, Optional.of(state)); + + when(s3Client.getObject(any(GetObjectRequest.class))).thenReturn(generateGzipInputStream(total)); + s3ObjectReader = new S3ObjectReader(s3Client); + + lenient().when(coordinator.createPartition(any(SourcePartition.class))).thenReturn(true); + lenient().doNothing().when(coordinator).completePartition(any(SourcePartition.class)); + lenient().doNothing().when(coordinator).saveProgressStateForPartition(any(SourcePartition.class)); + lenient().doNothing().when(coordinator).giveUpPartition(any(SourcePartition.class)); + + lenient().doNothing().when(recordConverter).writeToBuffer(any(List.class)); + + checkpointer = new DataFileCheckpointer(coordinator, dataFilePartition); + + } + + private ResponseInputStream generateGzipInputStream(int numberOfRecords) throws IOException { + + StringJoiner stringJoiner = new StringJoiner("\\n"); + for (int i = 0; i < numberOfRecords; i++) { + stringJoiner.add(UUID.randomUUID().toString()); + } + final String data = stringJoiner.toString(); + + final byte[] dataBytes = data.getBytes(StandardCharsets.UTF_8); + + final ByteArrayOutputStream byteOut = new ByteArrayOutputStream(); + final GZIPOutputStream gzipOut = new GZIPOutputStream(byteOut); + gzipOut.write(dataBytes, 0, dataBytes.length); + gzipOut.close(); + final byte[] bites = byteOut.toByteArray(); + final ByteArrayInputStream byteInStream = new ByteArrayInputStream(bites); + + final ResponseInputStream fileInputStream = new ResponseInputStream<>( + GetObjectResponse.builder().contentLength((long) data.length()).build(), + AbortableInputStream.create(byteInStream) + ); + return fileInputStream; + + } + + @Test + void test_run_loadFile_correctly() throws InterruptedException { + + DataFileLoader loader = DataFileLoader.builder() + .bucketName(bucketName) + .key(manifestKey) + .s3ObjectReader(s3ObjectReader) + .recordConverter(recordConverter) + .checkpointer(checkpointer) + .build(); + + loader.run(); + // Run for a while + Thread.sleep(500); + + // Should call s3 getObject + verify(s3Client).getObject(any(GetObjectRequest.class)); + + // Should write to buffer + verify(recordConverter).writeToBuffer(any(List.class)); + + // Should do one last checkpoint when done. + verify(coordinator).saveProgressStateForPartition(any(DataFilePartition.class)); + + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileSchedulerTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileSchedulerTest.java new file mode 100644 index 0000000000..f3ae4b8098 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/DataFileSchedulerTest.java @@ -0,0 +1,144 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.DataFilePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.DataFileProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.LoadStatus; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableInfo; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableMetadata; + +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.opensearch.dataprepper.plugins.source.dynamodb.export.DataFileScheduler.EXPORT_FILE_SUCCESS_COUNT; + +@ExtendWith(MockitoExtension.class) +class DataFileSchedulerTest { + + @Mock + private EnhancedSourceCoordinator coordinator; + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private GlobalState tableInfoGlobalState; + + @Mock + private GlobalState exportInfoGlobalState; + + @Mock + private Counter exportFileSuccess; + + @Mock + private DataFileLoaderFactory loaderFactory; + + private DataFileScheduler scheduler; + + private DataFilePartition dataFilePartition; + + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + private final String manifestKey = UUID.randomUUID().toString(); + private final String bucketName = UUID.randomUUID().toString(); + private final String prefix = UUID.randomUUID().toString(); + + private final String exportArn = tableArn + "/export/01693291918297-bfeccbea"; + private final String streamArn = tableArn + "/stream/2023-09-14T05:46:45.367"; + + + @BeforeEach + void setup() { + + + DataFileProgressState state = new DataFileProgressState(); + state.setLoaded(0); + state.setTotal(100); +// lenient().when(dataFilePartition.getProgressState()).thenReturn(Optional.of(state)); + + dataFilePartition = new DataFilePartition(exportArn, bucketName, manifestKey, Optional.of(state)); + + // Mock Global Table Info + lenient().when(coordinator.getPartition(tableArn)).thenReturn(Optional.of(tableInfoGlobalState)); + TableMetadata metadata = TableMetadata.builder() + .exportRequired(true) + .streamRequired(true) + .partitionKeyAttributeName("PK") + .sortKeyAttributeName("SK") + .streamArn(streamArn) + .build(); +// Map tableState = metadata; + lenient().when(tableInfoGlobalState.getProgressState()).thenReturn(Optional.of(metadata.toMap())); + + + // Mock Global Export Info + LoadStatus loadStatus = new LoadStatus(1, 0, 100, 0); + lenient().when(coordinator.getPartition(exportArn)).thenReturn(Optional.of(exportInfoGlobalState)); + lenient().when(exportInfoGlobalState.getProgressState()).thenReturn(Optional.of(loadStatus.toMap())); + + given(pluginMetrics.counter(EXPORT_FILE_SUCCESS_COUNT)).willReturn(exportFileSuccess); + + lenient().when(coordinator.createPartition(any(SourcePartition.class))).thenReturn(true); + lenient().doNothing().when(coordinator).completePartition(any(SourcePartition.class)); + lenient().doNothing().when(coordinator).giveUpPartition(any(SourcePartition.class)); + + lenient().when(loaderFactory.createDataFileLoader(any(DataFilePartition.class), any(TableInfo.class))).thenReturn(() -> System.out.println("Hello")); + + } + + @Test + public void test_run_DataFileLoader_correctly() throws InterruptedException { + given(coordinator.acquireAvailablePartition(DataFilePartition.PARTITION_TYPE)).willReturn(Optional.of(dataFilePartition)).willReturn(Optional.empty()); + + scheduler = new DataFileScheduler(coordinator, loaderFactory, pluginMetrics); + + ExecutorService executor = Executors.newSingleThreadExecutor(); + executor.submit(scheduler); + + // Run for a while + Thread.sleep(500); + + // Should acquire data file partition + verify(coordinator).acquireAvailablePartition(DataFilePartition.PARTITION_TYPE); + // Should create a loader + verify(loaderFactory).createDataFileLoader(any(DataFilePartition.class), any(TableInfo.class)); + // Need to call getPartition for 3 times (3 global states, 2 TableInfo) + verify(coordinator, times(3)).getPartition(anyString()); + // Should update global state with load status + verify(coordinator).saveProgressStateForPartition(any(GlobalState.class)); + // Should create a partition to inform streaming can start. + verify(coordinator).createPartition(any(GlobalState.class)); + // Should mask the partition as completed. + verify(coordinator).completePartition(any(DataFilePartition.class)); + // Should update metrics. + verify(exportFileSuccess).increment(); + + executor.shutdownNow(); + + + } + + +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ExportSchedulerTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ExportSchedulerTest.java new file mode 100644 index 0000000000..6f78f17809 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ExportSchedulerTest.java @@ -0,0 +1,171 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import io.micrometer.core.instrument.Counter; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.ExportPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.ExportProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.ExportSummary; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.dynamodb.model.DescribeExportRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeExportResponse; +import software.amazon.awssdk.services.dynamodb.model.ExportDescription; +import software.amazon.awssdk.services.dynamodb.model.ExportFormat; +import software.amazon.awssdk.services.dynamodb.model.ExportStatus; +import software.amazon.awssdk.services.dynamodb.model.ExportTableToPointInTimeRequest; +import software.amazon.awssdk.services.dynamodb.model.ExportTableToPointInTimeResponse; + +import java.time.Instant; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.dynamodb.export.ExportScheduler.EXPORT_FILES_TOTAL_COUNT; +import static org.opensearch.dataprepper.plugins.source.dynamodb.export.ExportScheduler.EXPORT_JOB_ERROR_COUNT; +import static org.opensearch.dataprepper.plugins.source.dynamodb.export.ExportScheduler.EXPORT_JOB_SUCCESS_COUNT; +import static org.opensearch.dataprepper.plugins.source.dynamodb.export.ExportScheduler.EXPORT_RECORDS_TOTAL_COUNT; + + +@ExtendWith(MockitoExtension.class) +class ExportSchedulerTest { + + + @Mock + private EnhancedSourceCoordinator coordinator; + @Mock + private DynamoDbClient dynamoDBClient; + + @Mock + private ManifestFileReader manifestFileReader; + + @Mock + private PluginMetrics pluginMetrics; + + private ExportScheduler scheduler; + + @Mock + private ExportPartition exportPartition; + + + @Mock + private Counter exportJobSuccess; + + @Mock + private Counter exportJobErrors; + + @Mock + private Counter exportFilesTotal; + + @Mock + private Counter exportRecordsTotal; + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String manifestKey = UUID.randomUUID().toString(); + private final String bucketName = UUID.randomUUID().toString(); + private final String prefix = UUID.randomUUID().toString(); + + private final String exportArn = tableArn + "/export/01693291918297-bfeccbea"; + + private final long exportTimeMills = 1695021857760L; + private final Instant exportTime = Instant.ofEpochMilli(exportTimeMills); + + + @BeforeEach + void setup() { + + when(exportPartition.getTableArn()).thenReturn(tableArn); + when(exportPartition.getExportTime()).thenReturn(exportTime); + + ExportProgressState state = new ExportProgressState(); + state.setBucket(bucketName); + state.setPrefix(prefix); + when(exportPartition.getProgressState()).thenReturn(Optional.of(state)); + + given(pluginMetrics.counter(EXPORT_JOB_SUCCESS_COUNT)).willReturn(exportJobSuccess); + given(pluginMetrics.counter(EXPORT_JOB_ERROR_COUNT)).willReturn(exportJobErrors); + given(pluginMetrics.counter(EXPORT_FILES_TOTAL_COUNT)).willReturn(exportFilesTotal); + given(pluginMetrics.counter(EXPORT_RECORDS_TOTAL_COUNT)).willReturn(exportRecordsTotal); + + ExportSummary summary = mock(ExportSummary.class); + lenient().when(manifestFileReader.parseSummaryFile(anyString(), anyString())).thenReturn(summary); + lenient().when(summary.getS3Bucket()).thenReturn(bucketName); + lenient().when(summary.getManifestFilesS3Key()).thenReturn(manifestKey); + lenient().when(manifestFileReader.parseDataFile(anyString(), anyString())).thenReturn(Map.of("Key1", 100, "Key2", 200)); + + lenient().when(coordinator.createPartition(any(SourcePartition.class))).thenReturn(true); + lenient().doNothing().when(coordinator).completePartition(any(SourcePartition.class)); + lenient().doNothing().when(coordinator).giveUpPartition(any(SourcePartition.class)); + + } + + + @Test + public void test_run_exportJob_correctly() throws InterruptedException { + given(coordinator.acquireAvailablePartition(ExportPartition.PARTITION_TYPE)).willReturn(Optional.of(exportPartition)).willReturn(Optional.empty()); + + // Set up mock behavior + ExportDescription desc = ExportDescription.builder() + .exportArn(exportArn) + .exportStatus(ExportStatus.COMPLETED) + .exportFormat(ExportFormat.ION) + .exportManifest(manifestKey) + .build(); + + ExportTableToPointInTimeResponse exportResponse = ExportTableToPointInTimeResponse.builder() + .exportDescription(desc) + .build(); + + final ArgumentCaptor exportRequestArgumentCaptor = ArgumentCaptor.forClass(ExportTableToPointInTimeRequest.class); + when(dynamoDBClient.exportTableToPointInTime(exportRequestArgumentCaptor.capture())).thenReturn(exportResponse); + DescribeExportResponse describeExportResponse = DescribeExportResponse.builder().exportDescription(desc).build(); + when(dynamoDBClient.describeExport(any(DescribeExportRequest.class))).thenReturn(describeExportResponse); + + scheduler = new ExportScheduler(coordinator, dynamoDBClient, manifestFileReader, pluginMetrics); + + ExecutorService executor = Executors.newSingleThreadExecutor(); + executor.submit(scheduler); + + Thread.sleep(500); + + verify(dynamoDBClient).exportTableToPointInTime(any(ExportTableToPointInTimeRequest.class)); + verify(dynamoDBClient, times(2)).describeExport(any(DescribeExportRequest.class)); + + // Create 2 data file partitions + 1 global state + verify(coordinator, times(3)).createPartition(any(SourcePartition.class)); + // Complete the export partition + verify(coordinator).completePartition(any(SourcePartition.class)); + verify(exportJobSuccess).increment(); + verify(exportFilesTotal).increment(2); + verify(exportRecordsTotal).increment(300); + verifyNoInteractions(exportJobErrors); + + executor.shutdownNow(); + + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ManifestFileReaderTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ManifestFileReaderTest.java new file mode 100644 index 0000000000..bfd3f2369d --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/export/ManifestFileReaderTest.java @@ -0,0 +1,110 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.export; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.ExportSummary; + +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.util.Map; +import java.util.Random; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ManifestFileReaderTest { + + + @Mock + private S3ObjectReader s3ObjectReader; + + + @Test + void parseSummaryFile() { + + final Random random = new Random(); + + final String version = "2020-06-30"; + final String exportArn = UUID.randomUUID().toString(); + final String startTime = "2023-08-14T08:48:17.020Z"; + final String endTime = "2023-08-14T08:53:26.545Z"; + final String tableArn = UUID.randomUUID().toString(); + final String tableId = UUID.randomUUID().toString(); + final String exportTime = "2023-08-14T08:48:16.212Z"; + final String s3Bucket = UUID.randomUUID().toString(); + final String s3Prefix = UUID.randomUUID().toString(); + final String s3SseAlgorithm = "AES256"; + final String manifestFilesS3Key = UUID.randomUUID().toString(); + final String outputFormat = "DYNAMODB_JSON"; + long billedSizeBytes = random.nextLong(); + int itemCount = random.nextInt(10000); + + + String summaryData = String.format("{\"version\":\"%s\",\"exportArn\": \"%s\",\"startTime\":\"%s\",\"endTime\":\"%s\",\"tableArn\":\"%s\",\"tableId\":\"%s\",\"exportTime\":\"%s\",\"s3Bucket\":\"%s\",\"s3Prefix\":\"%s\",\"s3SseAlgorithm\":\"%s\",\"s3SseKmsKeyId\":null,\"manifestFilesS3Key\":\"%s\",\"billedSizeBytes\":%d,\"itemCount\":%d,\"outputFormat\":\"%s\"}", + version, exportArn, startTime, endTime, tableArn, tableId, exportTime, s3Bucket, s3Prefix, s3SseAlgorithm, manifestFilesS3Key, billedSizeBytes, itemCount, outputFormat); + InputStream fileObjectStream = new ByteArrayInputStream(summaryData.getBytes()); + + when(s3ObjectReader.readFile(anyString(), anyString())).thenReturn(fileObjectStream); + + ManifestFileReader reader = new ManifestFileReader(s3ObjectReader); + ExportSummary exportSummary = reader.parseSummaryFile("test", "test"); + + assertThat(exportSummary, notNullValue()); + assertThat(exportSummary.getVersion(), equalTo(version)); + assertThat(exportSummary.getExportArn(), equalTo(exportArn)); + assertThat(exportSummary.getStartTime(), equalTo(startTime)); + assertThat(exportSummary.getEndTime(), equalTo(endTime)); + assertThat(exportSummary.getTableArn(), equalTo(tableArn)); + assertThat(exportSummary.getTableId(), equalTo(tableId)); + assertThat(exportSummary.getExportTime(), equalTo(exportTime)); + assertThat(exportSummary.getS3Bucket(), equalTo(s3Bucket)); + assertThat(exportSummary.getS3Prefix(), equalTo(s3Prefix)); + assertThat(exportSummary.getS3SseAlgorithm(), equalTo(s3SseAlgorithm)); + assertThat(exportSummary.getManifestFilesS3Key(), equalTo(manifestFilesS3Key)); + assertNull(exportSummary.getS3SseKmsKeyId()); + assertThat(exportSummary.getBilledSizeBytes(), equalTo(billedSizeBytes)); + assertThat(exportSummary.getItemCount(), equalTo(itemCount)); + assertThat(exportSummary.getOutputFormat(), equalTo(outputFormat)); + + + } + + @Test + void parseDataFile() { + + final String dataFileS3Key1 = UUID.randomUUID().toString(); + final String dataFileS3Key2 = UUID.randomUUID().toString(); + + final Random random = new Random(); + final int itemCount1 = random.nextInt(10000); + final int itemCount2 = random.nextInt(10000); + + String summaryData = "{\"itemCount\":" + itemCount1 + ",\"md5Checksum\":\"a0k21IY3eelgr2PuWJLjJw==\",\"etag\":\"51f9f394903c5d682321c6211aae8b6a-1\",\"dataFileS3Key\":\"" + dataFileS3Key1 + "\"}\n" + + "{\"itemCount\":" + itemCount2 + ",\"md5Checksum\":\"j76iIYpnqVKrD/zt6HKV1Q==\",\"etag\":\"551fa137e144877aacf87f75340483bf-1\",\"dataFileS3Key\":\"" + dataFileS3Key2 + "\"}"; + InputStream fileObjectStream = new ByteArrayInputStream(summaryData.getBytes()); + when(s3ObjectReader.readFile(anyString(), anyString())).thenReturn(fileObjectStream); + + ManifestFileReader reader = new ManifestFileReader(s3ObjectReader); + Map dataFiles = reader.parseDataFile("test", "test"); + System.out.println(dataFiles); + + assertThat(dataFiles, notNullValue()); + assertThat(dataFiles.size(), equalTo(2)); + assertThat(dataFiles.get(dataFileS3Key1), equalTo(itemCount1)); + assertThat(dataFiles.get(dataFileS3Key2), equalTo(itemCount2)); + + } +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumerFactoryTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumerFactoryTest.java new file mode 100644 index 0000000000..52a0716180 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumerFactoryTest.java @@ -0,0 +1,97 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.stream; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableMetadata; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; + +import java.util.Optional; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.lenient; + +@ExtendWith(MockitoExtension.class) +class ShardConsumerFactoryTest { + + @Mock + private EnhancedSourceCoordinator coordinator; + @Mock + private DynamoDbStreamsClient dynamoDbStreamsClient; + @Mock + private ShardManager shardManager; + @Mock + private PluginMetrics pluginMetrics; + + + private StreamPartition streamPartition; + + + @Mock + private Buffer> buffer; + + @Mock + private GlobalState tableInfoGlobalState; + + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String exportArn = tableArn + "/export/01693291918297-bfeccbea"; + private final String streamArn = tableArn + "/stream/2023-09-14T05:46:45.367"; + + private final String shardId = "shardId-" + UUID.randomUUID(); + private final String shardIterator = UUID.randomUUID().toString(); + + + @BeforeEach + void setup() { + + StreamProgressState state = new StreamProgressState(); + state.setWaitForExport(false); + state.setStartTime(0); + streamPartition = new StreamPartition(streamArn, shardId, Optional.of(state)); + + lenient().when(shardManager.getShardIterator(eq(streamArn), eq(shardId), eq(null))).thenReturn(shardIterator); + + // Mock Global Table Info + lenient().when(coordinator.getPartition(tableArn)).thenReturn(Optional.of(tableInfoGlobalState)); + TableMetadata metadata = TableMetadata.builder() + .exportRequired(true) + .streamRequired(true) + .partitionKeyAttributeName("PK") + .sortKeyAttributeName("SK") + .build(); + lenient().when(tableInfoGlobalState.getProgressState()).thenReturn(Optional.of(metadata.toMap())); + + } + + @Test + public void test_create_shardConsumer_correctly() { + + ShardConsumerFactory consumerFactory = new ShardConsumerFactory(coordinator, dynamoDbStreamsClient, pluginMetrics, shardManager, buffer); + + Runnable consumer = consumerFactory.createConsumer(streamPartition); + + assertThat(consumer, notNullValue()); + } + +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumerTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumerTest.java new file mode 100644 index 0000000000..8fb1120aa0 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardConsumerTest.java @@ -0,0 +1,154 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.stream; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.dynamodb.converter.StreamRecordConverter; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.dynamodb.model.TableMetadata; +import software.amazon.awssdk.services.dynamodb.model.AttributeValue; +import software.amazon.awssdk.services.dynamodb.model.GetRecordsRequest; +import software.amazon.awssdk.services.dynamodb.model.GetRecordsResponse; +import software.amazon.awssdk.services.dynamodb.model.OperationType; +import software.amazon.awssdk.services.dynamodb.model.Record; +import software.amazon.awssdk.services.dynamodb.model.StreamRecord; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; + +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.UUID; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +class ShardConsumerTest { + + @Mock + private EnhancedSourceCoordinator coordinator; + @Mock + private DynamoDbStreamsClient dynamoDbStreamsClient; + + @Mock + private StreamRecordConverter recordConverter; + + @Mock + private GlobalState tableInfoGlobalState; + + private StreamCheckpointer checkpointer; + + private StreamPartition streamPartition; + + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String partitionKeyAttrName = "PK"; + private final String sortKeyAttrName = "SK"; + + + private final String exportArn = tableArn + "/export/01693291918297-bfeccbea"; + private final String streamArn = tableArn + "/stream/2023-09-14T05:46:45.367"; + + private final String shardId = "shardId-" + UUID.randomUUID(); + private final String shardIterator = UUID.randomUUID().toString(); + + + @BeforeEach + void setup() { + + StreamProgressState state = new StreamProgressState(); + state.setWaitForExport(false); + state.setStartTime(0); + streamPartition = new StreamPartition(streamArn, shardId, Optional.of(state)); + + + // Mock Global Table Info + lenient().when(coordinator.getPartition(tableArn)).thenReturn(Optional.of(tableInfoGlobalState)); + TableMetadata metadata = TableMetadata.builder() + .exportRequired(true) + .streamRequired(true) + .partitionKeyAttributeName(partitionKeyAttrName) + .sortKeyAttributeName(sortKeyAttrName) + .build(); + lenient().when(tableInfoGlobalState.getProgressState()).thenReturn(Optional.of(metadata.toMap())); + + lenient().when(coordinator.createPartition(any(SourcePartition.class))).thenReturn(true); + lenient().doNothing().when(coordinator).completePartition(any(SourcePartition.class)); + lenient().doNothing().when(coordinator).saveProgressStateForPartition(any(SourcePartition.class)); + lenient().doNothing().when(coordinator).giveUpPartition(any(SourcePartition.class)); + + checkpointer = new StreamCheckpointer(coordinator, streamPartition); + + List records = buildRecords(10); + GetRecordsResponse response = GetRecordsResponse.builder() + .records(records) + .nextShardIterator(null) + .build(); + when(dynamoDbStreamsClient.getRecords(any(GetRecordsRequest.class))).thenReturn(response); + } + + + @Test + void test_run_shardConsumer_correctly() { + + ShardConsumer shardConsumer = ShardConsumer.builder(dynamoDbStreamsClient) + .shardIterator(shardIterator) + .checkpointer(checkpointer) + .recordConverter(recordConverter) + .startTime(null) + .waitForExport(false) + .build(); + shardConsumer.run(); + + // Should call GetRecords + verify(dynamoDbStreamsClient).getRecords(any(GetRecordsRequest.class)); + + // Should write to buffer + verify(recordConverter).writeToBuffer(any(List.class)); + + // Should complete the consumer as reach to end of shard + verify(coordinator).saveProgressStateForPartition(any(StreamPartition.class)); + } + + /** + * Helper function to generate some data. + */ + private List buildRecords(int count) { + List records = new ArrayList<>(); + for (int i = 0; i < count; i++) { + Map data = Map.of( + partitionKeyAttrName, AttributeValue.builder().s(UUID.randomUUID().toString()).build(), + sortKeyAttrName, AttributeValue.builder().s(UUID.randomUUID().toString()).build()); + + StreamRecord streamRecord = StreamRecord.builder() + .newImage(data) + .sequenceNumber(UUID.randomUUID().toString()) + .approximateCreationDateTime(Instant.now()) + .build(); + Record record = Record.builder() + .dynamodb(streamRecord) + .eventName(OperationType.INSERT) + .build(); + records.add(record); + } + + return records; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardManagerTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardManagerTest.java new file mode 100644 index 0000000000..7c1fa7b53b --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/ShardManagerTest.java @@ -0,0 +1,150 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.stream; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import software.amazon.awssdk.services.dynamodb.model.DescribeStreamRequest; +import software.amazon.awssdk.services.dynamodb.model.DescribeStreamResponse; +import software.amazon.awssdk.services.dynamodb.model.GetShardIteratorRequest; +import software.amazon.awssdk.services.dynamodb.model.GetShardIteratorResponse; +import software.amazon.awssdk.services.dynamodb.model.SequenceNumberRange; +import software.amazon.awssdk.services.dynamodb.model.Shard; +import software.amazon.awssdk.services.dynamodb.model.ShardIteratorType; +import software.amazon.awssdk.services.dynamodb.model.StreamDescription; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; + +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.when; + + +@ExtendWith(MockitoExtension.class) +class ShardManagerTest { + + + @Mock + private DynamoDbStreamsClient dynamoDbStreamsClient; + + @Mock + private ShardManager shardManager; + + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String streamArn = tableArn + "/stream/2023-09-14T05:46:45.367"; + + private final String shardId = "shardId-" + UUID.randomUUID(); + private final String shardIterator = UUID.randomUUID().toString(); + private final String sequenceNumber = UUID.randomUUID().toString(); + + + private Shard buildShard(String shardId, String parentShardId, boolean isOpen) { + String endingSequenceNumber = isOpen ? null : UUID.randomUUID().toString(); + return Shard.builder() + .shardId(shardId) + .parentShardId(parentShardId) + .sequenceNumberRange(SequenceNumberRange.builder() + .endingSequenceNumber(endingSequenceNumber) + .startingSequenceNumber(UUID.randomUUID().toString()) + .build()) + .build(); + + } + + + @BeforeEach + void setup() { + // Initialize some shards + List shardList = new ArrayList<>(); + shardList.add(buildShard("Shard-006", "Shard-004", true)); + shardList.add(buildShard("Shard-005", "Shard-003", true)); + shardList.add(buildShard("Shard-004", "Shard-002", false)); + shardList.add(buildShard("Shard-003", "Shard-001", false)); + + StreamDescription description = StreamDescription.builder() + .shards(shardList) + .lastEvaluatedShardId(null) + .build(); + DescribeStreamResponse response = DescribeStreamResponse.builder() + .streamDescription(description) + .build(); + + lenient().when(dynamoDbStreamsClient.describeStream(any(DescribeStreamRequest.class))).thenReturn(response); + shardManager = new ShardManager(dynamoDbStreamsClient); + + } + + @Test + void test_getChildShardIds_should_return_child_shards() { + List childShards = shardManager.getChildShardIds(streamArn, "Shard-003"); + assertThat(childShards, notNullValue()); + assertThat(childShards.size(), equalTo(1)); + assertThat(childShards.get(0), equalTo("Shard-005")); + + } + + @Test + void test_getActiveShards_should_return_open_shards() { + List activeShards = shardManager.getActiveShards(streamArn); + assertThat(activeShards, notNullValue()); + assertThat(activeShards.size(), equalTo(2)); + assertThat(activeShards.contains("Shard-006"), equalTo(true)); + assertThat(activeShards.contains("Shard-005"), equalTo(true)); + } + + @Test + void test_getShardIterator_with_sequenceNumber_should_return_shardIterator() { + final ArgumentCaptor getShardIterRequestArgumentCaptor = ArgumentCaptor.forClass(GetShardIteratorRequest.class); + GetShardIteratorResponse response = GetShardIteratorResponse.builder() + .shardIterator(shardIterator) + .build(); + + when(dynamoDbStreamsClient.getShardIterator(getShardIterRequestArgumentCaptor.capture())).thenReturn(response); + String shardIterator1 = shardManager.getShardIterator(streamArn, shardId, sequenceNumber); + assertThat(getShardIterRequestArgumentCaptor.getValue().shardId(), equalTo(shardId)); + assertThat(getShardIterRequestArgumentCaptor.getValue().shardIteratorType(), equalTo(ShardIteratorType.AFTER_SEQUENCE_NUMBER)); + assertThat(getShardIterRequestArgumentCaptor.getValue().sequenceNumber(), equalTo(sequenceNumber)); + assertThat(shardIterator1, equalTo(shardIterator)); + + + } + + @Test + void test_getShardIterator_without_sequenceNumber_should_return_shardIterator() { + final ArgumentCaptor getShardIterRequestArgumentCaptor = ArgumentCaptor.forClass(GetShardIteratorRequest.class); + GetShardIteratorResponse response = GetShardIteratorResponse.builder() + .shardIterator(shardIterator) + .build(); + + when(dynamoDbStreamsClient.getShardIterator(getShardIterRequestArgumentCaptor.capture())).thenReturn(response); + String shardIterator1 = shardManager.getShardIterator(streamArn, shardId, null); + assertThat(getShardIterRequestArgumentCaptor.getValue().shardId(), equalTo(shardId)); + assertThat(getShardIterRequestArgumentCaptor.getValue().shardIteratorType(), equalTo(ShardIteratorType.TRIM_HORIZON)); + assertThat(shardIterator1, equalTo(shardIterator)); + } + + @Test + void test_getRootShardIds_should_return_root_shards() { + List activeShards = shardManager.getRootShardIds(streamArn); + assertThat(activeShards, notNullValue()); + assertThat(activeShards.size(), equalTo(2)); + assertThat(activeShards.contains("Shard-003"), equalTo(true)); + assertThat(activeShards.contains("Shard-004"), equalTo(true)); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/StreamSchedulerTest.java b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/StreamSchedulerTest.java new file mode 100644 index 0000000000..30ced8d0f7 --- /dev/null +++ b/data-prepper-plugins/dynamodb-source/src/test/java/org/opensearch/dataprepper/plugins/source/dynamodb/stream/StreamSchedulerTest.java @@ -0,0 +1,110 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ + +package org.opensearch.dataprepper.plugins.source.dynamodb.stream; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.SourcePartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.dynamodb.coordination.state.StreamProgressState; +import software.amazon.awssdk.services.dynamodb.streams.DynamoDbStreamsClient; + +import java.time.Instant; +import java.util.List; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.verify; + +@ExtendWith(MockitoExtension.class) +class StreamSchedulerTest { + + @Mock + private EnhancedSourceCoordinator coordinator; + + @Mock + private DynamoDbStreamsClient dynamoDbStreamsClient; + + + @Mock + private ShardManager shardManager; + + private StreamScheduler scheduler; + + + private StreamPartition streamPartition; + + + @Mock + private ShardConsumerFactory consumerFactory; + + + private final String tableName = UUID.randomUUID().toString(); + private final String tableArn = "arn:aws:dynamodb:us-west-2:123456789012:table/" + tableName; + + private final String exportArn = tableArn + "/export/01693291918297-bfeccbea"; + private final String streamArn = tableArn + "/stream/2023-09-14T05:46:45.367"; + + private final String shardId = "shardId-" + UUID.randomUUID(); + + private final long exportTimeMills = 1695021857760L; + private final Instant exportTime = Instant.ofEpochMilli(exportTimeMills); + + + @BeforeEach + void setup() { + + StreamProgressState state = new StreamProgressState(); + state.setWaitForExport(false); + state.setStartTime(0); + + streamPartition = new StreamPartition(streamArn, shardId, Optional.of(state)); + // Mock Coordinator methods + lenient().when(coordinator.createPartition(any(SourcePartition.class))).thenReturn(true); + lenient().doNothing().when(coordinator).completePartition(any(SourcePartition.class)); + lenient().doNothing().when(coordinator).saveProgressStateForPartition(any(SourcePartition.class)); + lenient().doNothing().when(coordinator).giveUpPartition(any(SourcePartition.class)); + + lenient().when(consumerFactory.createConsumer(any(StreamPartition.class))).thenReturn(() -> System.out.println("Hello")); + lenient().when(shardManager.getChildShardIds(anyString(), anyString())).thenReturn(List.of(shardId)); + + } + + + @Test + public void test_normal_run() throws InterruptedException { + given(coordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE)).willReturn(Optional.of(streamPartition)).willReturn(Optional.empty()); + + scheduler = new StreamScheduler(coordinator, consumerFactory, shardManager); + + ExecutorService executor = Executors.newSingleThreadExecutor(); + executor.submit(scheduler); + + // Need to run a while + Thread.sleep(500); + // Should acquire the stream partition + verify(coordinator).acquireAvailablePartition(StreamPartition.PARTITION_TYPE); + // Should start a new consumer + verify(consumerFactory).createConsumer(any(StreamPartition.class)); + // Should create stream partition for child shards. + verify(coordinator).createPartition(any(StreamPartition.class)); + // Should mask the stream partition as completed. + verify(coordinator).completePartition(any(StreamPartition.class)); + + executor.shutdownNow(); + + } +} \ No newline at end of file diff --git a/settings.gradle b/settings.gradle index 4e15bc1257..7cffdd4eeb 100644 --- a/settings.gradle +++ b/settings.gradle @@ -142,4 +142,4 @@ include 'data-prepper-plugins:http-sink' include 'data-prepper-plugins:sns-sink' include 'data-prepper-plugins:prometheus-sink' include 'data-prepper-plugins:dissect-processor' - +include 'data-prepper-plugins:dynamodb-source'