diff --git a/data-prepper-plugins/neptune-source/build.gradle b/data-prepper-plugins/neptune-source/build.gradle new file mode 100644 index 0000000000..6961ecf7de --- /dev/null +++ b/data-prepper-plugins/neptune-source/build.gradle @@ -0,0 +1,32 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +dependencies { + implementation project(path: ':data-prepper-api') + + implementation libs.commons.lang3 + implementation 'io.micrometer:micrometer-core' + implementation 'org.apache.httpcomponents:httpclient:4.5.x' + implementation 'com.amazonaws:amazon-neptune-sigv4-signer:3.0.1' + implementation 'org.eclipse.rdf4j:rdf4j-rio-nquads:5.0.2' + implementation 'software.amazon.awssdk:neptunedata' + implementation 'software.amazon.awssdk:sts' + implementation 'com.fasterxml.jackson.datatype:jackson-datatype-jsr310' + compileOnly 'org.projectlombok:lombok:1.18.22' + annotationProcessor 'org.projectlombok:lombok:1.18.22' + + implementation 'com.fasterxml.jackson.core:jackson-core' + implementation 'com.fasterxml.jackson.core:jackson-databind' + implementation 'software.amazon.awssdk:s3' + + implementation project(path: ':data-prepper-plugins:aws-plugin-api') + implementation project(path: ':data-prepper-plugins:buffer-common') + implementation project(path: ':data-prepper-plugins:http-common') + implementation project(path: ':data-prepper-plugins:common') + + testImplementation testLibs.bundles.junit + testImplementation testLibs.slf4j.simple + testImplementation project(path: ':data-prepper-test-common') + +} \ No newline at end of file diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/NeptuneService.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/NeptuneService.java new file mode 100644 index 0000000000..79802859fe --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/NeptuneService.java @@ -0,0 +1,105 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune; + +import com.amazonaws.neptune.auth.NeptuneSigV4SignerException; +import org.opensearch.dataprepper.common.concurrent.BackgroundThreadFactory; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.leader.LeaderScheduler; +import org.opensearch.dataprepper.plugins.source.neptune.s3partition.S3PartitionCreatorScheduler; +import org.opensearch.dataprepper.plugins.source.neptune.stream.StreamScheduler; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Instant; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * This service class + * - reading stream data from Neptune Stream Http Endpoints + * - export (initial load) - to be implemented later + */ +public class NeptuneService { + private static final Logger LOG = LoggerFactory.getLogger(NeptuneService.class); + + private static final String S3_PATH_DELIMITER = "/"; + + private final EnhancedSourceCoordinator sourceCoordinator; + private final PluginMetrics pluginMetrics; + private final NeptuneSourceConfig sourceConfig; + private final AcknowledgementSetManager acknowledgementSetManager; + + private ExecutorService executor; + + public NeptuneService(final EnhancedSourceCoordinator sourceCoordinator, + final NeptuneSourceConfig sourceConfig, + final PluginMetrics pluginMetrics, + final AcknowledgementSetManager acknowledgementSetManager) throws NeptuneSigV4SignerException { + this.sourceCoordinator = sourceCoordinator; + this.pluginMetrics = pluginMetrics; + this.acknowledgementSetManager = acknowledgementSetManager; + this.sourceConfig = sourceConfig; + } + + /** + * This service start 3 long-running threads (scheduler) + * - Leader Scheduler + * - Stream 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 Neptune service"); + + final String s3PathPrefix = getS3PathPrefix(); + final LeaderScheduler leaderScheduler = new LeaderScheduler(sourceCoordinator, sourceConfig, s3PathPrefix); + final S3PartitionCreatorScheduler s3PartitionCreatorScheduler = new S3PartitionCreatorScheduler(sourceCoordinator); + final StreamScheduler streamScheduler = new StreamScheduler(sourceCoordinator, buffer, acknowledgementSetManager, sourceConfig, s3PathPrefix, pluginMetrics); + + executor = Executors.newFixedThreadPool(3, BackgroundThreadFactory.defaultExecutorThreadFactory("neptune-source")); + executor.submit(leaderScheduler); + executor.submit(s3PartitionCreatorScheduler); + executor.submit(streamScheduler); + } + + private String getS3PathPrefix() { + final String s3UserPathPrefix; + if (sourceConfig.getS3Prefix() != null && !sourceConfig.getS3Prefix().isBlank()) { + s3UserPathPrefix = sourceConfig.getS3Prefix() + S3_PATH_DELIMITER; + } else { + s3UserPathPrefix = ""; + } + + final String s3PathPrefix; + final Instant now = Instant.now(); + if (sourceCoordinator.getPartitionPrefix() != null) { + s3PathPrefix = s3UserPathPrefix + sourceCoordinator.getPartitionPrefix() + S3_PATH_DELIMITER + now.toEpochMilli() + S3_PATH_DELIMITER; + } else { + s3PathPrefix = s3UserPathPrefix + now.toEpochMilli() + S3_PATH_DELIMITER; + } + return s3PathPrefix; + } + + /** + * Interrupt the running of schedulers. + * Each scheduler must implement logic for gracefully shutdown. + */ + public void shutdown() { + if (executor != null) { + LOG.info("shutdown Neptune Service scheduler and worker"); + executor.shutdownNow(); + } + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/NeptuneSource.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/NeptuneSource.java new file mode 100644 index 0000000000..effb25ecdb --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/NeptuneSource.java @@ -0,0 +1,94 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune; + +import com.amazonaws.neptune.auth.NeptuneSigV4SignerException; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.annotations.DataPrepperPlugin; +import org.opensearch.dataprepper.model.annotations.DataPrepperPluginConstructor; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.plugin.PluginConfigObservable; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.Source; +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.UsesEnhancedSourceCoordination; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.PartitionFactory; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.LeaderPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Objects; +import java.util.function.Function; + + +@DataPrepperPlugin(name = "neptune", pluginType = Source.class, pluginConfigurationType = NeptuneSourceConfig.class) +public class NeptuneSource implements Source>, UsesEnhancedSourceCoordination { + private static final Logger LOG = LoggerFactory.getLogger(NeptuneSource.class); + + private final PluginMetrics pluginMetrics; + private final NeptuneSourceConfig sourceConfig; + private final PluginConfigObservable pluginConfigObservable; + private EnhancedSourceCoordinator sourceCoordinator; + private final AcknowledgementSetManager acknowledgementSetManager; + private NeptuneService neptuneService; + + private final boolean acknowledgementsEnabled; + + @DataPrepperPluginConstructor + public NeptuneSource(final PluginMetrics pluginMetrics, + final NeptuneSourceConfig sourceConfig, + final AcknowledgementSetManager acknowledgementSetManager, + final PluginConfigObservable pluginConfigObservable) { + this.pluginMetrics = pluginMetrics; + this.sourceConfig = sourceConfig; + this.acknowledgementSetManager = acknowledgementSetManager; + this.pluginConfigObservable = pluginConfigObservable; + this.acknowledgementsEnabled = sourceConfig.isAcknowledgments(); + } + + @Override + public void start(final Buffer> buffer) { + Objects.requireNonNull(sourceCoordinator); + sourceCoordinator.createPartition(new LeaderPartition()); + + try { + neptuneService = new NeptuneService(sourceCoordinator, sourceConfig, pluginMetrics, acknowledgementSetManager); + LOG.info("Start Neptune service"); + neptuneService.start(buffer); + } catch (NeptuneSigV4SignerException e) { + LOG.error("SigV4 ERROR"); + } + } + + + @Override + public void stop() { + LOG.info("Stop Neptune service"); + if (Objects.nonNull(neptuneService)) { + neptuneService.shutdown(); + } + } + + @Override + public void setEnhancedSourceCoordinator(EnhancedSourceCoordinator sourceCoordinator) { + this.sourceCoordinator = sourceCoordinator; + this.sourceCoordinator.initialize(); + } + + @Override + public Function getPartitionFactory() { + return new PartitionFactory(); + } + + @Override + public boolean areAcknowledgementsEnabled() { + return acknowledgementsEnabled; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/buffer/RecordBufferWriter.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/buffer/RecordBufferWriter.java new file mode 100644 index 0000000000..84e158ebeb --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/buffer/RecordBufferWriter.java @@ -0,0 +1,84 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.buffer; + +import io.micrometer.core.instrument.Counter; +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * Record Buffer writer that transform the source data into a JacksonEvent, + * and then writes to buffer. + */ +public class RecordBufferWriter { + private static final Logger LOG = LoggerFactory.getLogger(RecordBufferWriter.class); + static final String RECORDS_PROCESSED_COUNT = "recordsProcessed"; + static final String RECORDS_PROCESSING_ERROR_COUNT = "recordProcessingErrors"; + private final BufferAccumulator> bufferAccumulator; + private final Counter recordSuccessCounter; + private final Counter recordErrorCounter; + + private RecordBufferWriter(final BufferAccumulator> bufferAccumulator, + final PluginMetrics pluginMetrics) { + this.bufferAccumulator = bufferAccumulator; + this.recordSuccessCounter = pluginMetrics.counter(RECORDS_PROCESSED_COUNT); + this.recordErrorCounter = pluginMetrics.counter(RECORDS_PROCESSING_ERROR_COUNT); + } + + public static RecordBufferWriter create(final BufferAccumulator> bufferAccumulator, + final PluginMetrics pluginMetrics) { + return new RecordBufferWriter(bufferAccumulator, pluginMetrics); + } + + void flushBuffer() throws Exception { + bufferAccumulator.flush(); + } + + /** + * Add event record to buffer + * + * @param acknowledgementSet acknowledgmentSet keeps track of set of events + * @param record record to be written to buffer + * @throws Exception Exception if failed to write to buffer. + */ + public void addToBuffer(final AcknowledgementSet acknowledgementSet, + final Event record) throws Exception { + if (acknowledgementSet != null) { + acknowledgementSet.add(record); + } + + bufferAccumulator.add(new Record<>(record)); + } + + public void writeToBuffer(final AcknowledgementSet acknowledgementSet, + final List records) { + + int eventCount = 0; + for (final Event record : records) { + try { + addToBuffer(acknowledgementSet, record); + eventCount++; + } catch (Exception e) { + // will this cause too many logs? + LOG.error("Failed to add event to buffer due to {}", e.getMessage()); + } + } + + try { + flushBuffer(); + recordSuccessCounter.increment(eventCount); + } catch (Exception e) { + LOG.error("Failed to write {} events to buffer due to {}", eventCount, e.getMessage()); + recordErrorCounter.increment(eventCount); + } + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneDataClientFactory.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneDataClientFactory.java new file mode 100644 index 0000000000..55b3c9be4b --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneDataClientFactory.java @@ -0,0 +1,86 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.client; + +import org.opensearch.dataprepper.plugins.source.neptune.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import software.amazon.awssdk.arns.Arn; +import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; +import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; +import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.signer.NoOpSigner; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.neptunedata.NeptunedataClient; +import software.amazon.awssdk.services.sts.StsClient; +import software.amazon.awssdk.services.sts.auth.StsAssumeRoleCredentialsProvider; +import software.amazon.awssdk.services.sts.model.AssumeRoleRequest; + +import java.net.URI; +import java.util.UUID; + +public class NeptuneDataClientFactory { + public static NeptunedataClient provideNeptuneDataClient(final NeptuneSourceConfig sourceConfig) { + final URI endpoint = URI.create(String.format("https://%s:%s", sourceConfig.getHost(), sourceConfig.getPort())); + if (sourceConfig.isIamAuth()) { + final AwsConfig awsConfig = sourceConfig.getAwsConfig(); + return NeptunedataClient.builder() + .endpointOverride(endpoint) + .region(Region.of(sourceConfig.getRegion())) + .credentialsProvider(getAwsCredentials(Region.of(sourceConfig.getRegion()), awsConfig.getAwsStsRoleArn(), awsConfig.getAwsStsExternalId())) + .build(); + } else { + final ClientOverrideConfiguration clientOverrideConfiguration = + // Do not sign the request + ClientOverrideConfiguration.builder() + .putAdvancedOption(SdkAdvancedClientOption.SIGNER, new NoOpSigner()) + .build(); + + return NeptunedataClient.builder() + .endpointOverride(endpoint) + .region(Region.of(sourceConfig.getRegion())) + .overrideConfiguration(clientOverrideConfiguration) + .credentialsProvider(AnonymousCredentialsProvider.create()) + .build(); + } + } + + private static AwsCredentialsProvider getAwsCredentials( + final Region region, final String stsRoleArn, final String stsExternalId + ) { + AwsCredentialsProvider awsCredentialsProvider; + if (stsRoleArn != null && !stsRoleArn.isEmpty()) { + try { + Arn.fromString(stsRoleArn); + } catch (final Exception e) { + throw new IllegalArgumentException("Invalid ARN format for dynamodb sts_role_arn"); + } + + final StsClient stsClient = StsClient.builder() + .region(region) + .build(); + + AssumeRoleRequest.Builder assumeRoleRequestBuilder = AssumeRoleRequest.builder() + .roleSessionName("Dynamo-Source-Coordination-" + UUID.randomUUID()) + .roleArn(stsRoleArn); + + if (stsExternalId != null && !stsExternalId.isEmpty()) { + assumeRoleRequestBuilder = assumeRoleRequestBuilder.externalId(stsExternalId); + } + + awsCredentialsProvider = StsAssumeRoleCredentialsProvider.builder() + .stsClient(stsClient) + .refreshRequest(assumeRoleRequestBuilder.build()) + .build(); + + } else { + // use default credential provider + awsCredentialsProvider = DefaultCredentialsProvider.create(); + } + + return awsCredentialsProvider; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneDataClientWrapper.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneDataClientWrapper.java new file mode 100644 index 0000000000..fd2199c921 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneDataClientWrapper.java @@ -0,0 +1,106 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.client; + +import lombok.Getter; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.NeptuneStreamRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.neptunedata.NeptunedataClient; +import software.amazon.awssdk.services.neptunedata.model.GetPropertygraphStreamRequest; +import software.amazon.awssdk.services.neptunedata.model.GetPropertygraphStreamResponse; +import software.amazon.awssdk.services.neptunedata.model.GetSparqlStreamRequest; +import software.amazon.awssdk.services.neptunedata.model.GetSparqlStreamResponse; +import software.amazon.awssdk.services.neptunedata.model.IteratorType; + +import java.util.List; +import java.util.stream.Collectors; + +public class NeptuneDataClientWrapper { + private static final Logger LOG = LoggerFactory.getLogger(NeptuneDataClientWrapper.class); + private final NeptunedataClient client; + + @Getter + private final StreamType streamType; + + @Getter + private final long batchSize; + + public enum StreamType { + PROPERTY_GRAPH, + SPARQL; + + public static StreamType fromString(String name) { + if (name.equalsIgnoreCase("propertygraph")) { + return PROPERTY_GRAPH; + } else if (name.equalsIgnoreCase("sparql")) { + return SPARQL; + } + throw new IllegalArgumentException("Unknown stream type: " + name); + } + } + + protected NeptuneDataClientWrapper(final NeptuneSourceConfig sourceConfig, final long batchSize) { + this.client = NeptuneDataClientFactory.provideNeptuneDataClient(sourceConfig); + this.streamType = StreamType.fromString(sourceConfig.getStreamType()); + this.batchSize = batchSize; + } + + public static NeptuneDataClientWrapper create(final NeptuneSourceConfig sourceConfig, final long batchSize) { + return new NeptuneDataClientWrapper(sourceConfig, batchSize); + } + + private IteratorType getIteratorType(final long checkPointCommitNum, + final long checkPointOpNum) { + return (checkPointOpNum == 0 && checkPointCommitNum == 0) + ? IteratorType.TRIM_HORIZON : + IteratorType.AFTER_SEQUENCE_NUMBER; + } + + public List getStreamRecords(final long checkPointCommitNum, + final long checkPointOpNum) { + final IteratorType iteratorType = getIteratorType(checkPointCommitNum, checkPointOpNum); + final List records; + if (this.streamType == StreamType.PROPERTY_GRAPH) { + final GetPropertygraphStreamResponse propertyGraphStream = + this.getPropertyGraphStream(checkPointCommitNum, checkPointOpNum, iteratorType); + records = propertyGraphStream.records(); + } else { + records = this.getSparqlStream(checkPointCommitNum, checkPointOpNum, iteratorType).records(); + } + return records.stream().map(NeptuneStreamRecord::fromStreamRecord).collect(Collectors.toList()); + } + + public GetPropertygraphStreamResponse getPropertyGraphStream(final long checkPointCommitNum, + final long checkPointOpNum, + final IteratorType iteratorType) { + assert this.streamType == StreamType.PROPERTY_GRAPH; + final GetPropertygraphStreamRequest request = + GetPropertygraphStreamRequest + .builder() + .limit(batchSize) + .commitNum(iteratorType == IteratorType.TRIM_HORIZON ? null : checkPointCommitNum) + .opNum(iteratorType == IteratorType.TRIM_HORIZON ? null : checkPointOpNum) + .iteratorType(iteratorType) + .build(); + return this.client.getPropertygraphStream(request); + } + + public GetSparqlStreamResponse getSparqlStream(final long checkPointCommitNum, + final long checkPointOpNum, + final IteratorType iteratorType) { + assert this.streamType == StreamType.SPARQL; + final GetSparqlStreamRequest request = GetSparqlStreamRequest + .builder() + .limit(batchSize) + .commitNum(checkPointCommitNum) + .opNum(checkPointOpNum) + .iteratorType(iteratorType) + .build(); + return this.client.getSparqlStream(request); + } + +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneStreamClient.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneStreamClient.java new file mode 100644 index 0000000000..f42608ccbf --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneStreamClient.java @@ -0,0 +1,66 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.client; + +import lombok.Getter; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.NeptuneStreamRecord; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamPosition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.neptunedata.model.StreamRecordsNotFoundException; + +import java.time.Duration; +import java.util.List; + +public class NeptuneStreamClient { + private static final Logger LOG = LoggerFactory.getLogger(NeptuneStreamClient.class); + private static final long MAX_BACKOFF_TIME = 60; + private final NeptuneDataClientWrapper dataClient; + private final NeptuneStreamEventListener listener; + + @Getter + private StreamPosition streamPositionInfo; + private long retryCount; + + public NeptuneStreamClient(final NeptuneSourceConfig config, final int batchSize, final NeptuneStreamEventListener listener) { + this.dataClient = new NeptuneDataClientWrapper(config, batchSize); + this.listener = listener; + this.streamPositionInfo = StreamPosition.empty(); + this.retryCount = 0; + } + + public void setStreamPosition(final long commitNum, final long opNum) { + streamPositionInfo = new StreamPosition(commitNum, opNum); + } + + public void start() throws InterruptedException { + while (!Thread.currentThread().isInterrupted() && !listener.shouldStopNeptuneStream(streamPositionInfo)) { + try { + final List streamRecords = + this.dataClient.getStreamRecords(streamPositionInfo.getCommitNum(), streamPositionInfo.getOpNum()); + retryCount = 0; + if (!streamRecords.isEmpty()) { + final NeptuneStreamRecord lastRecord = streamRecords.get(streamRecords.size() - 1); + setStreamPosition(lastRecord.getCommitNum(), lastRecord.getOpNum()); + } + listener.onNeptuneStreamEvents(streamRecords, streamPositionInfo); + } catch (final StreamRecordsNotFoundException exception) { + final long nextBackoff = getNextBackoff(); + LOG.info("Stream is up-to-date, Sleeping for {} seconds before retrying again.", nextBackoff); + Thread.sleep(Duration.ofSeconds(nextBackoff).toMillis()); + } catch (final Exception exception) { + if (!listener.onNeptuneStreamException(exception, streamPositionInfo)) { + break; + } + } + } + } + + private long getNextBackoff() { + final long nextBackoff = (long) Math.pow(2.0f, retryCount++); + return Math.min(MAX_BACKOFF_TIME, nextBackoff); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneStreamEventListener.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneStreamEventListener.java new file mode 100644 index 0000000000..9dffc1b396 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/client/NeptuneStreamEventListener.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.client; + +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.NeptuneStreamRecord; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamPosition; + +import java.util.List; + +public interface NeptuneStreamEventListener { + /** + * @param records the records returned from the Stream. + * @param streamPosition current commitNum and OpNum in the stream. + */ + void onNeptuneStreamEvents(final List records, final StreamPosition streamPosition); + + /** + * + * @param exception any encountered exception during stream processing + * @param streamPosition current commitNum and OpNum in the stream + * @return boolean if the execution should continue after that exception is encountered. + */ + boolean onNeptuneStreamException(final Exception exception, final StreamPosition streamPosition); + + /** + * + * @param streamPosition current commitNum and OpNum in the stream + * @return boolean if stream processing should be stopped + */ + boolean shouldStopNeptuneStream(final StreamPosition streamPosition); +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/configuration/AwsConfig.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/configuration/AwsConfig.java new file mode 100644 index 0000000000..eeebf63518 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/configuration/AwsConfig.java @@ -0,0 +1,61 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import jakarta.validation.constraints.Size; +import software.amazon.awssdk.arns.Arn; + +import java.util.Map; +import java.util.Optional; + +public class AwsConfig { + private static final String AWS_IAM_ROLE = "role"; + private static final String AWS_IAM = "iam"; + + @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; + + private void validateStsRoleArn() { + final Arn arn = getArn(); + if (!AWS_IAM.equals(arn.service())) { + throw new IllegalArgumentException("sts_role_arn must be an IAM Role"); + } + final Optional resourceType = arn.resource().resourceType(); + if (resourceType.isEmpty() || !resourceType.get().equals(AWS_IAM_ROLE)) { + throw new IllegalArgumentException("sts_role_arn must be an IAM Role"); + } + } + + private Arn getArn() { + try { + return Arn.fromString(awsStsRoleArn); + } catch (final Exception e) { + throw new IllegalArgumentException(String.format("Invalid ARN format for awsStsRoleArn. Check the format of %s", awsStsRoleArn)); + } + } + + public String getAwsStsRoleArn() { + return awsStsRoleArn; + } + + public String getAwsStsExternalId() { + return awsStsExternalId; + } + + public Map getAwsStsHeaderOverrides() { + return awsStsHeaderOverrides; + } +} + diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/configuration/NeptuneSourceConfig.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/configuration/NeptuneSourceConfig.java new file mode 100644 index 0000000000..3a75b3455a --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/configuration/NeptuneSourceConfig.java @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.configuration; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.datatype.jsr310.deser.DurationDeserializer; +import jakarta.validation.Valid; +import jakarta.validation.constraints.NotNull; +import lombok.Getter; +import lombok.Setter; + +import java.time.Duration; + +@Getter +@Setter +public class NeptuneSourceConfig { + private static final int DEFAULT_PORT = 8182; + private static final Duration DEFAULT_ACKNOWLEDGEMENT_SET_TIMEOUT = Duration.ofHours(2); + + @JsonProperty("host") + private @NotNull String host; + @JsonProperty("port") + private Integer port = DEFAULT_PORT; + @JsonProperty("region") + private String region; + @JsonProperty("iam_auth") + private boolean iamAuth; + + @JsonProperty("trust_store_file_path") + private String trustStoreFilePath; + @JsonProperty("trust_store_password") + private String trustStorePassword; + + @JsonProperty("s3_bucket") + private String s3Bucket; + @JsonProperty("s3_prefix") + private String s3Prefix; + @JsonProperty("s3_region") + private String s3Region; + + @JsonProperty("enable_non_string_indexing") + private boolean enableNonStringIndexing = false; + + @JsonProperty("acknowledgments") + private boolean acknowledgments = false; + @JsonProperty("partition_acknowledgment_timeout") + @JsonDeserialize(using = DurationDeserializer.class) + private Duration partitionAcknowledgmentTimeout = DEFAULT_ACKNOWLEDGEMENT_SET_TIMEOUT; + + @JsonProperty("aws") + @NotNull + @Valid + private AwsConfig awsConfig; + + @JsonProperty("export") + private boolean export = false; + @JsonProperty("stream") + private boolean stream = false; + @JsonProperty("stream_type") + private String streamType; +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/MetadataKeyAttributes.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/MetadataKeyAttributes.java new file mode 100644 index 0000000000..b8fef52688 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/MetadataKeyAttributes.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.converter; + +public class MetadataKeyAttributes { + + static final String ID_METADATA_ATTRIBUTE = "id"; + + static final String NEPTUNE_COMMIT_TIMESTAMP_METADATA_ATTRIBUTE = "commitTimestamp"; + + static final String EVENT_VERSION_FROM_TIMESTAMP = "document_version"; + + static final String NEPTUNE_STREAM_OP_NAME_METADATA_ATTRIBUTE = "op"; + + static final String EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE = "opensearch_action"; // index or update, etc + + static final String INGESTION_EVENT_TYPE_ATTRIBUTE = "ingestion_type"; // stream or export + + static final String EVENT_S3_PARTITION_KEY = "s3_partition_key"; + +} + diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/NeptuneS3Record.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/NeptuneS3Record.java new file mode 100644 index 0000000000..99e14780cb --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/NeptuneS3Record.java @@ -0,0 +1,124 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.converter; + +import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.Builder; +import lombok.Getter; +import org.apache.commons.lang3.tuple.ImmutablePair; +import org.eclipse.rdf4j.model.Literal; +import org.eclipse.rdf4j.model.Statement; +import org.eclipse.rdf4j.model.vocabulary.RDF; +import org.opensearch.dataprepper.plugins.source.neptune.stream.StreamUtils; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.NeptuneStreamRecord; +import software.amazon.awssdk.services.neptunedata.model.PropertygraphData; +import software.amazon.awssdk.services.neptunedata.model.SparqlData; + +import java.io.IOException; + +/** + * Represents a single record stored in S3. + */ +@Builder +@Getter +public class NeptuneS3Record { + + private static final String DOCUMENT_TYPE_VERTEX = "vertex"; + private static final String DOCUMENT_TYPE_EDGE = "edge"; + private static final String DOCUMENT_TYPE_RDF = "rdf-resource"; + private static final String VERTEX_ID_PREFIX = "v://"; + private static final String EDGE_ID_PREFIX = "e://"; + private static final String PG_ENTITY_ID_KEY = "label"; + + // Reference to Neptune entity corresponding to document. For Gremlin, it will be Vertex Id for Vertex document & + // Edge ID for Edge Document. For Sparql, it will be RDF subject URI + @JsonProperty("entity_id") + private String entityId; + + // Store the Neptune entity type. Vertex/Edge label for gremlin. rdf:type for Sparql + @JsonProperty("entity_type") + private String entityType; + + // Classify Open Search document. It could be one of vertex / edge / rdf-resource + @JsonProperty("document_type") + private String documentType; + + // Nested Field for storing predicate corresponding to Graph vertex / Edge + @JsonProperty("predicate") + private ImmutablePair predicate; + + private static NeptuneS3Record convertPropertyGraphStreamRecord(final NeptuneStreamRecord record) { + if (!(record.getData() instanceof PropertygraphData)) { + throw new IllegalArgumentException("Data must be a PropertygraphData"); + } + + String entityType = null; + ImmutablePair predicate = null; + final PropertygraphData propertygraphData = ((PropertygraphData) record.getData()); + final String key = propertygraphData.key(); + if (key.equalsIgnoreCase(PG_ENTITY_ID_KEY)) { + entityType = propertygraphData.value().asMap().get("value").asString(); + } else { + predicate = ImmutablePair.of(key, NeptuneS3RecordPredicate.fromPropertGraphData(propertygraphData)); + } + + return NeptuneS3Record + .builder() + .entityId(getEntityIdForPropertyGraph(propertygraphData.type(), propertygraphData.id())) + .documentType(getDocumentTypeForPropertyGraph(propertygraphData.type())) + .entityType(entityType) + .predicate(predicate) + .build(); + } + + private static NeptuneS3Record convertSparqlStreamRecord(final NeptuneStreamRecord record) throws IOException { + if (!(record.getData() instanceof SparqlData)) { + throw new IllegalArgumentException("Data must be a SparqlData"); + } + final Statement stmt = StreamUtils.parseSparqlStatement(((SparqlData) record.getData()).stmt()); + final boolean isTypeStmt = stmt.getPredicate().isIRI() && stmt.getPredicate().equals(RDF.TYPE); + ImmutablePair predicate = null; + String entityType = null; + + if (isTypeStmt) { + entityType = stmt.getObject().stringValue(); + } else { + final String predicateName = stmt.getPredicate().stringValue(); + final NeptuneS3RecordPredicate predicateValue = NeptuneS3RecordPredicate + .builder() + .value(stmt.getObject().stringValue()) + .graph(stmt.getContext().stringValue()) + .language(stmt.getObject() instanceof Literal ? ((Literal) stmt.getObject()).getLanguage().orElse(null) : null) + .build(); + predicate = ImmutablePair.of(predicateName, predicateValue); + } + return NeptuneS3Record + .builder() + .entityId(stmt.getSubject().stringValue()) + .documentType(DOCUMENT_TYPE_RDF) + .entityType(entityType) + .predicate(predicate) + .build(); + + } + + public static NeptuneS3Record fromNeptuneStreamRecord(final NeptuneStreamRecord neptuneStreamRecord) throws IOException { + if (neptuneStreamRecord.getData() instanceof PropertygraphData) { + return convertPropertyGraphStreamRecord(neptuneStreamRecord); + } + return convertSparqlStreamRecord(neptuneStreamRecord); + } + + private static String getEntityIdForPropertyGraph(final String type, final String entityId) { + return String.format("%s%s", type.startsWith("v") ? VERTEX_ID_PREFIX : EDGE_ID_PREFIX, entityId); + } + + private static String getDocumentTypeForPropertyGraph(final String type) { + if (type.equalsIgnoreCase("vl") || type.equalsIgnoreCase("vp")) { + return DOCUMENT_TYPE_VERTEX; + } + return DOCUMENT_TYPE_EDGE; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/NeptuneS3RecordPredicate.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/NeptuneS3RecordPredicate.java new file mode 100644 index 0000000000..8a104cf022 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/NeptuneS3RecordPredicate.java @@ -0,0 +1,48 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.converter; + +import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.Builder; +import lombok.Getter; +import software.amazon.awssdk.services.neptunedata.model.PropertygraphData; + +import java.util.Objects; + +@Builder +@Getter +public class NeptuneS3RecordPredicate { + + @JsonProperty("value") + private String value; + + @JsonProperty("graph") + private String graph; + + @JsonProperty("language") + private String language; + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + NeptuneS3RecordPredicate that = (NeptuneS3RecordPredicate) o; + return Objects.equals(value, that.value) && Objects.equals(graph, that.graph) && + Objects.equals(language, that.language); + } + + @Override + public int hashCode() { + return Objects.hash(value, graph, language); + } + + public static NeptuneS3RecordPredicate fromPropertGraphData(final PropertygraphData propertygraphData) { + final String value = propertygraphData.value().asMap().get("value").asString(); + return NeptuneS3RecordPredicate + .builder() + .value(value) + .build(); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/NeptuneStreamRecordValidator.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/NeptuneStreamRecordValidator.java new file mode 100644 index 0000000000..e4947e93ea --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/NeptuneStreamRecordValidator.java @@ -0,0 +1,73 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.converter; + +import org.eclipse.rdf4j.model.IRI; +import org.eclipse.rdf4j.model.Literal; +import org.eclipse.rdf4j.model.Statement; +import org.eclipse.rdf4j.model.vocabulary.RDF; +import org.eclipse.rdf4j.model.vocabulary.XSD; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.stream.StreamUtils; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.NeptuneStreamRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.neptunedata.model.PropertygraphData; +import software.amazon.awssdk.services.neptunedata.model.SparqlData; + +import java.io.IOException; + + +/** + * Validates if the record from Neptune Streams is a valid record. + * (1) If enableNonStringIndexing in {@link NeptuneSourceConfig} is true, then all datatypes are valid and mapped to + * OS datatypes as defined in Mapping of SPARQL and Gremlin datatypes to OpenSearch + * (2) Otherwise, only String datatypes are supported and any non-string record is dropped. + */ +public class NeptuneStreamRecordValidator { + private static final Logger LOG = LoggerFactory.getLogger(NeptuneStreamRecordValidator.class); + private final boolean allowNonStringDatatypes; + + public NeptuneStreamRecordValidator(final boolean allowNonStringDatatypes) { + this.allowNonStringDatatypes = allowNonStringDatatypes; + } + + public boolean isValid(final NeptuneStreamRecord record) { + if (record.getData() instanceof SparqlData) { + return isValidSparqlRecord(record); + } + return isValidPropertyGraphRecord(record); + } + + private boolean isValidPropertyGraphRecord(final NeptuneStreamRecord record) { + if (allowNonStringDatatypes) { + return true; + } + final PropertygraphData data = (PropertygraphData) record.getData(); + final String datatype = data.value().asMap().get("dataType").asString(); + return datatype.equalsIgnoreCase("String"); + } + + private boolean isValidSparqlRecord(final NeptuneStreamRecord record) { + if (allowNonStringDatatypes) { + return true; + } + final SparqlData data = (SparqlData) record.getData(); + try { + final Statement statement = StreamUtils.parseSparqlStatement(data.stmt()); + if (!statement.getObject().isLiteral()) { + return false; + } + return isSparqlStringDatatype(((Literal) statement.getObject()).getDatatype()); + } catch (IOException e) { + LOG.error("Failed to parse Sparql statement, Skipping record: ", e); + return false; + } + } + + private static boolean isSparqlStringDatatype(IRI datatype) { + return XSD.STRING.equals(datatype) || RDF.LANGSTRING.equals(datatype); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/StreamRecordConverter.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/StreamRecordConverter.java new file mode 100644 index 0000000000..75ac68f20e --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/converter/StreamRecordConverter.java @@ -0,0 +1,102 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.converter; + +import lombok.Setter; +import lombok.SneakyThrows; +import org.opensearch.dataprepper.model.document.JacksonDocument; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.event.EventMetadata; +import org.opensearch.dataprepper.model.opensearch.OpenSearchBulkActions; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.NeptuneStreamRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.ArrayList; +import java.util.List; + +/** + * The record convert transform the source data into a JacksonEvent. + */ +public class StreamRecordConverter { + private static final Logger LOG = LoggerFactory.getLogger(StreamRecordConverter.class); + private static final String S3_PATH_DELIMITER = "/"; + private static final String STREAM_INGESTION_TYPE = "STREAM"; + private static final String STREAM_OP_ADD = "ADD"; + private static final String STREAM_OP_REMOVE = "REMOVE"; + + final String s3PathPrefix; + @Setter + private List partitions = new ArrayList<>(); + + public StreamRecordConverter(final String s3PathPrefix) { + this.s3PathPrefix = s3PathPrefix; + } + + /** + * Convert the source data into a JacksonEvent. + * + * @param neptuneRecord record that will be converted to Event + * @return Jackson document event + */ + @SneakyThrows + public Event convert(final NeptuneStreamRecord neptuneRecord) { + final Event event = JacksonDocument.builder() + .withData(NeptuneS3Record.fromNeptuneStreamRecord(neptuneRecord)) + .build(); + final EventMetadata eventMetadata = event.getMetadata(); + + eventMetadata.setAttribute(MetadataKeyAttributes.ID_METADATA_ATTRIBUTE, neptuneRecord.getId()); + eventMetadata.setAttribute(MetadataKeyAttributes.NEPTUNE_COMMIT_TIMESTAMP_METADATA_ATTRIBUTE, neptuneRecord.getCommitTimestampInMillis()); + eventMetadata.setAttribute(MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP, neptuneRecord.getCommitTimestampInMillis()); + eventMetadata.setAttribute(MetadataKeyAttributes.NEPTUNE_STREAM_OP_NAME_METADATA_ATTRIBUTE, neptuneRecord.getOp()); + eventMetadata.setAttribute(MetadataKeyAttributes.EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE, mapStreamEventNameToBulkAction(neptuneRecord.getOp())); + eventMetadata.setAttribute(MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE, STREAM_INGESTION_TYPE); + eventMetadata.setAttribute(MetadataKeyAttributes.EVENT_S3_PARTITION_KEY, this.s3PathPrefix + S3_PATH_DELIMITER + hashKeyToPartition(neptuneRecord.getId())); + + return event; + } + + private String mapStreamEventNameToBulkAction(final String op) { + switch (op) { + // Neptune's data model always uses upsert with custom script to update the document + case STREAM_OP_REMOVE: + case STREAM_OP_ADD: + return OpenSearchBulkActions.UPSERT.toString(); + default: + throw new RuntimeException("Unknown stream operation: " + op); + } + } + + private String hashKeyToPartition(final String key) { + return partitions.get(hashKeyToIndex(key)); + } + + private int hashKeyToIndex(final String key) { + try { + // Create a SHA-256 hash instance + final MessageDigest digest = MessageDigest.getInstance("SHA-256"); + // Hash the key + byte[] hashBytes = digest.digest(key.getBytes()); + // Convert the hash to an integer + int hashValue = bytesToInt(hashBytes); + // Map the hash value to an index in the list + return Math.abs(hashValue) % partitions.size(); + } catch (final NoSuchAlgorithmException e) { + return -1; + } + } + + private static int bytesToInt(byte[] bytes) { + int result = 0; + for (int i = 0; i < 4 && i < bytes.length; i++) { + result <<= 8; + result |= (bytes[i] & 0xFF); + } + return result; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/PartitionFactory.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/PartitionFactory.java new file mode 100644 index 0000000000..e7bcaa9309 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/PartitionFactory.java @@ -0,0 +1,43 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.coordination; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.DataQueryPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.LeaderPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.S3FolderPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.StreamPartition; + +import java.util.function.Function; + +/** + * Partition factory for Neptune source. + */ +public class PartitionFactory implements Function { + + @Override + public EnhancedSourcePartition apply(final SourcePartitionStoreItem partitionStoreItem) { + String sourceIdentifier = partitionStoreItem.getSourceIdentifier(); + String partitionType = sourceIdentifier.substring(sourceIdentifier.lastIndexOf('|') + 1); + + switch (partitionType) { + case DataQueryPartition.PARTITION_TYPE: + return new DataQueryPartition(partitionStoreItem); + case StreamPartition.PARTITION_TYPE: + return new StreamPartition(partitionStoreItem); + case LeaderPartition.PARTITION_TYPE: + return new LeaderPartition(partitionStoreItem); + case S3FolderPartition.PARTITION_TYPE: + return new S3FolderPartition(partitionStoreItem); + default: + // Unable to acquire other partitions. + return new GlobalState(partitionStoreItem); + } + } + + +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/DataQueryPartition.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/DataQueryPartition.java new file mode 100644 index 0000000000..732c848fba --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/DataQueryPartition.java @@ -0,0 +1,64 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.DataQueryProgressState; + +import java.util.Optional; + +public class DataQueryPartition extends EnhancedSourcePartition { + public static final String PARTITION_TYPE = "DATA_QUERY"; + + private final String collection; + private final String query; + private final DataQueryProgressState state; + + public DataQueryPartition(final SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + collection = getCollection(sourcePartitionStoreItem.getSourcePartitionKey()); + query = sourcePartitionStoreItem.getSourcePartitionKey(); + this.state = convertStringToPartitionProgressState(DataQueryProgressState.class, sourcePartitionStoreItem.getPartitionProgressState()); + + } + + public DataQueryPartition(final String query, final DataQueryProgressState state) { + this.collection = getCollection(query); + this.query = query; + this.state = state; + } + + private String getCollection(final String partitionKey) { + String[] keySplits = partitionKey.split("\\|"); + return keySplits[0]; + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return query; + } + + @Override + public Optional getProgressState() { + if (state != null) { + return Optional.of(state); + } + return Optional.empty(); + } + + public String getCollection() { + return collection; + } + + public String getQuery() { + return query; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/GlobalState.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/GlobalState.java new file mode 100644 index 0000000000..29af0e9360 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/GlobalState.java @@ -0,0 +1,60 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; + +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 EnhancedSourcePartition> { + + 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, Map state) { + this.stateName = stateName; + this.state = state; + + } + + @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/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/LeaderPartition.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/LeaderPartition.java new file mode 100644 index 0000000000..6fd8f959b7 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/LeaderPartition.java @@ -0,0 +1,55 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.LeaderProgressState; + +import java.util.Optional; + +/** + *

A LeaderPartition is for some tasks that should be done in a single node only.

+ *

Hence whatever node owns the lease of this partition will be acted as a 'leader'.

+ *

In this DocumentDB/MongoDB source design, a leader node will be responsible for:

+ *
    + *
  • Initialization process
  • + *
  • Read stream data
  • + *
+ */ +public class LeaderPartition extends EnhancedSourcePartition { + + public static final String PARTITION_TYPE = "LEADER"; + + private static final String DEFAULT_PARTITION_KEY = "GLOBAL"; + + private final LeaderProgressState state; + + public LeaderPartition() { + this.state = new LeaderProgressState(); + this.state.setInitialized(false); + } + + public LeaderPartition(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + this.state = convertStringToPartitionProgressState(LeaderProgressState.class, sourcePartitionStoreItem.getPartitionProgressState()); + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return DEFAULT_PARTITION_KEY; + } + + @Override + public Optional getProgressState() { + return Optional.of(state); + } + +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/S3FolderPartition.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/S3FolderPartition.java new file mode 100644 index 0000000000..10e6b0392b --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/S3FolderPartition.java @@ -0,0 +1,71 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; + +import java.util.Optional; + +/** + * A S3 Folder partition represents an S3 partition job to create S3 path prefix/sub folder that will + * be used to group records based on record key. + */ +public class S3FolderPartition extends EnhancedSourcePartition { + + public static final String PARTITION_TYPE = "S3_FOLDER"; + private final String bucketName; + private final String pathPrefix; + private final String region; + private final int partitionCount; + + public S3FolderPartition(final SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + String[] keySplits = sourcePartitionStoreItem.getSourcePartitionKey().split("\\|"); + bucketName = keySplits[0]; + pathPrefix = keySplits[1]; + partitionCount = Integer.parseInt(keySplits[2]); + region = keySplits[3]; + } + + public S3FolderPartition(final String bucketName, final String pathPrefix, final String region, final int partitionCount) { + this.bucketName = bucketName; + this.pathPrefix = pathPrefix; + this.region = region; + this.partitionCount = partitionCount; + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public String getPartitionKey() { + return bucketName + "|" + pathPrefix + "|" + partitionCount + "|" + region; + } + + @Override + public Optional getProgressState() { + return Optional.empty(); + } + + + public String getBucketName() { + return bucketName; + } + + public String getPathPrefix() { + return pathPrefix; + } + + public String getRegion() { + return region; + } + + public int getPartitionCount() { + return partitionCount; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/StreamPartition.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/StreamPartition.java new file mode 100644 index 0000000000..7699056fd6 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/partition/StreamPartition.java @@ -0,0 +1,46 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.coordination.partition; + +import org.opensearch.dataprepper.model.source.coordinator.SourcePartitionStoreItem; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.StreamProgressState; + +import java.util.Optional; + +public class StreamPartition extends EnhancedSourcePartition { + + public static final String PARTITION_TYPE = "STREAM"; + + private final StreamProgressState state; + + public StreamPartition(final StreamProgressState state) { + this.state = state; + } + + public StreamPartition(SourcePartitionStoreItem sourcePartitionStoreItem) { + setSourcePartitionStoreItem(sourcePartitionStoreItem); + String[] keySplits = sourcePartitionStoreItem.getSourcePartitionKey().split("\\|"); + this.state = convertStringToPartitionProgressState(StreamProgressState.class, sourcePartitionStoreItem.getPartitionProgressState()); + } + + @Override + public String getPartitionKey() { + return "neptune"; // FIXME + } + + @Override + public String getPartitionType() { + return PARTITION_TYPE; + } + + @Override + public Optional getProgressState() { + if (state != null) { + return Optional.of(state); + } + return Optional.empty(); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/state/DataQueryProgressState.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/state/DataQueryProgressState.java new file mode 100644 index 0000000000..974b65a59a --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/state/DataQueryProgressState.java @@ -0,0 +1,44 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class DataQueryProgressState { + + + @JsonProperty("executedQueries") + private long executedQueries; + + @JsonProperty("loadedRecords") + private long loadedRecords; + + @JsonProperty("exportStartTime") + private long startTime; + + public long getExecutedQueries() { + return executedQueries; + } + + public long getLoadedRecords() { + return loadedRecords; + } + + public void setExecutedQueries(long executedQueries) { + this.executedQueries = executedQueries; + } + + public void setLoadedRecords(long loadedRecords) { + this.loadedRecords = loadedRecords; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + public long getStartTime() { + return startTime; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/state/LeaderProgressState.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/state/LeaderProgressState.java new file mode 100644 index 0000000000..9197a2e97e --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/state/LeaderProgressState.java @@ -0,0 +1,21 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class LeaderProgressState { + + @JsonProperty("initialized") + private boolean initialized = false; + + public boolean isInitialized() { + return initialized; + } + + public void setInitialized(boolean initialized) { + this.initialized = initialized; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/state/StreamProgressState.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/state/StreamProgressState.java new file mode 100644 index 0000000000..124fde3c47 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/state/StreamProgressState.java @@ -0,0 +1,42 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.coordination.state; + +import com.fasterxml.jackson.annotation.JsonProperty; +import lombok.Getter; +import lombok.Setter; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamCheckpoint; + +import java.time.Instant; + +@Getter +@Setter +public class StreamProgressState { + + @JsonProperty("startTime") + private long startTime; + + @JsonProperty("commitNum") + private Long commitNum; + + @JsonProperty("opNum") + private Long opNum; + + @JsonProperty("loadedRecords") + private Long loadedRecords; + + @JsonProperty("lastUpdateTimestamp") + private long lastUpdateTimestamp; + + @JsonProperty("waitForExport") + private boolean waitForExport = false; + + public void updateFromCheckpoint(final StreamCheckpoint progress) { + this.setCommitNum(progress.getCommitNum()); + this.setOpNum(progress.getOpNum()); + this.setLoadedRecords(progress.getRecordCount()); + this.setLastUpdateTimestamp(Instant.now().toEpochMilli()); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/leader/LeaderScheduler.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/leader/LeaderScheduler.java new file mode 100644 index 0000000000..8b4db4af91 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/leader/LeaderScheduler.java @@ -0,0 +1,154 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.leader; + +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.LeaderPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.S3FolderPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.LeaderProgressState; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.StreamProgressState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.time.Instant; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; + +public class LeaderScheduler implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(LeaderScheduler.class); + + private static final int DEFAULT_PARTITION_COUNT = 100; + + /** + * Default duration to extend the timeout of lease + */ + static final int DEFAULT_EXTEND_LEASE_MINUTES = 3; + + /** + * Default interval to run lease check and shard discovery + */ + private static final Duration DEFAULT_LEASE_INTERVAL = Duration.ofMinutes(1); + + private final NeptuneSourceConfig sourceConfig; + + private final EnhancedSourceCoordinator coordinator; + private final String s3PathPrefix; + + private final Duration leaseInterval; + + private LeaderPartition leaderPartition; + + public LeaderScheduler(final EnhancedSourceCoordinator coordinator, final NeptuneSourceConfig sourceConfig, final String s3PathPrefix) { + this(coordinator, sourceConfig, s3PathPrefix, DEFAULT_LEASE_INTERVAL); + } + + LeaderScheduler(final EnhancedSourceCoordinator coordinator, + final NeptuneSourceConfig sourceConfig, + final String s3PathPrefix, + final Duration leaseInterval) { + this.sourceConfig = sourceConfig; + this.coordinator = coordinator; + checkArgument(Objects.nonNull(s3PathPrefix), "S3 path prefix must not be null"); + this.s3PathPrefix = s3PathPrefix; + this.leaseInterval = leaseInterval; + } + + @Override + public void run() { + LOG.info("Starting Leader Scheduler for initialization and stream discovery"); + + while (!Thread.currentThread().isInterrupted()) { + try { + // Try to acquire the lease if not owned. + if (leaderPartition == null) { + final Optional sourcePartition = coordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE); + LOG.info("Leader partition {}", sourcePartition); + if (sourcePartition.isPresent()) { + LOG.info("Running as a LEADER node"); + leaderPartition = (LeaderPartition) sourcePartition.get(); + } + } + // Once owned, run Normal LEADER node process. + // May want to quit this scheduler if streaming is not required + if (leaderPartition != null) { + LeaderProgressState leaderProgressState = leaderPartition.getProgressState().get(); + if (!leaderProgressState.isInitialized()) { + LOG.info("The service is not been initialized"); + init(); + } + } + + } catch (final Exception e) { + LOG.error("Exception occurred in primary leader scheduling loop", e); + } finally { + if (leaderPartition != null) { + // Extend the timeout + // will always be a leader until shutdown + coordinator.saveProgressStateForPartition(leaderPartition, Duration.ofMinutes(DEFAULT_EXTEND_LEASE_MINUTES)); + } + try { + Thread.sleep(leaseInterval.toMillis()); + } catch (final InterruptedException e) { + LOG.info("InterruptedException occurred"); + break; + } + } + } + // Should Stop + LOG.warn("Quitting Leader Scheduler"); + if (leaderPartition != null) { + coordinator.giveUpPartition(leaderPartition); + } + } + + + private void init() { + LOG.info("Try to initialize Neptune Leader Partition"); + + coordinator.createPartition(new GlobalState("neptune", null)); + + final Instant startTime = Instant.now(); + final String s3Prefix = s3PathPrefix + "neptune"; + createS3Partition(sourceConfig.getS3Bucket(), sourceConfig.getS3Region(), s3Prefix); + + if (sourceConfig.isStream()) { + createStreamPartition(startTime); + } + + LOG.debug("Update initialization state"); + LeaderProgressState leaderProgressState = leaderPartition.getProgressState().get(); + leaderProgressState.setInitialized(true); + } + + /** + * Create a partition for a S3 partition creator job in the coordination table. + */ + private void createS3Partition(final String s3Bucket, final String s3Region, final String s3PathPrefix) { + LOG.info("Creating s3 folder global partition..."); + // TODO: change to read partition count from config + coordinator.createPartition(new S3FolderPartition(s3Bucket, s3PathPrefix, s3Region, DEFAULT_PARTITION_COUNT)); + } + + /** + * Create a partition for a stream job in the coordination table. + * + * @param streamTime the start time for change events, any change events with creation datetime before this should be ignored. + */ + private void createStreamPartition(final Instant streamTime) { + LOG.info("Creating stream global partition..."); + final StreamProgressState streamProgressState = new StreamProgressState(); + streamProgressState.setStartTime(streamTime.toEpochMilli()); + streamProgressState.setLastUpdateTimestamp(Instant.now().toEpochMilli()); + coordinator.createPartition(new StreamPartition(streamProgressState)); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/ExportLoadStatus.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/ExportLoadStatus.java new file mode 100644 index 0000000000..00d07ffb8b --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/ExportLoadStatus.java @@ -0,0 +1,95 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.model; + +import java.util.Map; + +public class ExportLoadStatus { + + public static final String TOTAL_PARTITIONS = "totalPartitions"; + public static final String LOADED_PARTITIONS = "loadedPartitions"; + public static final String LOADED_RECORDS = "loadedRecords"; + + public static final String LAST_UPDATE_TIMESTAMP = "lastUpdateTimestamp"; + public static final String TOTAL_PARTITIONS_COMPLETE = "totalPartitionsComplete"; + + private long totalPartitions; + private long loadedPartitions; + private long loadedRecords; + private long lastUpdateTimestamp; + private boolean isTotalParitionsComplete; + + public ExportLoadStatus(long totalPartitions, + long loadedPartitions, + long loadedRecords, + long lastUpdateTimestamp, + boolean isTotalParitionsComplete) { + this.totalPartitions = totalPartitions; + this.loadedPartitions = loadedPartitions; + this.loadedRecords = loadedRecords; + this.lastUpdateTimestamp = lastUpdateTimestamp; + this.isTotalParitionsComplete = isTotalParitionsComplete; + } + + public long getTotalPartitions() { + return totalPartitions; + } + + public void setLoadedPartitions(long loadedPartitions) { + this.loadedPartitions = loadedPartitions; + } + + public long getLoadedPartitions() { + return loadedPartitions; + } + + public void setTotalPartitions(long totalPartitions) { + this.totalPartitions = totalPartitions; + } + + public long getLoadedRecords() { + return loadedRecords; + } + + public void setLoadedRecords(long loadedRecords) { + this.loadedRecords = loadedRecords; + } + + public void setLastUpdateTimestamp(long lastUpdateTimestamp) { + this.lastUpdateTimestamp = lastUpdateTimestamp; + } + + public boolean isTotalParitionsComplete() { + return isTotalParitionsComplete; + } + + public void setTotalParitionsComplete(boolean totalParitionsComplete) { + isTotalParitionsComplete = totalParitionsComplete; + } + + public long getLastUpdateTimestamp() { + return lastUpdateTimestamp; + } + + public Map toMap() { + return Map.of( + TOTAL_PARTITIONS, totalPartitions, + LOADED_PARTITIONS, loadedPartitions, + LOADED_RECORDS, loadedRecords, + LAST_UPDATE_TIMESTAMP, lastUpdateTimestamp, + TOTAL_PARTITIONS_COMPLETE, isTotalParitionsComplete + ); + } + + public static ExportLoadStatus fromMap(Map map) { + return new ExportLoadStatus( + ((Number) map.get(TOTAL_PARTITIONS)).intValue(), + ((Number) map.get(LOADED_PARTITIONS)).intValue(), + ((Number) map.get(LOADED_RECORDS)).longValue(), + ((Number) map.get(LAST_UPDATE_TIMESTAMP)).longValue(), + (Boolean) map.get(TOTAL_PARTITIONS_COMPLETE) + ); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/PartitionIdentifierBatch.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/PartitionIdentifierBatch.java new file mode 100644 index 0000000000..16281665df --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/PartitionIdentifierBatch.java @@ -0,0 +1,36 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.model; + +import org.opensearch.dataprepper.model.source.coordinator.PartitionIdentifier; + +import java.util.List; + +public class PartitionIdentifierBatch { + private final List partitionIdentifiers; + + private final boolean isLastBatch; + private final Object endDocId; + + public PartitionIdentifierBatch(final List partitionIdentifiers, + final boolean isLastBatch, + final Object endDocId) { + this.partitionIdentifiers = partitionIdentifiers; + this.isLastBatch = isLastBatch; + this.endDocId = endDocId; + } + + public List getPartitionIdentifiers() { + return partitionIdentifiers; + } + + public boolean isLastBatch() { + return isLastBatch; + } + + public Object getEndDocId() { + return endDocId; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/S3PartitionStatus.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/S3PartitionStatus.java new file mode 100644 index 0000000000..15ec246296 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/S3PartitionStatus.java @@ -0,0 +1,35 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.model; + +import java.util.List; +import java.util.Map; + +public class S3PartitionStatus { + + private static final String PARTITIONS = "partitions"; + + private final List partitions; + + public S3PartitionStatus(final List partitions) { + this.partitions = partitions; + } + + public List getPartitions() { + return partitions; + } + + public Map toMap() { + return Map.of( + PARTITIONS, partitions + ); + } + + public static S3PartitionStatus fromMap(Map map) { + return new S3PartitionStatus( + ((List) map.get(PARTITIONS)) + ); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/StreamLoadStatus.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/StreamLoadStatus.java new file mode 100644 index 0000000000..b3ecaf94c3 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/model/StreamLoadStatus.java @@ -0,0 +1,38 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.model; + +import java.util.Map; + +public class StreamLoadStatus { + + private static final String EXPORT_END_TIMESTAMP = "exportEndTimestamp"; + + private long exportEndTimestamp; + + public StreamLoadStatus(long exportEndTimestamp) { + this.exportEndTimestamp = exportEndTimestamp; + } + + public void setExportEndTimestamp(long exportEndTimestamp) { + this.exportEndTimestamp = exportEndTimestamp; + } + + public long getExportEndTimestamp() { + return exportEndTimestamp; + } + + public Map toMap() { + return Map.of( + EXPORT_END_TIMESTAMP, exportEndTimestamp + ); + } + + public static StreamLoadStatus fromMap(Map map) { + return new StreamLoadStatus( + ((Number) map.get(EXPORT_END_TIMESTAMP)).longValue() + ); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3FolderPartitionCoordinator.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3FolderPartitionCoordinator.java new file mode 100644 index 0000000000..9fc896485a --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3FolderPartitionCoordinator.java @@ -0,0 +1,33 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.s3partition; + +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.neptune.model.S3PartitionStatus; + +import java.util.Optional; + +/** + * A helper class to query S3 Folder partition status using source coordinator APIs under the hood. + */ +public class S3FolderPartitionCoordinator { + private final EnhancedSourceCoordinator enhancedSourceCoordinator; + + public S3FolderPartitionCoordinator(final EnhancedSourceCoordinator enhancedSourceCoordinator) { + this.enhancedSourceCoordinator = enhancedSourceCoordinator; + } + + public Optional getGlobalS3FolderCreationStatus() { + final Optional partition = enhancedSourceCoordinator.getPartition(S3PartitionCreatorScheduler.S3_FOLDER_PREFIX); + if (partition.isPresent()) { + final GlobalState globalState = (GlobalState) partition.get(); + return Optional.of(S3PartitionStatus.fromMap(globalState.getProgressState().get())); + } else { + return Optional.empty(); + } + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreator.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreator.java new file mode 100644 index 0000000000..bb58f65fd3 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreator.java @@ -0,0 +1,30 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.s3partition; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; + +public class S3PartitionCreator { + private static final Logger LOG = LoggerFactory.getLogger(S3PartitionCreator.class); + private final int partitionCount; + + S3PartitionCreator(final int partitionCount) { + this.partitionCount = partitionCount; + } + + List createPartition() { + final List partitions = new ArrayList<>(); + for (int i = 0; i < partitionCount; i++) { + String partitionName = String.format("%02x", i) + "/"; + partitions.add(partitionName); + } + LOG.info("S3 partition created successfully."); + return partitions; + } +} \ No newline at end of file diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreatorScheduler.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreatorScheduler.java new file mode 100644 index 0000000000..44b36560f5 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreatorScheduler.java @@ -0,0 +1,73 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.s3partition; + +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.S3FolderPartition; +import org.opensearch.dataprepper.plugins.source.neptune.model.S3PartitionStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; +import java.util.Optional; + +public class S3PartitionCreatorScheduler extends S3FolderPartitionCoordinator implements Runnable { + + private static final Logger LOG = LoggerFactory.getLogger(S3PartitionCreatorScheduler.class); + public static final String S3_FOLDER_PREFIX = "S3-FOLDER-"; + private static final int DEFAULT_TAKE_LEASE_INTERVAL_MILLIS = 60_000; + + private final EnhancedSourceCoordinator sourceCoordinator; + + public S3PartitionCreatorScheduler(final EnhancedSourceCoordinator sourceCoordinator) { + super(sourceCoordinator); + this.sourceCoordinator = sourceCoordinator; + } + + @Override + public void run() { + while (!Thread.currentThread().isInterrupted()) { + try { + final Optional sourcePartition = sourceCoordinator.acquireAvailablePartition(S3FolderPartition.PARTITION_TYPE); + if (sourcePartition.isPresent()) { + final S3FolderPartition s3FolderPartition = (S3FolderPartition) sourcePartition.get(); + final List s3Folders = createS3BucketPartitions(s3FolderPartition.getPartitionCount()); + sourceCoordinator.completePartition(s3FolderPartition); + final S3PartitionStatus s3PartitionStatus = new S3PartitionStatus(s3Folders); + sourceCoordinator.createPartition(new GlobalState(S3_FOLDER_PREFIX, s3PartitionStatus.toMap())); + break; + } + + try { + Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException e) { + LOG.info("The S3 partition creator scheduler was interrupted while waiting to retry, stopping processing"); + break; + } + + final Optional s3PartitionStatus = getGlobalS3FolderCreationStatus(); + if (s3PartitionStatus.isPresent()) { + LOG.info("The S3 folder partition global state created."); + break; + } + } catch (final Exception e) { + LOG.error("Received an exception during creation of S3 partition folder, backing off and retrying", e); + try { + Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException ex) { + LOG.info("The S3 partition creator scheduler was interrupted while waiting to retry, stopping processing"); + break; + } + } + } + } + + private List createS3BucketPartitions(int partitionCount) { + final S3PartitionCreator s3PartitionCreator = new S3PartitionCreator(partitionCount); + return s3PartitionCreator.createPartition(); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/CheckpointStatus.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/CheckpointStatus.java new file mode 100644 index 0000000000..0047108e95 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/CheckpointStatus.java @@ -0,0 +1,38 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream; + +import lombok.Getter; +import lombok.Setter; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamCheckpoint; + +@Getter +@Setter +public class CheckpointStatus { + private final StreamCheckpoint checkpoint; + private AcknowledgmentStatus acknowledgeStatus; + private final long createTimestamp; + private Long acknowledgedTimestamp; + + enum AcknowledgmentStatus { + POSITIVE_ACK, + NEGATIVE_ACK, + NO_ACK + } + + public CheckpointStatus(final StreamCheckpoint checkpoint, final long createTimestamp) { + this.checkpoint = checkpoint; + this.acknowledgeStatus = AcknowledgmentStatus.NO_ACK; + this.createTimestamp = createTimestamp; + } + + public boolean isPositiveAcknowledgement() { + return this.acknowledgeStatus == AcknowledgmentStatus.POSITIVE_ACK; + } + + public boolean isNegativeAcknowledgement() { + return this.acknowledgeStatus == AcknowledgmentStatus.NEGATIVE_ACK; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/DataStreamPartitionCheckpoint.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/DataStreamPartitionCheckpoint.java new file mode 100644 index 0000000000..fbc3b06ad6 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/DataStreamPartitionCheckpoint.java @@ -0,0 +1,99 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream; + +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.neptune.model.StreamLoadStatus; +import org.opensearch.dataprepper.plugins.source.neptune.s3partition.S3FolderPartitionCoordinator; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamCheckpoint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Optional; + +import static org.opensearch.dataprepper.model.source.s3.S3ScanEnvironmentVariables.STOP_S3_SCAN_PROCESSING_PROPERTY; + +/** + * A helper class to handle the data query partition status and the progress state + * It will use coordinator APIs under the hood. + */ +public class DataStreamPartitionCheckpoint extends S3FolderPartitionCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(DataStreamPartitionCheckpoint.class); + + public static final String STREAM_PREFIX = "STREAM-"; + static final Duration CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE = Duration.ofMinutes(5); + + private final EnhancedSourceCoordinator enhancedSourceCoordinator; + + private final StreamPartition streamPartition; + + public DataStreamPartitionCheckpoint(final EnhancedSourceCoordinator enhancedSourceCoordinator, + final StreamPartition streamPartition) { + super(enhancedSourceCoordinator); + this.enhancedSourceCoordinator = enhancedSourceCoordinator; + this.streamPartition = streamPartition; + } + + private void setProgressState(final StreamCheckpoint progress) { + //Always has a state. + Optional progressState = streamPartition.getProgressState(); + progressState.get().updateFromCheckpoint(progress); + } + + /** + * This method is to do a checkpoint with latest resume token processed. + * Note that this should be called on a regular basis even there are no changes to resume token + * As the checkpoint will also extend the timeout for the lease + * + * @param checkpointProgress checkpoint progress object, including info e.g. stream position, record count + */ + public void checkpoint(final StreamCheckpoint checkpointProgress) { + LOG.debug("Checkpoint stream partition with record number {}", checkpointProgress.getRecordCount()); + setProgressState(checkpointProgress); + enhancedSourceCoordinator.saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + } + + + public void extendLease() { + LOG.debug("Extending lease of stream partition"); + enhancedSourceCoordinator.saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + } + + /** + * This method is to reset checkpoint when change stream is invalid. The current thread will give up partition and new thread + * will take ownership of partition. If change stream is valid then new thread proceeds with processing change stream else the + * process repeats. + */ + public void resetCheckpoint() { + LOG.debug("Resetting checkpoint stream partition"); + setProgressState(StreamCheckpoint.emptyProgress()); + enhancedSourceCoordinator.giveUpPartition(streamPartition); + System.clearProperty(STOP_S3_SCAN_PROCESSING_PROPERTY); + } + + public Optional getGlobalStreamLoadStatus() { + final Optional partition = enhancedSourceCoordinator.getPartition(STREAM_PREFIX + streamPartition.getPartitionKey()); + if (partition.isPresent()) { + final GlobalState globalState = (GlobalState) partition.get(); + return Optional.of(StreamLoadStatus.fromMap(globalState.getProgressState().get())); + } else { + return Optional.empty(); + } + } + + public void updateStreamPartitionForAcknowledgmentWait(final Duration acknowledgmentSetTimeout) { + enhancedSourceCoordinator.saveProgressStateForPartition(streamPartition, acknowledgmentSetTimeout); + } + + public void giveUpPartition() { + enhancedSourceCoordinator.giveUpPartition(streamPartition); + System.clearProperty(STOP_S3_SCAN_PROCESSING_PROPERTY); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamAcknowledgementManager.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamAcknowledgementManager.java new file mode 100644 index 0000000000..e794dd14c3 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamAcknowledgementManager.java @@ -0,0 +1,169 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream; + +import com.google.common.annotations.VisibleForTesting; +import org.opensearch.dataprepper.common.concurrent.BackgroundThreadFactory; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamCheckpoint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.time.Instant; +import java.util.Optional; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + + +public class StreamAcknowledgementManager { + private static final Logger LOG = LoggerFactory.getLogger(StreamAcknowledgementManager.class); + private static final int CHECKPOINT_RECORD_INTERVAL = 50; + private final ConcurrentLinkedQueue checkpoints = new ConcurrentLinkedQueue<>(); + private final ConcurrentHashMap ackStatus = new ConcurrentHashMap<>(); + private final AcknowledgementSetManager acknowledgementSetManager; + private final DataStreamPartitionCheckpoint partitionCheckpoint; + + private final Duration partitionAcknowledgmentTimeout; + private final int acknowledgementMonitorWaitTimeInMs; + private final int checkPointIntervalInMs; + private final ExecutorService executorService; + private Future monitoringTask; + private boolean enableAcknowledgement = false; + + public StreamAcknowledgementManager(final AcknowledgementSetManager acknowledgementSetManager, + final DataStreamPartitionCheckpoint partitionCheckpoint, + final Duration partitionAcknowledgmentTimeout, + final int acknowledgementMonitorWaitTimeInMs, + final int checkPointIntervalInMs) { + this.acknowledgementSetManager = acknowledgementSetManager; + this.partitionCheckpoint = partitionCheckpoint; + this.partitionAcknowledgmentTimeout = partitionAcknowledgmentTimeout; + this.acknowledgementMonitorWaitTimeInMs = acknowledgementMonitorWaitTimeInMs; + this.checkPointIntervalInMs = checkPointIntervalInMs; + executorService = Executors.newSingleThreadExecutor(BackgroundThreadFactory.defaultExecutorThreadFactory("neptune-stream-ack-monitor")); + } + + void init(final Consumer stopWorkerConsumer) { + enableAcknowledgement = true; + monitoringTask = executorService.submit(() -> monitorAcknowledgment(executorService, stopWorkerConsumer)); + } + + private void monitorAcknowledgment(final ExecutorService executorService, final Consumer stopWorkerConsumer) { + long lastCheckpointTime = System.currentTimeMillis(); + CheckpointStatus lastCheckpointStatus = null; + while (!Thread.currentThread().isInterrupted()) { + try { + CheckpointStatus checkpointStatus = checkpoints.peek(); + if (checkpointStatus != null) { + if (checkpointStatus.isPositiveAcknowledgement()) { + if (System.currentTimeMillis() - lastCheckpointTime >= checkPointIntervalInMs) { + long ackCount = 0; + do { + lastCheckpointStatus = checkpoints.poll(); + ackStatus.remove(checkpointStatus.getCheckpoint().getPosition().asAckString()); + checkpointStatus = checkpoints.peek(); + ackCount++; + // at high TPS each ack contains 100 records. This should checkpoint every 100*50 = 5000 records. + if (ackCount % CHECKPOINT_RECORD_INTERVAL == 0) { + checkpoint(lastCheckpointStatus.getCheckpoint()); + } + } while (checkpointStatus != null && checkpointStatus.isPositiveAcknowledgement()); + checkpoint(lastCheckpointStatus.getCheckpoint()); + lastCheckpointTime = System.currentTimeMillis(); + } + } else { + LOG.debug("Checkpoint not complete for: {}", checkpointStatus.getCheckpoint()); + final Duration ackWaitDuration = Duration.between(Instant.ofEpochMilli(checkpointStatus.getCreateTimestamp()), Instant.now()); + if (checkpointStatus.isNegativeAcknowledgement()) { + // Give up partition and should interrupt parent thread to stop processing stream + if (lastCheckpointStatus != null && lastCheckpointStatus.isPositiveAcknowledgement()) { + partitionCheckpoint.checkpoint(lastCheckpointStatus.getCheckpoint()); + } + LOG.warn("Acknowledgement not received for the checkpoint {} past wait time. Giving up partition.", checkpointStatus.getCheckpoint()); + partitionCheckpoint.giveUpPartition(); + break; + } + } + } else { + if (System.currentTimeMillis() - lastCheckpointTime >= checkPointIntervalInMs) { + LOG.debug("No records processed. Extend the lease of the partition worker."); + partitionCheckpoint.extendLease(); + lastCheckpointTime = System.currentTimeMillis(); + } + } + } catch (Exception e) { + LOG.warn("Exception monitoring acknowledgments. The stream record processing will start from previous checkpoint.", e); + break; + } + + try { + Thread.sleep(acknowledgementMonitorWaitTimeInMs); + } catch (InterruptedException ex) { + break; + } + } + stopWorkerConsumer.accept(null); + executorService.shutdown(); + } + + private void checkpoint(final StreamCheckpoint progress) { + LOG.debug("Perform regular checkpointing for: {}", progress); + partitionCheckpoint.checkpoint(progress); + } + + Optional createAcknowledgementSet(final StreamCheckpoint checkpoint) { + if (!enableAcknowledgement) { + return Optional.empty(); + } + + final CheckpointStatus checkpointStatus = new CheckpointStatus(checkpoint, Instant.now().toEpochMilli()); + checkpoints.add(checkpointStatus); + ackStatus.put(checkpointStatus.getCheckpoint().getPosition().asAckString(), checkpointStatus); + LOG.debug("Creating acknowledgment for: {}", checkpoint); + return Optional.of(acknowledgementSetManager.create((result) -> { + final CheckpointStatus ackCheckpointStatus = ackStatus.get(checkpoint.getPosition().asAckString()); + ackCheckpointStatus.setAcknowledgedTimestamp(Instant.now().toEpochMilli()); + if (result) { + ackCheckpointStatus.setAcknowledgeStatus(CheckpointStatus.AcknowledgmentStatus.POSITIVE_ACK); + LOG.debug("Received acknowledgment of completion from sink for checkpoint: {}", ackCheckpointStatus.getCheckpoint()); + } else { + ackCheckpointStatus.setAcknowledgeStatus(CheckpointStatus.AcknowledgmentStatus.NEGATIVE_ACK); + LOG.warn("Negative acknowledgment received for checkpoint, resetting checkpoint {}", ackCheckpointStatus.getCheckpoint()); + // default CheckpointStatus acknowledged value is false. The monitorCheckpoints method will time out + // and reprocess stream from last successful checkpoint in the order. + } + }, partitionAcknowledgmentTimeout)); + } + + void shutdown() { + if (monitoringTask != null) { + monitoringTask.cancel(true); + } + try { + if (!executorService.awaitTermination(5, TimeUnit.SECONDS)) { + this.executorService.shutdownNow(); + } + } catch (InterruptedException e) { + this.executorService.shutdownNow(); + } + } + + @VisibleForTesting + ConcurrentHashMap getAcknowledgementStatus() { + return ackStatus; + } + + @VisibleForTesting + ConcurrentLinkedQueue getCheckpoints() { + return checkpoints; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamScheduler.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamScheduler.java new file mode 100644 index 0000000000..fff98a9eed --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamScheduler.java @@ -0,0 +1,120 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream; + +import org.opensearch.dataprepper.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.buffer.RecordBufferWriter; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.converter.StreamRecordConverter; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.StreamPartition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.time.Duration; +import java.util.Objects; +import java.util.Optional; + +import static com.google.common.base.Preconditions.checkArgument; +//import static org.opensearch.dataprepper.model.source.s3.S3ScanEnvironmentVariables.STOP_S3_SCAN_PROCESSING_PROPERTY; + +public class StreamScheduler implements Runnable { + private static final Logger LOG = LoggerFactory.getLogger(StreamScheduler.class); + private static final int DEFAULT_TAKE_LEASE_INTERVAL_MILLIS = 60_000; + static final int DEFAULT_CHECKPOINT_INTERVAL_MILLS = 60_000; + static final int DEFAULT_BUFFER_WRITE_INTERVAL_MILLS = 5_000; + private static final int DEFAULT_MONITOR_WAIT_TIME_MS = 15_000; + /** + * Number of records to accumulate before flushing to buffer + */ + static final int DEFAULT_BUFFER_BATCH_SIZE = 10; + /** + * Number of stream records to accumulate to write to buffer and checkpoint + */ + static final int DEFAULT_RECORD_FLUSH_BATCH_SIZE = 100; + + static final Duration BUFFER_TIMEOUT = Duration.ofSeconds(60); + private final EnhancedSourceCoordinator sourceCoordinator; + private final RecordBufferWriter recordBufferWriter; + private final AcknowledgementSetManager acknowledgementSetManager; + private final NeptuneSourceConfig sourceConfig; + private final String s3PathPrefix; + private final PluginMetrics pluginMetrics; + + public StreamScheduler(final EnhancedSourceCoordinator sourceCoordinator, + final Buffer> buffer, + final AcknowledgementSetManager acknowledgementSetManager, + final NeptuneSourceConfig sourceConfig, + final String s3PathPrefix, + final PluginMetrics pluginMetrics) { + this.sourceCoordinator = sourceCoordinator; + final BufferAccumulator> bufferAccumulator = BufferAccumulator.create(buffer, DEFAULT_BUFFER_BATCH_SIZE, BUFFER_TIMEOUT); + recordBufferWriter = RecordBufferWriter.create(bufferAccumulator, pluginMetrics); + this.acknowledgementSetManager = acknowledgementSetManager; + this.sourceConfig = sourceConfig; + checkArgument(Objects.nonNull(s3PathPrefix), "S3 path prefix must not be null"); + this.s3PathPrefix = s3PathPrefix; + this.pluginMetrics = pluginMetrics; + } + + @Override + public void run() { + StreamPartition streamPartition = null; + while (!Thread.currentThread().isInterrupted()) { + try { + final Optional sourcePartition = sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE); + if (sourcePartition.isPresent()) { + LOG.info("Acquired partition to read from streams"); + + // TODO: check if this is necessary +// if (sourceConfig.isDisableS3ReadForLeader()) { +// System.setProperty(STOP_S3_SCAN_PROCESSING_PROPERTY, "true"); +// } + + streamPartition = (StreamPartition) sourcePartition.get(); + final StreamWorker streamWorker = getStreamWorker(streamPartition); + streamWorker.processStream(streamPartition); + } + try { + Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException e) { + LOG.info("The StreamScheduler was interrupted while waiting to retry, stopping processing"); + break; + } + } catch (final Exception e) { + LOG.error("Received an exception during stream processing from Neptune, backing off and retrying", e); + if (streamPartition != null) { +// if (sourceConfig.isDisableS3ReadForLeader()) { +// System.clearProperty(STOP_S3_SCAN_PROCESSING_PROPERTY); +// } + sourceCoordinator.giveUpPartition(streamPartition); + } + + try { + Thread.sleep(DEFAULT_TAKE_LEASE_INTERVAL_MILLIS); + } catch (final InterruptedException ex) { + LOG.info("The StreamScheduler was interrupted while waiting to retry, stopping processing"); + break; + } + } + } + } + + private StreamWorker getStreamWorker(final StreamPartition streamPartition) { + final DataStreamPartitionCheckpoint partitionCheckpoint = new DataStreamPartitionCheckpoint(sourceCoordinator, streamPartition); + final StreamAcknowledgementManager streamAcknowledgementManager = new StreamAcknowledgementManager(acknowledgementSetManager, partitionCheckpoint, + sourceConfig.getPartitionAcknowledgmentTimeout(), DEFAULT_MONITOR_WAIT_TIME_MS, DEFAULT_CHECKPOINT_INTERVAL_MILLS); + final StreamRecordConverter recordConverter = new StreamRecordConverter(s3PathPrefix); + return StreamWorker.create(recordBufferWriter, recordConverter, sourceConfig, + streamAcknowledgementManager, partitionCheckpoint, pluginMetrics, DEFAULT_RECORD_FLUSH_BATCH_SIZE, + DEFAULT_CHECKPOINT_INTERVAL_MILLS, DEFAULT_BUFFER_WRITE_INTERVAL_MILLS, 100 /* FIXME */); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamUtils.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamUtils.java new file mode 100644 index 0000000000..d75b0bf56f --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamUtils.java @@ -0,0 +1,28 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream; + +import org.eclipse.rdf4j.model.Model; +import org.eclipse.rdf4j.model.Statement; +import org.eclipse.rdf4j.rio.RDFFormat; +import org.eclipse.rdf4j.rio.Rio; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; + +public class StreamUtils { + public static Statement parseSparqlStatement(final String stmt) throws IOException { + final InputStream inputStream = new ByteArrayInputStream(stmt.getBytes(StandardCharsets.UTF_8)); + final Model parse = Rio.parse(inputStream, RDFFormat.NQUADS); + return new ArrayList<>(parse).get(0); + } + + public static String getSparqlSubject(final String stmt) throws IOException { + return parseSparqlStatement(stmt).getSubject().stringValue(); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamWorker.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamWorker.java new file mode 100644 index 0000000000..56220d410f --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamWorker.java @@ -0,0 +1,362 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream; + +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; +import org.opensearch.dataprepper.common.concurrent.BackgroundThreadFactory; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.plugins.source.neptune.buffer.RecordBufferWriter; +import org.opensearch.dataprepper.plugins.source.neptune.client.NeptuneStreamClient; +import org.opensearch.dataprepper.plugins.source.neptune.client.NeptuneStreamEventListener; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.converter.StreamRecordConverter; +import org.opensearch.dataprepper.plugins.source.neptune.converter.NeptuneStreamRecordValidator; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.neptune.model.S3PartitionStatus; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.NeptuneStreamRecord; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamCheckpoint; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamPosition; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import software.amazon.awssdk.services.neptunedata.model.ExpiredStreamException; +import software.amazon.awssdk.services.neptunedata.model.InvalidParameterException; + +import java.time.Duration; +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +public class StreamWorker implements NeptuneStreamEventListener { + private static final Logger LOG = LoggerFactory.getLogger(StreamWorker.class); + + static final String SUCCESS_ITEM_COUNTER_NAME = "changeEventsProcessed"; + static final String FAILURE_ITEM_COUNTER_NAME = "changeEventsProcessingErrors"; + static final String BYTES_RECEIVED = "bytesReceived"; + static final String BYTES_PROCESSED = "bytesProcessed"; + + private static final long BUFFER_WRITE_TIMEOUT_MILLIS = Duration.ofSeconds(15).toMillis(); + private static final long S3_PARTITIONS_WAIT_TIME_MILLIS = Duration.ofSeconds(10).toMillis(); + + private static final int STREAM_RECORDS_BATCH_SIZE = 10_000; + + private final NeptuneSourceConfig sourceConfig; + private final RecordBufferWriter recordBufferWriter; + private final StreamRecordConverter streamRecordConverter; + private final DataStreamPartitionCheckpoint partitionCheckpoint; + + private final Counter successItemsCounter; + private final Counter failureItemsCounter; + private final DistributionSummary bytesReceivedSummary; + private final DistributionSummary bytesProcessedSummary; + private final NeptuneStreamRecordValidator streamRecordValidator; + private final NeptuneStreamClient streamClient; + private final StreamAcknowledgementManager streamAcknowledgementManager; + private final PluginMetrics pluginMetrics; + + private final int recordFlushBatchSize; + private final int checkPointIntervalInMs; + private final int bufferWriteIntervalInMs; + private final int streamBatchSize; + + private boolean stopWorker = false; + private boolean isUnrecoverableError = false; + + private final ExecutorService streamCheckpointExecutorService; + + S3PartitionStatus s3PartitionStatus; + + final List records; + long lastBufferWriteTime = System.currentTimeMillis(); + + private StreamCheckpoint lastLocalCheckpoint; + private final StreamCheckpoint currentCheckpoint; + + private final Lock lock; + + public static StreamWorker create(final RecordBufferWriter recordBufferWriter, + final StreamRecordConverter recordConverter, + final NeptuneSourceConfig sourceConfig, + final StreamAcknowledgementManager streamAcknowledgementManager, + final DataStreamPartitionCheckpoint partitionCheckpoint, + final PluginMetrics pluginMetrics, + final int recordFlushBatchSize, + final int checkPointIntervalInMs, + final int bufferWriteIntervalInMs, + final int streamBatchSize + ) { + return new StreamWorker(recordBufferWriter, recordConverter, sourceConfig, streamAcknowledgementManager, partitionCheckpoint, + pluginMetrics, recordFlushBatchSize, checkPointIntervalInMs, bufferWriteIntervalInMs, streamBatchSize); + } + + public StreamWorker(final RecordBufferWriter recordBufferWriter, + final StreamRecordConverter streamRecordConverter, + final NeptuneSourceConfig sourceConfig, + final StreamAcknowledgementManager streamAcknowledgementManager, + final DataStreamPartitionCheckpoint partitionCheckpoint, + final PluginMetrics pluginMetrics, + final int recordFlushBatchSize, + final int checkPointIntervalInMs, + final int bufferWriteIntervalInMs, + final int streamBatchSize + ) { + this.recordBufferWriter = recordBufferWriter; + this.streamRecordConverter = streamRecordConverter; + this.sourceConfig = sourceConfig; + this.streamAcknowledgementManager = streamAcknowledgementManager; + this.streamRecordValidator = new NeptuneStreamRecordValidator(sourceConfig.isEnableNonStringIndexing()); + this.partitionCheckpoint = partitionCheckpoint; + this.pluginMetrics = pluginMetrics; + this.recordFlushBatchSize = recordFlushBatchSize; + this.checkPointIntervalInMs = checkPointIntervalInMs; + this.bufferWriteIntervalInMs = bufferWriteIntervalInMs; + this.streamBatchSize = streamBatchSize; + this.successItemsCounter = pluginMetrics.counter(SUCCESS_ITEM_COUNTER_NAME); + this.failureItemsCounter = pluginMetrics.counter(FAILURE_ITEM_COUNTER_NAME); + this.bytesReceivedSummary = pluginMetrics.summary(BYTES_RECEIVED); + this.bytesProcessedSummary = pluginMetrics.summary(BYTES_PROCESSED); + this.lock = new ReentrantLock(); + this.lastLocalCheckpoint = this.currentCheckpoint = new StreamCheckpoint(new StreamPosition(0L, 0L), 0L); + this.streamClient = new NeptuneStreamClient(sourceConfig, STREAM_RECORDS_BATCH_SIZE, this); + this.records = new ArrayList<>(); + // this.documentDBAggregateMetrics = documentDBAggregateMetrics; + + if (sourceConfig.isAcknowledgments()) { + streamAcknowledgementManager.init((Void) -> stop()); + } + + // buffer write and checkpoint in separate thread on timeout + // TODO:: can probably a scheduled executor + this.streamCheckpointExecutorService = Executors.newSingleThreadExecutor(BackgroundThreadFactory.defaultExecutorThreadFactory("neptune-stream-checkpoint")); + this.streamCheckpointExecutorService.submit(this::bufferWriteAndCheckpointStream); + } + + private boolean shouldWaitForS3Partition() { + final Optional globalS3FolderCreationStatus = partitionCheckpoint.getGlobalS3FolderCreationStatus(); + if (globalS3FolderCreationStatus.isPresent()) { + s3PartitionStatus = globalS3FolderCreationStatus.get(); + return false; + } + return true; + } + + private void initializeS3Partitions() { + while (shouldWaitForS3Partition() && !Thread.currentThread().isInterrupted()) { + LOG.info("S3 partitions are not ready, waiting for them to be complete before resuming streams."); + try { + Thread.sleep(S3_PARTITIONS_WAIT_TIME_MILLIS); + } catch (final InterruptedException ex) { + LOG.info("The StreamScheduler was interrupted while waiting to retry, stopping processing"); + Thread.currentThread().interrupt(); + break; + } + } + + final List s3Partitions = s3PartitionStatus.getPartitions(); + if (s3Partitions.isEmpty()) { + // This should not happen unless the S3 partition creator failed. + // documentDBAggregateMetrics.getStream5xxErrors().increment(); + throw new IllegalStateException("S3 partitions are not created. Please check the S3 partition creator thread."); + } + streamRecordConverter.setPartitions(s3Partitions); + + } + + public void processStream(final StreamPartition streamPartition) { + // documentDBAggregateMetrics.getStreamApiInvocations().increment(); + + try { + initializeS3Partitions(); + + LOG.info("Starting to watch streams for change events."); + + setCheckpointInformation(streamPartition); + this.streamClient.setStreamPosition(currentCheckpoint.getCommitNum(), currentCheckpoint.getOpNum()); + + streamClient.start(); + + } catch (final InterruptedException e) { + LOG.info("StreamWorker thread got interrupted!"); + } catch (final Exception e) { + LOG.info("Exception encountered with Neptune stream client:", e); + throw e; + } finally { + this.shutdownCleanup(); + } + } + + + private void shutdownCleanup() { + + // Flush remaining records + if (!records.isEmpty()) { + LOG.info("Flushing and checkpointing last processed record batch from the stream before terminating"); + flushToBuffer(); + } + + // Do final checkpoint. + if (!sourceConfig.isAcknowledgments()) { + partitionCheckpoint.checkpoint(currentCheckpoint); + } + + stop(); + + // kill monitoring thread + if (this.streamAcknowledgementManager != null) { + this.streamAcknowledgementManager.shutdown(); + } + + // Stream is invalid, reset the checkpoint before quitting + if (isUnrecoverableError) { + partitionCheckpoint.resetCheckpoint(); + } + partitionCheckpoint.giveUpPartition(); + // stop the checkpointing thread + this.streamCheckpointExecutorService.shutdownNow(); + } + + private void setCheckpointInformation(final StreamPartition streamPartition) { + Optional commitNum = streamPartition.getProgressState().map(StreamProgressState::getCommitNum); + commitNum.ifPresent(currentCheckpoint::setCommitNum); + Optional opNum = streamPartition.getProgressState().map(StreamProgressState::getOpNum); + opNum.ifPresent(currentCheckpoint::setOpNum); + Optional loadedRecords = streamPartition.getProgressState().map(StreamProgressState::getLoadedRecords); + loadedRecords.ifPresent(currentCheckpoint::setRecordCount); + } + + private void flushToBuffer(final List records, final StreamCheckpoint progress) { + final AcknowledgementSet acknowledgementSet = streamAcknowledgementManager + .createAcknowledgementSet(new StreamCheckpoint(progress)) + .orElse(null); + recordBufferWriter.writeToBuffer(acknowledgementSet, records); + successItemsCounter.increment(records.size()); + if (acknowledgementSet != null) { + acknowledgementSet.complete(); + } + } + + private void flushToBuffer() { + LOG.debug("Write to buffer records [{}-{}]", lastLocalCheckpoint.getRecordCount(), currentCheckpoint.getRecordCount()); + flushToBuffer(records, currentCheckpoint); + this.lastLocalCheckpoint = new StreamCheckpoint(this.currentCheckpoint); + lastBufferWriteTime = System.currentTimeMillis(); + records.clear(); + } + + private void bufferWriteAndCheckpointStream() { + long lastCheckpointTime = System.currentTimeMillis(); + while (!Thread.currentThread().isInterrupted() && !stopWorker) { + if (!records.isEmpty() && lastBufferWriteTime < Instant.now().minusMillis(BUFFER_WRITE_TIMEOUT_MILLIS).toEpochMilli()) { + lock.lock(); + LOG.debug("Writing to buffer due to buffer write delay"); + try { + flushToBuffer(); + } catch (final Exception e) { + // this will only happen if writing to buffer gets interrupted from shutdown, + // otherwise it's infinite backoff and retry + LOG.error("Failed to add records to buffer with error", e); + failureItemsCounter.increment(records.size()); + } finally { + lock.unlock(); + } + } + + if (shouldCheckpoint(lastCheckpointTime)) { + try { + lock.lock(); + LOG.debug("Perform regular checkpoint for {}", lastLocalCheckpoint); + partitionCheckpoint.checkpoint(lastLocalCheckpoint); + } catch (Exception e) { + LOG.warn("Exception checkpointing the current state. The stream record processing will start from previous checkpoint.", e); + stop(); + } finally { + lock.unlock(); + } + lastCheckpointTime = System.currentTimeMillis(); + } + + try { + Thread.sleep(BUFFER_WRITE_TIMEOUT_MILLIS); + } catch (InterruptedException ex) { + break; + } + } + LOG.info("Checkpoint monitoring thread interrupted."); + } + + /** + * If End-to-End acknowledgements are not enabled then we checkpoint every {@link #checkPointIntervalInMs} ms. + */ + private boolean shouldCheckpoint(final long lastCheckpointTime) { + return !sourceConfig.isAcknowledgments() && (System.currentTimeMillis() - lastCheckpointTime >= checkPointIntervalInMs); + } + + private boolean shouldFlushRecords(final long recordCount) { + return (recordCount % recordFlushBatchSize == 0) || (System.currentTimeMillis() - lastBufferWriteTime >= bufferWriteIntervalInMs); + } + + void stop() { + stopWorker = true; + } + + @Override + public void onNeptuneStreamEvents(final List streamRecords, final StreamPosition streamPosition) { + for (int i = 0; i < streamRecords.size(); i++) { + if (!streamRecordValidator.isValid(streamRecords.get(i))) { + LOG.debug("Skipping record {}.", i); + continue; + } + final Event event = streamRecordConverter.convert(streamRecords.get(i)); + records.add(event); + // recordBytes.add(bytes); + lock.lock(); + try { + currentCheckpoint.setCommitNum(streamRecords.get(i).getCommitNum()); + currentCheckpoint.setOpNum(streamRecords.get(i).getOpNum()); + currentCheckpoint.incrementRecordCount(); + LOG.info("Process stream record - {} ", currentCheckpoint); + + if (shouldFlushRecords(currentCheckpoint.getRecordCount())) { + flushToBuffer(); + } + } catch (Exception e) { + // this will only happen if writing to buffer gets interrupted from shutdown, + // otherwise it's infinite backoff and retry + LOG.error("Failed to add records to buffer with error", e); + failureItemsCounter.increment(records.size()); + } finally { + lock.unlock(); + } + } + } + + @Override + public boolean onNeptuneStreamException(final Exception exception, final StreamPosition streamPosition) { + if (exception == null || stopWorker) { + return !stopWorker; + } + + if (exception instanceof InvalidParameterException || exception instanceof ExpiredStreamException) { + LOG.warn("Stream is corrupt, stopping the worker and resetting the stream."); + this.isUnrecoverableError = true; + } else { + LOG.info("Error fetching stream data, stopping processing: {}", exception.getMessage()); + } + return false; + } + + @Override + public boolean shouldStopNeptuneStream(final StreamPosition streamPosition) { + return stopWorker; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/model/NeptuneStreamRecord.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/model/NeptuneStreamRecord.java new file mode 100644 index 0000000000..9675f401c3 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/model/NeptuneStreamRecord.java @@ -0,0 +1,76 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream.model; + +import lombok.Builder; +import lombok.Getter; +import org.opensearch.dataprepper.plugins.source.neptune.stream.StreamUtils; +import software.amazon.awssdk.services.neptunedata.model.PropertygraphData; +import software.amazon.awssdk.services.neptunedata.model.PropertygraphRecord; +import software.amazon.awssdk.services.neptunedata.model.SparqlData; +import software.amazon.awssdk.services.neptunedata.model.SparqlRecord; +import software.amazon.awssdk.utils.builder.ToCopyableBuilder; + +import java.io.IOException; +import java.util.Map; + +/** + * Unified DTO for Records fetched from Neptune Streams that supports both RDF (SPARQL) and PropertyGraphs (OC, Gremlin). + * @param Type of the data returned from the SDK currently either {@link SparqlData} or {@link PropertygraphData}. + */ +@Builder +@Getter +public class NeptuneStreamRecord { + private final Long commitTimestampInMillis; + private final Map eventId; + private final T data; + private final String op; + private final String id; + private final Long commitNum; + private final Long opNum; + private final Boolean isLastOp; + + public static NeptuneStreamRecord fromStreamRecord(final T record) { + if (record instanceof SparqlRecord) { + return fromSparqlRecord((SparqlRecord) record); + } else if (record instanceof PropertygraphRecord) { + return fromPropertyGraphRecord((PropertygraphRecord) record); + } + throw new IllegalArgumentException("Unsupported record type: " + record.getClass()); + } + + public static NeptuneStreamRecord fromSparqlRecord(final SparqlRecord sparqlRecord) { + final String id; + try { + id = StreamUtils.getSparqlSubject(sparqlRecord.data().stmt()); + } catch (IOException e) { + throw new RuntimeException("Failed to extract sparql subject."); + } + return NeptuneStreamRecord.builder() + .op(sparqlRecord.op()) + .commitTimestampInMillis(sparqlRecord.commitTimestampInMillis()) + .isLastOp(sparqlRecord.isLastOp()) + .eventId(sparqlRecord.eventId()) + .data(sparqlRecord.data()) + .id(id) + .commitNum(Long.parseLong(sparqlRecord.eventId().get("commitNum"))) + .opNum(Long.parseLong(sparqlRecord.eventId().get("opNum"))) + .build(); + } + + public static NeptuneStreamRecord fromPropertyGraphRecord(final PropertygraphRecord propertygraphRecord) { + return NeptuneStreamRecord + .builder() + .op(propertygraphRecord.op()) + .commitTimestampInMillis(propertygraphRecord.commitTimestampInMillis()) + .isLastOp(propertygraphRecord.isLastOp()) + .data(propertygraphRecord.data()) + .id(propertygraphRecord.data().id()) + .eventId(propertygraphRecord.eventId()) + .commitNum(Long.parseLong(propertygraphRecord.eventId().get("commitNum"))) + .opNum(Long.parseLong(propertygraphRecord.eventId().get("opNum"))) + .build(); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/model/StreamCheckpoint.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/model/StreamCheckpoint.java new file mode 100644 index 0000000000..343fe89097 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/model/StreamCheckpoint.java @@ -0,0 +1,67 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream.model; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; + +import java.util.Objects; + +@AllArgsConstructor +@Getter +public class StreamCheckpoint { + private StreamPosition position; + + @Setter + private long recordCount; + + public long getCommitNum() { + return position.getCommitNum(); + } + + public long getOpNum() { + return position.getOpNum(); + } + + public void setCommitNum(final long commitNum) { + position.setCommitNum(commitNum); + } + + public void setOpNum(final long opNum) { + position.setOpNum(opNum); + } + + public void resetPosition() { + this.position = StreamPosition.empty(); + } + + public void incrementRecordCount() { + recordCount++; + } + + public static StreamCheckpoint emptyProgress() { + return new StreamCheckpoint(new StreamPosition(0L, 0L), 0L); + } + + public StreamCheckpoint(final StreamCheckpoint other) { + this.position = new StreamPosition(other.getCommitNum(), other.getOpNum()); + this.recordCount = other.getRecordCount(); + } + + @Override + public String toString() { + return String.format("Checkpoint [pos=%s, recordCount=%s]", position, recordCount); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StreamCheckpoint that = (StreamCheckpoint) o; + return recordCount == that.recordCount && Objects.equals(position, that.position); + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/model/StreamPosition.java b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/model/StreamPosition.java new file mode 100644 index 0000000000..54f702f68d --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/java/org/opensearch/dataprepper/plugins/source/neptune/stream/model/StreamPosition.java @@ -0,0 +1,39 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream.model; + +import lombok.AllArgsConstructor; +import lombok.Getter; +import lombok.Setter; + +@AllArgsConstructor +@Setter +@Getter +public class StreamPosition { + private long commitNum; + private long opNum; + + @Override + public String toString() { + return String.format("StreamPosition [commitNum=%s, opNum=%s]", commitNum, opNum); + } + + public String asAckString() { + return String.format("%d-%d", getCommitNum(), getOpNum()); + } + + public static StreamPosition empty() { + return new StreamPosition(0L, 0L); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + StreamPosition that = (StreamPosition) o; + return commitNum == that.commitNum && opNum == that.opNum; + } +} diff --git a/data-prepper-plugins/neptune-source/src/main/resources/org/opensearch/dataprepper/transforms/rules/neptune-rule.yaml b/data-prepper-plugins/neptune-source/src/main/resources/org/opensearch/dataprepper/transforms/rules/neptune-rule.yaml new file mode 100644 index 0000000000..e684ecbee7 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/resources/org/opensearch/dataprepper/transforms/rules/neptune-rule.yaml @@ -0,0 +1,4 @@ +plugin_name: "neptune" +apply_when: + - "$..source.neptune" + - "$..source.neptune.s3_bucket" \ No newline at end of file diff --git a/data-prepper-plugins/neptune-source/src/main/resources/org/opensearch/dataprepper/transforms/templates/neptune-template.yaml b/data-prepper-plugins/neptune-source/src/main/resources/org/opensearch/dataprepper/transforms/templates/neptune-template.yaml new file mode 100644 index 0000000000..6a2d188e7f --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/main/resources/org/opensearch/dataprepper/transforms/templates/neptune-template.yaml @@ -0,0 +1,82 @@ +"<>": + workers: "<<$.<>.workers>>" + delay: "<<$.<>.delay>>" + buffer: "<<$.<>.buffer>>" + source: + neptune: "<<$.<>.source.neptune>>" + routes: + - initial_load: 'getMetadata("ingestion_type") == "EXPORT"' + - stream_load: 'getMetadata("ingestion_type") == "STREAM"' + sink: + - s3: + routes: + - initial_load + aws: + region: "<<$.<>.source.neptune.s3_region>>" + sts_role_arn: "<<$.<>.source.neptune.aws.sts_role_arn>>" + sts_external_id: "<<$.<>.source.neptune.aws.sts_external_id>>" + sts_header_overrides: "<<$.<>.source.neptune.aws.sts_header_overrides>>" + bucket: "<<$.<>.source.neptune.s3_bucket>>" + threshold: + event_collect_timeout: "120s" + maximum_size: "2mb" + aggregate_threshold: + maximum_size: "128mb" + flush_capacity_ratio: 0 + object_key: + path_prefix: "${getMetadata(\"s3_partition_key\")}" + codec: + event_json: + default_bucket_owner: "<>.source.neptune.aws.sts_role_arn>>" + - s3: + routes: + - stream_load + aws: + region: "<<$.<>.source.neptune.s3_region>>" + sts_role_arn: "<<$.<>.source.neptune.aws.sts_role_arn>>" + sts_external_id: "<<$.<>.source.neptune.aws.sts_external_id>>" + sts_header_overrides: "<<$.<>.source.neptune.aws.sts_header_overrides>>" + bucket: "<<$.<>.source.neptune.s3_bucket>>" + threshold: + event_collect_timeout: "15s" + maximum_size: "1mb" + aggregate_threshold: + maximum_size: "128mb" + flush_capacity_ratio: 0 + object_key: + path_prefix: "${getMetadata(\"s3_partition_key\")}" + codec: + event_json: + default_bucket_owner: "<>.source.neptune.aws.sts_role_arn>>" +"<>-s3": + workers: "<<$.<>.workers>>" + delay: "<<$.<>.delay>>" + buffer: "<<$.<>.buffer>>" + source: + s3: + codec: + event_json: + compression: "none" + aws: + region: "<<$.<>.source.neptune.s3_region>>" + sts_role_arn: "<<$.<>.source.neptune.aws.sts_role_arn>>" + sts_external_id: "<<$.<>.source.neptune.aws.sts_external_id>>" + sts_header_overrides: "<<$.<>.source.neptune.aws.sts_header_overrides>>" + acknowledgments: true + delete_s3_objects_on_read: true + disable_s3_metadata_in_event: true + scan: + acknowledgment_timeout: "PT10M" + folder_partitions: + depth: "<>.source.neptune.s3_prefix>>" + max_objects_per_ownership: 50 + buckets: + - bucket: + name: "<<$.<>.source.neptune.s3_bucket>>" + filter: + include_prefix: [ "<>.source.neptune.s3_prefix>>" ] + scheduling: + interval: "20s" + processor: "<<$.<>.processor>>" + sink: "<<$.<>.sink>>" + routes: "<<$.<>.routes>>" # In placeholder, routes or route (defined as alias) will be transformed to route in json as route will be primarily picked in pipelineModel. \ No newline at end of file diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/buffer/RecordBufferWriterTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/buffer/RecordBufferWriterTest.java new file mode 100644 index 0000000000..01d092a4b2 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/buffer/RecordBufferWriterTest.java @@ -0,0 +1,136 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.buffer; + +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.buffer.common.BufferAccumulator; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; + +import java.util.ArrayList; +import java.util.List; +import java.util.Random; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyDouble; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.opensearch.dataprepper.plugins.source.neptune.buffer.RecordBufferWriter.RECORDS_PROCESSED_COUNT; +import static org.opensearch.dataprepper.plugins.source.neptune.buffer.RecordBufferWriter.RECORDS_PROCESSING_ERROR_COUNT; + +@ExtendWith(MockitoExtension.class) +class RecordBufferWriterTest { + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private BufferAccumulator> bufferAccumulator; + + @Mock + private AcknowledgementSet acknowledgementSet; + + @Mock + private Counter exportRecordSuccess; + + @Mock + private Counter exportRecordErrors; + + @Mock + private Event event; + + final Random random = new Random(); + + + @BeforeEach + void setup() { + given(pluginMetrics.counter(RECORDS_PROCESSED_COUNT)).willReturn(exportRecordSuccess); + given(pluginMetrics.counter(RECORDS_PROCESSING_ERROR_COUNT)).willReturn(exportRecordErrors); + } + + private List generateData(int count) { + final List result = new ArrayList<>(); + for (int i = 0; i < count; i++) { + result.add(mock(Event.class)); + } + return result; + } + + @Test + void test_writeToBuffer() throws Exception { + int numberOfRecords = random.nextInt(10); + + final List data = generateData(numberOfRecords); + final RecordBufferWriter recordBufferWriter = RecordBufferWriter.create(bufferAccumulator, + pluginMetrics); + + recordBufferWriter.writeToBuffer(null, data); + verify(bufferAccumulator, times(numberOfRecords)).add(any(Record.class)); + verify(exportRecordSuccess).increment(anyDouble()); + + verifyNoInteractions(exportRecordErrors); + } + + @Test + void test_writeSingleRecordToBuffer() throws Exception { + final ArgumentCaptor> recordArgumentCaptor = ArgumentCaptor.forClass(Record.class); + + final RecordBufferWriter recordBufferWriter = RecordBufferWriter.create(bufferAccumulator, + pluginMetrics); + + recordBufferWriter.writeToBuffer(null, List.of(event)); + verify(bufferAccumulator).add(recordArgumentCaptor.capture()); + assertThat(recordArgumentCaptor.getValue().getData(), equalTo(event)); + + verify(bufferAccumulator).flush(); + verifyNoInteractions(exportRecordErrors); + } + + @Test + void test_writeSingleRecordToBufferWithAcknowledgementSet() throws Exception { + final ArgumentCaptor> recordArgumentCaptor = ArgumentCaptor.forClass(Record.class); + + final RecordBufferWriter recordBufferWriter = RecordBufferWriter.create(bufferAccumulator, + pluginMetrics); + recordBufferWriter.writeToBuffer(acknowledgementSet, List.of(event)); + verify(bufferAccumulator).add(recordArgumentCaptor.capture()); + verify(acknowledgementSet).add(event); + assertThat(recordArgumentCaptor.getValue().getData(), equalTo(event)); + + verify(bufferAccumulator).flush(); + verifyNoInteractions(exportRecordErrors); + } + + @Test + void test_writeSingleRecordFlushException() throws Exception { + final ArgumentCaptor> recordArgumentCaptor = ArgumentCaptor.forClass(Record.class); + + final RecordBufferWriter recordBufferWriter = RecordBufferWriter.create(bufferAccumulator, + pluginMetrics); + doThrow(RuntimeException.class).when(bufferAccumulator).flush(); + + recordBufferWriter.writeToBuffer(acknowledgementSet, List.of(event)); + verify(bufferAccumulator).add(recordArgumentCaptor.capture()); + verify(acknowledgementSet).add(event); + assertThat(recordArgumentCaptor.getValue().getData(), equalTo(event)); + + verify(bufferAccumulator).flush(); + verify(exportRecordErrors).increment(1); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/converter/StreamRecordConverterTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/converter/StreamRecordConverterTest.java new file mode 100644 index 0000000000..8779660f84 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/converter/StreamRecordConverterTest.java @@ -0,0 +1,57 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.converter; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.event.JacksonEvent; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.NeptuneStreamRecord; +import software.amazon.awssdk.core.document.Document; +import software.amazon.awssdk.services.neptunedata.model.PropertygraphData; + +import java.util.List; +import java.util.Map; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +@ExtendWith(MockitoExtension.class) +class StreamRecordConverterTest { + private NeptuneStreamRecord record; + + @BeforeEach + void setup() { + PropertygraphData data = PropertygraphData.builder() + .key("label") + .value(Document.fromMap(Map.of("value", Document.fromString("a")))) + .type("v") + .id("i") + .build(); + record = NeptuneStreamRecord.builder().data(data).op("ADD").id("i").commitTimestampInMillis(1L).build(); + } + + @Test + void convert() { + final StreamRecordConverter recordConverter = new StreamRecordConverter("a"); + recordConverter.setPartitions(List.of("a")); + + final JacksonEvent event = (JacksonEvent) recordConverter.convert(record); + assertThat(event.getMetadata(), notNullValue()); + + assertThat(event.getMetadata().getAttribute(MetadataKeyAttributes.ID_METADATA_ATTRIBUTE), equalTo("i")); + assertThat(event.getMetadata().getAttribute(MetadataKeyAttributes.NEPTUNE_COMMIT_TIMESTAMP_METADATA_ATTRIBUTE), equalTo(1L)); + assertThat(event.getMetadata().getAttribute(MetadataKeyAttributes.EVENT_VERSION_FROM_TIMESTAMP), equalTo(1L)); + assertThat(event.getMetadata().getAttribute(MetadataKeyAttributes.NEPTUNE_STREAM_OP_NAME_METADATA_ATTRIBUTE), equalTo("ADD")); + assertThat(event.getMetadata().getAttribute(MetadataKeyAttributes.EVENT_NAME_BULK_ACTION_METADATA_ATTRIBUTE), equalTo("upsert")); + assertThat(event.getMetadata().getAttribute(MetadataKeyAttributes.INGESTION_EVENT_TYPE_ATTRIBUTE), equalTo("STREAM")); + assertThat(event.getMetadata().getAttribute(MetadataKeyAttributes.EVENT_S3_PARTITION_KEY), equalTo("a/a")); + assertThat(event.getEventHandle(), notNullValue()); + assertThat(event.getEventHandle().getExternalOriginationTime(), nullValue()); + } +} \ No newline at end of file diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/PartitionFactoryTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/PartitionFactoryTest.java new file mode 100644 index 0000000000..8317cfc5f6 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/coordination/PartitionFactoryTest.java @@ -0,0 +1,140 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.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.neptune.coordination.partition.DataQueryPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.LeaderPartition; +import org.opensearch.dataprepper.plugins.source.neptune.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 collectionName = UUID.randomUUID().toString(); + private final String databaseName = UUID.randomUUID().toString(); + private final String resumeToken = UUID.randomUUID().toString(); + private final String collection = collectionName + "." + databaseName; + private final String dataQuery = collection + "|" + UUID.randomUUID(); + + private final long exportTimeMills = 1695021857760L; + private final Instant exportTime = Instant.ofEpochMilli(exportTimeMills); + + private final Random random = new Random(); + + private final int partitionSize = random.nextInt(10000); + + private final long totalRecords = random.nextLong(); + private final long loadedRecords = random.nextLong(); + + + @Test + void testCreateStreamPartition() { + String sourceId = sourceIdentifier + "|" + StreamPartition.PARTITION_TYPE; + when(sourcePartitionStoreItem.getSourceIdentifier()).thenReturn(sourceId); + when(sourcePartitionStoreItem.getSourcePartitionKey()).thenReturn(collection); + + String state = "{\"startTime\":" + exportTimeMills + ",\"commitNum\":1,\"opNum\":1,\"loadedRecords\":" + loadedRecords + ",\"lastUpdateTimestamp\":" + exportTimeMills + ",\"waitForExport\": true}"; + when(sourcePartitionStoreItem.getPartitionProgressState()).thenReturn(state); + when(sourcePartitionStoreItem.getPartitionProgressState()).thenReturn(state); + + + PartitionFactory factory = new PartitionFactory(); + final StreamPartition streamPartition = (StreamPartition) factory.apply(sourcePartitionStoreItem); + assertThat(streamPartition, notNullValue()); + assertThat(streamPartition.getPartitionType(), equalTo(StreamPartition.PARTITION_TYPE)); + assertThat(streamPartition.getPartitionKey(), equalTo("neptune")); + + assertThat(streamPartition.getProgressState().isPresent(), equalTo(true)); + assertThat(streamPartition.getProgressState().get().getStartTime(), equalTo(exportTimeMills)); + assertThat(streamPartition.getProgressState().get().getCommitNum(), equalTo(1L)); + assertThat(streamPartition.getProgressState().get().getOpNum(), equalTo(1L)); + assertThat(streamPartition.getProgressState().get().getLoadedRecords(), equalTo(loadedRecords)); + assertThat(streamPartition.getProgressState().get().getLastUpdateTimestamp(), equalTo(exportTimeMills)); + assertThat(streamPartition.getProgressState().get().isWaitForExport(), equalTo(true)); + } + + + @Test + void testCreateDataQueryPartition() { + String sourceId = sourceIdentifier + "|" + DataQueryPartition.PARTITION_TYPE; + when(sourcePartitionStoreItem.getSourceIdentifier()).thenReturn(sourceId); + when(sourcePartitionStoreItem.getSourcePartitionKey()).thenReturn(dataQuery); + + String state = "{\"executedQueries\":" + totalRecords + ",\"loadedRecords\":" + loadedRecords + ",\"exportStartTime\":" + exportTimeMills + "}"; + when(sourcePartitionStoreItem.getPartitionProgressState()).thenReturn(state); + + + PartitionFactory factory = new PartitionFactory(); + final DataQueryPartition dataQueryPartition = (DataQueryPartition) factory.apply(sourcePartitionStoreItem); + assertThat(dataQueryPartition, notNullValue()); + assertThat(dataQueryPartition.getQuery(), equalTo(dataQuery)); + assertThat(dataQueryPartition.getCollection(), equalTo(collection)); + assertThat(dataQueryPartition.getPartitionType(), equalTo(DataQueryPartition.PARTITION_TYPE)); + assertThat(dataQueryPartition.getPartitionKey(), equalTo(dataQuery)); + + assertThat(dataQueryPartition.getProgressState().isPresent(), equalTo(true)); + assertThat(dataQueryPartition.getProgressState().get().getExecutedQueries(), equalTo(totalRecords)); + assertThat(dataQueryPartition.getProgressState().get().getLoadedRecords(), equalTo(loadedRecords)); + assertThat(dataQueryPartition.getProgressState().get().getStartTime(), equalTo(exportTimeMills)); + + } + + @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(); + GlobalState globalState = (GlobalState) factory.apply(sourcePartitionStoreItem); + assertThat(globalState, notNullValue()); + 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 testCreateLeaderPartition() { + String sourceId = sourceIdentifier + "|" + LeaderPartition.PARTITION_TYPE; + when(sourcePartitionStoreItem.getSourceIdentifier()).thenReturn(sourceId); + + PartitionFactory factory = new PartitionFactory(); + LeaderPartition leaderPartition = (LeaderPartition) factory.apply(sourcePartitionStoreItem); + assertThat(leaderPartition, notNullValue()); + assertThat(leaderPartition.getPartitionKey(), equalTo("GLOBAL")); + assertThat(leaderPartition.getPartitionType(), equalTo(LeaderPartition.PARTITION_TYPE)); + } +} diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/leader/LeaderSchedulerTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/leader/LeaderSchedulerTest.java new file mode 100644 index 0000000000..0bb728c7f5 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/leader/LeaderSchedulerTest.java @@ -0,0 +1,209 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.leader; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.Captor; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.LeaderPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.S3FolderPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.StreamPartition; +import software.amazon.awssdk.regions.Region; + +import java.time.Duration; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import static org.awaitility.Awaitility.await; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.atLeast; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.opensearch.dataprepper.plugins.source.neptune.leader.LeaderScheduler.DEFAULT_EXTEND_LEASE_MINUTES; + +@ExtendWith(MockitoExtension.class) +public class LeaderSchedulerTest { + private static final String TEST_COLLECTION = UUID.randomUUID().toString(); + private static final String TEST_S3_BUCKET_NAME = UUID.randomUUID().toString(); + private static final String TEST_S3_PATH_PREFIX = UUID.randomUUID().toString(); + private static final String TEST_S3_REGION = Region.US_EAST_1.toString(); + + @Mock + private EnhancedSourceCoordinator coordinator; + + @Mock + private NeptuneSourceConfig sourceConfig; + + @Captor + private ArgumentCaptor enhancedSourcePartitionArgumentCaptor; + + private LeaderScheduler leaderScheduler; + private LeaderPartition leaderPartition; + + @Test + void test_non_leader_run() { + leaderScheduler = new LeaderScheduler(coordinator, sourceConfig, TEST_S3_PATH_PREFIX, Duration.ofMillis(100)); + given(coordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)).willReturn(Optional.empty()); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> leaderScheduler.run()); + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verifyNoInteractions(sourceConfig)); + executorService.shutdownNow(); + } + + @Test + void test_should_init() { + leaderScheduler = new LeaderScheduler(coordinator, sourceConfig, TEST_S3_PATH_PREFIX, Duration.ofMillis(100)); + leaderPartition = new LeaderPartition(); + given(coordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)).willReturn(Optional.of(leaderPartition)); + given(sourceConfig.getS3Bucket()).willReturn(TEST_S3_BUCKET_NAME); + given(sourceConfig.getS3Region()).willReturn(TEST_S3_REGION); + final int partitionCount = Math.abs(new Random().nextInt(10)); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + final Future future = executorService.submit(() -> leaderScheduler.run()); + + // Acquire the init partition + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(coordinator, atLeast(1)).acquireAvailablePartition(eq(LeaderPartition.PARTITION_TYPE))); + + future.cancel(true); + + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(coordinator).giveUpPartition(leaderPartition)); + + // Should create 1 stream partitions + 1 S3 partition + 2 global table state + verify(coordinator, times(2)).createPartition( + enhancedSourcePartitionArgumentCaptor.capture()); + verify(coordinator, atLeast(1)).saveProgressStateForPartition(leaderPartition, Duration.ofMinutes(DEFAULT_EXTEND_LEASE_MINUTES)); + + assertThat(leaderPartition.getProgressState().get().isInitialized(), equalTo(true)); + final List allEnhancedSourcePartitions = + enhancedSourcePartitionArgumentCaptor.getAllValues(); + assertThat(allEnhancedSourcePartitions.get(0), instanceOf(GlobalState.class)); + assertThat(allEnhancedSourcePartitions.get(1), instanceOf(S3FolderPartition.class)); + final GlobalState exportGlobalState = (GlobalState) allEnhancedSourcePartitions.get(0); + assertThat(exportGlobalState.getPartitionKey(), equalTo("neptune")); + assertThat(exportGlobalState.getProgressState(), is(Optional.empty())); + assertThat(exportGlobalState.getSourcePartitionStoreItem(), nullValue()); + final S3FolderPartition s3FolderPartition = (S3FolderPartition) allEnhancedSourcePartitions.get(1); + final String[] partitionKeys = s3FolderPartition.getPartitionKey().split("\\|"); + assertThat(partitionKeys[0], is(TEST_S3_BUCKET_NAME)); + assertThat(partitionKeys[1], is(TEST_S3_PATH_PREFIX + "neptune")); + assertThat(partitionKeys[2], is(String.valueOf(100))); + assertThat(partitionKeys[3], is(TEST_S3_REGION)); + executorService.shutdownNow(); + } + + @Test + void test_should_init_stream() { + leaderScheduler = new LeaderScheduler(coordinator, sourceConfig, TEST_S3_PATH_PREFIX, Duration.ofMillis(100)); + leaderPartition = new LeaderPartition(); + given(coordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)).willReturn(Optional.of(leaderPartition)); + given(sourceConfig.getS3Bucket()).willReturn(TEST_S3_BUCKET_NAME); + given(sourceConfig.getS3Region()).willReturn(TEST_S3_REGION); + given(sourceConfig.isStream()).willReturn(true); + final int partitionCount = Math.abs(new Random().nextInt(10)); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + final Future future = executorService.submit(() -> leaderScheduler.run()); + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(coordinator, atLeast(1)).acquireAvailablePartition(eq(LeaderPartition.PARTITION_TYPE))); + + future.cancel(true); + + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(coordinator).giveUpPartition(leaderPartition)); + + // Should create 1 stream partitions + 1 S3 partition + 1 global table state + verify(coordinator, times(3)).createPartition( + enhancedSourcePartitionArgumentCaptor.capture()); + verify(coordinator, atLeast(1)).saveProgressStateForPartition(leaderPartition, Duration.ofMinutes(DEFAULT_EXTEND_LEASE_MINUTES)); + + assertThat(leaderPartition.getProgressState().get().isInitialized(), equalTo(true)); + final List allEnhancedSourcePartitions = + enhancedSourcePartitionArgumentCaptor.getAllValues(); + assertThat(allEnhancedSourcePartitions.get(0), instanceOf(GlobalState.class)); + assertThat(allEnhancedSourcePartitions.get(1), instanceOf(S3FolderPartition.class)); + final S3FolderPartition s3FolderPartition = (S3FolderPartition) allEnhancedSourcePartitions.get(1); + final String[] partitionKeys = s3FolderPartition.getPartitionKey().split("\\|"); + assertThat(partitionKeys[0], is(TEST_S3_BUCKET_NAME)); + assertThat(partitionKeys[1], is(TEST_S3_PATH_PREFIX + "neptune")); + assertThat(partitionKeys[2], is(String.valueOf(100))); + assertThat(partitionKeys[3], is(TEST_S3_REGION)); + assertThat(allEnhancedSourcePartitions.get(2), instanceOf(StreamPartition.class)); + executorService.shutdownNow(); + } + + @Test + void test_shouldInitStream_withEmptyS3PathPrefix() { + leaderScheduler = new LeaderScheduler(coordinator, sourceConfig, "", Duration.ofMillis(100)); + leaderPartition = new LeaderPartition(); + given(coordinator.acquireAvailablePartition(LeaderPartition.PARTITION_TYPE)).willReturn(Optional.of(leaderPartition)); + given(sourceConfig.getS3Bucket()).willReturn(TEST_S3_BUCKET_NAME); + given(sourceConfig.getS3Region()).willReturn(TEST_S3_REGION); + final int partitionCount = Math.abs(new Random().nextInt(10)); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + final Future future = executorService.submit(() -> leaderScheduler.run()); + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(coordinator, atLeast(1)).acquireAvailablePartition(eq(LeaderPartition.PARTITION_TYPE))); + + future.cancel(true); + + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(coordinator).giveUpPartition(leaderPartition)); + + // Should create 1 stream partitions + 1 S3 partition + 1 global table state + verify(coordinator, times(2)).createPartition( + enhancedSourcePartitionArgumentCaptor.capture()); + verify(coordinator, atLeast(1)).saveProgressStateForPartition(leaderPartition, Duration.ofMinutes(DEFAULT_EXTEND_LEASE_MINUTES)); + + assertThat(leaderPartition.getProgressState().get().isInitialized(), equalTo(true)); + final List allEnhancedSourcePartitions = + enhancedSourcePartitionArgumentCaptor.getAllValues(); + assertThat(allEnhancedSourcePartitions.get(0), instanceOf(GlobalState.class)); + assertThat(allEnhancedSourcePartitions.get(1), instanceOf(S3FolderPartition.class)); + final S3FolderPartition s3FolderPartition = (S3FolderPartition) allEnhancedSourcePartitions.get(1); + final String[] partitionKeys = s3FolderPartition.getPartitionKey().split("\\|"); + assertThat(partitionKeys[0], is(TEST_S3_BUCKET_NAME)); + assertThat(partitionKeys[1], is("neptune")); + assertThat(partitionKeys[2], is(String.valueOf(100))); + assertThat(partitionKeys[3], is(TEST_S3_REGION)); + executorService.shutdownNow(); + } + + @Test + void test_shouldInitStream_withNullS3PathPrefix() { + assertThrows(IllegalArgumentException.class, () -> new LeaderScheduler(coordinator, sourceConfig, null, Duration.ofMillis(100))); + } +} diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3FolderPartitionCoordinatorTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3FolderPartitionCoordinatorTest.java new file mode 100644 index 0000000000..193cc323a2 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3FolderPartitionCoordinatorTest.java @@ -0,0 +1,42 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.s3partition; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; + +@ExtendWith(MockitoExtension.class) +public class S3FolderPartitionCoordinatorTest { + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + + @InjectMocks + private S3FolderPartitionCoordinator s3FolderPartitionCoordinator; + + @Test + public void getGlobalS3FolderCreationStatus_empty() { +// final String collection = UUID.randomUUID().toString(); +// when(sourceCoordinator.getPartition(S3PartitionCreatorScheduler.S3_FOLDER_PREFIX)).thenReturn(Optional.empty()); +// Optional partitionStatus = s3FolderPartitionCoordinator.getGlobalS3FolderCreationStatus(collection); +// assertThat(partitionStatus.isEmpty(), is(true)); + } + + @Test + public void getGlobalS3FolderCreationStatus_nonEmpty() { +// final String collection = UUID.randomUUID().toString(); +// final List partitions = List.of(UUID.randomUUID().toString(), UUID.randomUUID().toString()); +// final GlobalState globalState = mock(GlobalState.class); +// final Map props = Map.of("partitions", partitions); +// when(globalState.getProgressState()).thenReturn(Optional.of(props)); +// when(sourceCoordinator.getPartition(S3PartitionCreatorScheduler.S3_FOLDER_PREFIX + collection)).thenReturn(Optional.of(globalState)); +// Optional partitionStatus = s3FolderPartitionCoordinator.getGlobalS3FolderCreationStatus(collection); +// assertThat(partitionStatus.isEmpty(), is(false)); +// assertThat(partitionStatus.get().getPartitions(), is(partitions)); + } +} diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreatorSchedulerTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreatorSchedulerTest.java new file mode 100644 index 0000000000..2d7d32329a --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreatorSchedulerTest.java @@ -0,0 +1,77 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.s3partition; + +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.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourcePartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.GlobalState; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.S3FolderPartition; + +import java.time.Duration; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static org.awaitility.Awaitility.await; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasKey; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.opensearch.dataprepper.plugins.source.neptune.s3partition.S3PartitionCreatorScheduler.S3_FOLDER_PREFIX; + +@ExtendWith(MockitoExtension.class) +public class S3PartitionCreatorSchedulerTest { + @Mock + private EnhancedSourceCoordinator coordinator; + private S3PartitionCreatorScheduler s3PartitionCreatorScheduler; + + @BeforeEach + public void setup() { + s3PartitionCreatorScheduler = new S3PartitionCreatorScheduler(coordinator); + } + + @Test + void test_S3FolderPartition_empty() { + given(coordinator.acquireAvailablePartition(S3FolderPartition.PARTITION_TYPE)).willReturn(Optional.empty()); + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> s3PartitionCreatorScheduler.run()); + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(coordinator, never()).completePartition(any(EnhancedSourcePartition.class))); + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(coordinator, never()).createPartition(any(EnhancedSourcePartition.class))); + executorService.shutdownNow(); + } + + @Test + void test_S3FolderPartition_exist() { + final S3FolderPartition s3FolderPartition = mock(S3FolderPartition.class); + given(s3FolderPartition.getPartitionCount()).willReturn(Math.abs(new Random().nextInt(100))); + given(coordinator.acquireAvailablePartition(S3FolderPartition.PARTITION_TYPE)).willReturn(Optional.of(s3FolderPartition)); + s3PartitionCreatorScheduler.run(); + verify(coordinator).completePartition(s3FolderPartition); + final ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(GlobalState.class); + verify(coordinator).createPartition(argumentCaptor.capture()); + final GlobalState globalState = argumentCaptor.getValue(); + assertThat(globalState.getPartitionKey(), is(S3_FOLDER_PREFIX)); + assertThat(globalState.getProgressState().get(), hasKey("partitions")); + final List partitions = (List) globalState.getProgressState().get().get("partitions"); + assertThat(partitions, hasSize(s3FolderPartition.getPartitionCount())); + } +} diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreatorTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreatorTest.java new file mode 100644 index 0000000000..ab20b2fc5b --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/s3partition/S3PartitionCreatorTest.java @@ -0,0 +1,24 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.s3partition; + +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.Random; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.hasSize; + +public class S3PartitionCreatorTest { + + @Test + public void createPartitionTest() { + final int partitionCount = Math.abs(new Random().nextInt(1000)); + final S3PartitionCreator s3PartitionCreator = new S3PartitionCreator(partitionCount); + final List partitions = s3PartitionCreator.createPartition(); + assertThat(partitions, hasSize(partitionCount)); + } +} diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/DataStreamPartitionCheckpointTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/DataStreamPartitionCheckpointTest.java new file mode 100644 index 0000000000..0fcacf6dfc --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/DataStreamPartitionCheckpointTest.java @@ -0,0 +1,83 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream; + +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; +import org.mockito.ArgumentCaptor; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.StreamProgressState; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamCheckpoint; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamPosition; + +import java.time.Duration; +import java.util.Optional; +import java.util.Random; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.CoreMatchers.equalToObject; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.neptune.stream.DataStreamPartitionCheckpoint.CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE; + +@ExtendWith(MockitoExtension.class) +public class DataStreamPartitionCheckpointTest { + @Mock + private EnhancedSourceCoordinator enhancedSourceCoordinator; + + @Mock + private StreamPartition streamPartition; + + @Mock + private StreamProgressState streamProgressState; + + @InjectMocks + private DataStreamPartitionCheckpoint dataStreamPartitionCheckpoint; + + @Test + public void checkpoint_success() { + final long commitNum = new Random().nextLong(); + final long opNum = new Random().nextLong(); + final long recordNumber = new Random().nextLong(); + when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); + + final StreamCheckpoint checkpoint = new StreamCheckpoint(new StreamPosition(commitNum, opNum), recordNumber); + dataStreamPartitionCheckpoint.checkpoint(checkpoint); + + verify(enhancedSourceCoordinator).saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + verify(streamProgressState).updateFromCheckpoint(checkpoint); + } + + @Test + public void updateStreamPartitionForAcknowledgmentWait_success() { + final int minutes = new Random().nextInt(); + final Duration duration = Duration.ofMinutes(minutes); + dataStreamPartitionCheckpoint.updateStreamPartitionForAcknowledgmentWait(duration); + verify(enhancedSourceCoordinator).saveProgressStateForPartition(streamPartition, duration); + } + + @Test + public void resetCheckpoint_success() { + when(streamPartition.getProgressState()).thenReturn(Optional.of(streamProgressState)); + dataStreamPartitionCheckpoint.resetCheckpoint(); + verify(enhancedSourceCoordinator).giveUpPartition(streamPartition); + + final ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(StreamCheckpoint.class); + verify(streamProgressState).updateFromCheckpoint(argumentCaptor.capture()); + + final StreamCheckpoint checkpoint = argumentCaptor.getValue(); + assertThat(checkpoint, equalToObject(StreamCheckpoint.emptyProgress())); + } + + @Test + public void extendLease_success() { + dataStreamPartitionCheckpoint.extendLease(); + verify(enhancedSourceCoordinator).saveProgressStateForPartition(streamPartition, CHECKPOINT_OWNERSHIP_TIMEOUT_INCREASE); + } +} diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamAcknowledgementManagerTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamAcknowledgementManagerTest.java new file mode 100644 index 0000000000..96b9353b2a --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamAcknowledgementManagerTest.java @@ -0,0 +1,181 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.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 org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSet; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamCheckpoint; +import org.opensearch.dataprepper.plugins.source.neptune.stream.model.StreamPosition; + +import java.time.Duration; +import java.util.List; +import java.util.Optional; +import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Consumer; + +import static org.awaitility.Awaitility.await; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.lenient; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +@ExtendWith(MockitoExtension.class) +public class StreamAcknowledgementManagerTest { + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + @Mock + private DataStreamPartitionCheckpoint partitionCheckpoint; + @Mock + private Duration timeout; + @Mock + private AcknowledgementSet acknowledgementSet; + @Mock + private Consumer stopWorkerConsumer; + private StreamAcknowledgementManager streamAckManager; + + @BeforeEach + public void setup() { + streamAckManager = new StreamAcknowledgementManager(acknowledgementSetManager, partitionCheckpoint, timeout, 0, 0); + } + + @Test + public void createAcknowledgementSet_disabled_emptyAckSet() { + final StreamCheckpoint checkpoint = createCheckpoint(new Random().nextLong(), new Random().nextLong(), new Random().nextLong()); + final Optional ackSet = streamAckManager.createAcknowledgementSet(checkpoint); + assertThat(ackSet.isEmpty(), is(true)); + } + + private StreamCheckpoint createCheckpoint(final long commNum, final long opNum, final long numOfRecords) { + return new StreamCheckpoint(new StreamPosition(commNum, opNum), numOfRecords); + } + + @Test + public void createAcknowledgementSet_enabled_ackSetWithAck() { + lenient().when(timeout.getSeconds()).thenReturn(10_000L); + streamAckManager = new StreamAcknowledgementManager(acknowledgementSetManager, partitionCheckpoint, timeout, 0, 0); + streamAckManager.init(stopWorkerConsumer); + final long recordCount = new Random().nextLong(); + final StreamCheckpoint checkpoint = createCheckpoint(1L, 1L, recordCount); + + when(acknowledgementSetManager.create(any(Consumer.class), eq(timeout))).thenReturn(acknowledgementSet); + final Optional ackSet = streamAckManager.createAcknowledgementSet(checkpoint); + assertThat(ackSet.isEmpty(), is(false)); + assertThat(ackSet.get(), is(acknowledgementSet)); + assertThat(streamAckManager.getCheckpoints().peek().getCheckpoint(), is(checkpoint)); + + final ArgumentCaptor> argumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSetManager).create(argumentCaptor.capture(), eq(timeout)); + final Consumer consumer = argumentCaptor.getValue(); + consumer.accept(true); + + final ConcurrentHashMap ackStatus = streamAckManager.getAcknowledgementStatus(); + final CheckpointStatus ackCheckpointStatus = ackStatus.get("1-1"); + assertThat(ackCheckpointStatus.isPositiveAcknowledgement(), is(true)); + await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> verify(partitionCheckpoint).checkpoint(checkpoint)); + assertThat(streamAckManager.getCheckpoints().peek(), is(nullValue())); + } + + @Test + public void createAcknowledgementSet_enabled_multipleAckSetWithAck() { + lenient().when(timeout.getSeconds()).thenReturn(10_000L); + streamAckManager = new StreamAcknowledgementManager(acknowledgementSetManager, partitionCheckpoint, timeout, 0, 0); + streamAckManager.init(stopWorkerConsumer); + + final long recordCount1 = new Random().nextLong(); + final StreamCheckpoint checkpoint1 = createCheckpoint(1L, 1L, recordCount1); + when(acknowledgementSetManager.create(any(Consumer.class), eq(timeout))).thenReturn(acknowledgementSet); + Optional ackSet = streamAckManager.createAcknowledgementSet(checkpoint1); + assertThat(ackSet.isEmpty(), is(false)); + assertThat(ackSet.get(), is(acknowledgementSet)); + assertThat(streamAckManager.getCheckpoints().peek().getCheckpoint(), is(checkpoint1)); + + final long recordCount2 = new Random().nextLong(); + when(acknowledgementSetManager.create(any(Consumer.class), eq(timeout))).thenReturn(acknowledgementSet); + final StreamCheckpoint checkpoint2 = createCheckpoint(1L, 2L, recordCount2); + ackSet = streamAckManager.createAcknowledgementSet(checkpoint2); + assertThat(ackSet.isEmpty(), is(false)); + assertThat(ackSet.get(), is(acknowledgementSet)); + assertThat(streamAckManager.getCheckpoints().peek().getCheckpoint().getRecordCount(), is(recordCount1)); + + ArgumentCaptor> argumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSetManager, times(2)).create(argumentCaptor.capture(), eq(timeout)); + List> consumers = argumentCaptor.getAllValues(); + consumers.get(0).accept(true); + consumers.get(1).accept(true); + ConcurrentHashMap ackStatus = streamAckManager.getAcknowledgementStatus(); + CheckpointStatus ackCheckpointStatus = ackStatus.get("1-2"); + assertThat(ackCheckpointStatus.isPositiveAcknowledgement(), is(true)); + await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> verify(partitionCheckpoint).checkpoint(checkpoint2)); + assertThat(streamAckManager.getCheckpoints().peek(), is(nullValue())); + } + + @Test + public void createAcknowledgementSet_enabled_multipleAckSetWithAckFailure() { + streamAckManager.init(stopWorkerConsumer); + + final long recordCount1 = new Random().nextLong(); + final StreamCheckpoint checkpoint1 = createCheckpoint(1L, 1L, recordCount1); + when(acknowledgementSetManager.create(any(Consumer.class), eq(timeout))).thenReturn(acknowledgementSet); + Optional ackSet = streamAckManager.createAcknowledgementSet(checkpoint1); + assertThat(ackSet.isEmpty(), is(false)); + assertThat(ackSet.get(), is(acknowledgementSet)); + assertThat(streamAckManager.getCheckpoints().peek().getCheckpoint().getRecordCount(), is(recordCount1)); + + final long recordCount2 = new Random().nextLong(); + final StreamCheckpoint checkpoint2 = createCheckpoint(1L, 2L, recordCount2); + when(acknowledgementSetManager.create(any(Consumer.class), eq(timeout))).thenReturn(acknowledgementSet); + ackSet = streamAckManager.createAcknowledgementSet(checkpoint2); + assertThat(ackSet.isEmpty(), is(false)); + assertThat(ackSet.get(), is(acknowledgementSet)); + assertThat(streamAckManager.getCheckpoints().peek().getCheckpoint().getRecordCount(), is(recordCount1)); + ArgumentCaptor> argumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSetManager, times(2)).create(argumentCaptor.capture(), eq(timeout)); + List> consumers = argumentCaptor.getAllValues(); + consumers.get(0).accept(false); + consumers.get(1).accept(true); + ConcurrentHashMap ackStatus = streamAckManager.getAcknowledgementStatus(); + CheckpointStatus ackCheckpointStatus = ackStatus.get("1-2"); + assertThat(ackCheckpointStatus.isPositiveAcknowledgement(), is(true)); + await() + .atMost(Duration.ofSeconds(10)).untilAsserted(() -> + verify(partitionCheckpoint).giveUpPartition()); + assertThat(streamAckManager.getCheckpoints().peek().getCheckpoint().getRecordCount(), is(recordCount1)); + verify(stopWorkerConsumer).accept(null); + } + + @Test + public void createAcknowledgementSet_enabled_ackSetWithNoAck() { + streamAckManager.init(stopWorkerConsumer); + final long recordCount = new Random().nextLong(); + final StreamCheckpoint checkpoint = createCheckpoint(1L, 1L, recordCount); + + when(acknowledgementSetManager.create(any(Consumer.class), eq(timeout))).thenReturn(acknowledgementSet); + final Optional ackSet = streamAckManager.createAcknowledgementSet(checkpoint); + assertThat(ackSet.isEmpty(), is(false)); + assertThat(ackSet.get(), is(acknowledgementSet)); + assertThat(streamAckManager.getCheckpoints().peek().getCheckpoint().getRecordCount(), is(recordCount)); + final ArgumentCaptor> argumentCaptor = ArgumentCaptor.forClass(Consumer.class); + verify(acknowledgementSetManager).create(argumentCaptor.capture(), eq(timeout)); + final Consumer consumer = argumentCaptor.getValue(); + consumer.accept(false); + final ConcurrentHashMap ackStatus = streamAckManager.getAcknowledgementStatus(); + final CheckpointStatus ackCheckpointStatus = ackStatus.get("1-1"); + assertThat(ackCheckpointStatus.isPositiveAcknowledgement(), is(false)); + await().atMost(Duration.ofSeconds(10)).untilAsserted(() -> verify(stopWorkerConsumer).accept(null)); + } +} diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamSchedulerTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamSchedulerTest.java new file mode 100644 index 0000000000..c51d48ccc3 --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamSchedulerTest.java @@ -0,0 +1,165 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.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.MockedStatic; +import org.mockito.junit.jupiter.MockitoExtension; +import org.opensearch.dataprepper.metrics.PluginMetrics; +import org.opensearch.dataprepper.model.acknowledgements.AcknowledgementSetManager; +import org.opensearch.dataprepper.model.buffer.Buffer; +import org.opensearch.dataprepper.model.event.Event; +import org.opensearch.dataprepper.model.record.Record; +import org.opensearch.dataprepper.model.source.coordinator.enhanced.EnhancedSourceCoordinator; +import org.opensearch.dataprepper.plugins.source.neptune.buffer.RecordBufferWriter; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.converter.StreamRecordConverter; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.StreamProgressState; + +import java.time.Duration; +import java.util.Optional; +import java.util.UUID; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import static org.awaitility.Awaitility.await; +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.mockStatic; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoInteractions; +import static org.opensearch.dataprepper.plugins.source.neptune.stream.StreamScheduler.DEFAULT_BUFFER_WRITE_INTERVAL_MILLS; +import static org.opensearch.dataprepper.plugins.source.neptune.stream.StreamScheduler.DEFAULT_CHECKPOINT_INTERVAL_MILLS; +import static org.opensearch.dataprepper.plugins.source.neptune.stream.StreamScheduler.DEFAULT_RECORD_FLUSH_BATCH_SIZE; + + +@ExtendWith(MockitoExtension.class) +public class StreamSchedulerTest { + private final String S3_PATH_PREFIX = UUID.randomUUID().toString(); + @Mock + private EnhancedSourceCoordinator sourceCoordinator; + + @Mock + private Buffer> buffer; + + @Mock + private PluginMetrics pluginMetrics; + + @Mock + private AcknowledgementSetManager acknowledgementSetManager; + + @Mock + private NeptuneSourceConfig sourceConfig; + + @Mock + private StreamWorker streamWorker; + + + private StreamScheduler streamScheduler; + + @BeforeEach + void setup() { + streamScheduler = new StreamScheduler(sourceCoordinator, buffer, acknowledgementSetManager, sourceConfig, S3_PATH_PREFIX, pluginMetrics); + } + + + @Test + void test_no_stream_run() throws InterruptedException { + given(sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE)).willReturn(Optional.empty()); + + ExecutorService executorService = Executors.newSingleThreadExecutor(); + executorService.submit(() -> streamScheduler.run()); + Thread.sleep(100); + executorService.shutdownNow(); + + verifyNoInteractions(streamWorker); + } + + @Test + void test_stream_run() { + final StreamProgressState streamProgressState = new StreamProgressState(); + final StreamPartition streamPartition = new StreamPartition(streamProgressState); + given(sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE)).willReturn(Optional.of(streamPartition)); + final int streamBatchSize = 100; + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + final Future future = executorService.submit(() -> { + try (MockedStatic streamWorkerMockedStatic = mockStatic(StreamWorker.class)) { + streamWorkerMockedStatic.when(() -> StreamWorker.create(any(RecordBufferWriter.class), any(StreamRecordConverter.class), eq(sourceConfig), + any(StreamAcknowledgementManager.class), any(DataStreamPartitionCheckpoint.class), eq(pluginMetrics), eq(DEFAULT_RECORD_FLUSH_BATCH_SIZE), + eq(DEFAULT_CHECKPOINT_INTERVAL_MILLS), eq(DEFAULT_BUFFER_WRITE_INTERVAL_MILLS), eq(streamBatchSize))) + .thenReturn(streamWorker); + streamScheduler.run(); + } + }); + + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(streamWorker).processStream(eq(streamPartition))); + + future.cancel(true); + executorService.shutdownNow(); + + } + + @Test + void test_stream_runThrowsException() { + final StreamProgressState streamProgressState = new StreamProgressState(); + final StreamPartition streamPartition = new StreamPartition(streamProgressState); + given(sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE)).willReturn(Optional.of(streamPartition)); + final int streamBatchSize = 100; + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + final Future future = executorService.submit(() -> { + try (MockedStatic streamWorkerMockedStatic = mockStatic(StreamWorker.class)) { + streamWorkerMockedStatic.when(() -> StreamWorker.create(any(RecordBufferWriter.class), any(StreamRecordConverter.class), eq(sourceConfig), + any(StreamAcknowledgementManager.class), any(DataStreamPartitionCheckpoint.class), eq(pluginMetrics), eq(DEFAULT_RECORD_FLUSH_BATCH_SIZE), + eq(DEFAULT_CHECKPOINT_INTERVAL_MILLS), eq(DEFAULT_BUFFER_WRITE_INTERVAL_MILLS), eq(streamBatchSize))) + .thenThrow(RuntimeException.class); + streamScheduler.run(); + } + }); + + await() + .atMost(Duration.ofSeconds(10)) + .untilAsserted(() -> verify(sourceCoordinator).giveUpPartition(streamPartition)); + + future.cancel(true); + executorService.shutdownNow(); + + } + + @Test + void test_stream_sourceCoordinatorThrowsException() { + final StreamProgressState streamProgressState = new StreamProgressState(); + + final StreamPartition streamPartition = new StreamPartition(streamProgressState); + given(sourceCoordinator.acquireAvailablePartition(StreamPartition.PARTITION_TYPE)).willThrow(RuntimeException.class); + + final ExecutorService executorService = Executors.newSingleThreadExecutor(); + final Future future = executorService.submit(() -> streamScheduler.run()); + + await() + .atMost(Duration.ofSeconds(2)) + .untilAsserted(() -> verify(sourceCoordinator, never()).giveUpPartition(streamPartition)); + + future.cancel(true); + executorService.shutdownNow(); + + } + + @Test + void test_stream_withNullS3PathPrefix() { + assertThrows(IllegalArgumentException.class, () -> new StreamScheduler(sourceCoordinator, buffer, acknowledgementSetManager, sourceConfig, null, pluginMetrics)); + } +} diff --git a/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamWorkerTest.java b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamWorkerTest.java new file mode 100644 index 0000000000..38f220824e --- /dev/null +++ b/data-prepper-plugins/neptune-source/src/test/java/org/opensearch/dataprepper/plugins/source/neptune/stream/StreamWorkerTest.java @@ -0,0 +1,83 @@ +/* + * Copyright OpenSearch Contributors + * SPDX-License-Identifier: Apache-2.0 + */ +package org.opensearch.dataprepper.plugins.source.neptune.stream; + +import io.micrometer.core.instrument.Counter; +import io.micrometer.core.instrument.DistributionSummary; +import org.junit.jupiter.api.BeforeEach; +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.neptune.buffer.RecordBufferWriter; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.AwsConfig; +import org.opensearch.dataprepper.plugins.source.neptune.configuration.NeptuneSourceConfig; +import org.opensearch.dataprepper.plugins.source.neptune.converter.StreamRecordConverter; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.partition.StreamPartition; +import org.opensearch.dataprepper.plugins.source.neptune.coordination.state.StreamProgressState; + +import java.util.Random; + +import static org.mockito.Mockito.when; +import static org.opensearch.dataprepper.plugins.source.neptune.stream.StreamWorker.BYTES_PROCESSED; +import static org.opensearch.dataprepper.plugins.source.neptune.stream.StreamWorker.BYTES_RECEIVED; +import static org.opensearch.dataprepper.plugins.source.neptune.stream.StreamWorker.FAILURE_ITEM_COUNTER_NAME; +import static org.opensearch.dataprepper.plugins.source.neptune.stream.StreamWorker.SUCCESS_ITEM_COUNTER_NAME; + +@ExtendWith(MockitoExtension.class) +public class StreamWorkerTest { + @Mock + private RecordBufferWriter mockRecordBufferWriter; + @Mock + private StreamRecordConverter mockRecordConverter; + @Mock + private NeptuneSourceConfig mockSourceConfig; + @Mock + private StreamAcknowledgementManager mockStreamAcknowledgementManager; + @Mock + private DataStreamPartitionCheckpoint mockPartitionCheckpoint; + @Mock + private PluginMetrics mockPluginMetrics; + + @Mock + private AwsConfig mockAwsConfig; + @Mock + private StreamPartition streamPartition; + @Mock + private StreamProgressState streamProgressState; + @Mock + private Counter successItemsCounter; + @Mock + private DistributionSummary bytesReceivedSummary; + @Mock + private DistributionSummary bytesProcessedSummary; + @Mock + private Counter failureItemsCounter; + + @Mock + private Counter streamApiInvocations; + + @Mock + private Counter stream4xxErrors; + + @Mock + private Counter stream5xxErrors; + + private StreamWorker streamWorker; + + private static final Random random = new Random(); + + @BeforeEach + public void setup() { + when(mockPluginMetrics.counter(SUCCESS_ITEM_COUNTER_NAME)).thenReturn(successItemsCounter); + when(mockPluginMetrics.counter(FAILURE_ITEM_COUNTER_NAME)).thenReturn(failureItemsCounter); + when(mockPluginMetrics.summary(BYTES_RECEIVED)).thenReturn(bytesReceivedSummary); + when(mockPluginMetrics.summary(BYTES_PROCESSED)).thenReturn(bytesProcessedSummary); + when(mockSourceConfig.isAcknowledgments()).thenReturn(false); + when(mockSourceConfig.getRegion()).thenReturn("us-east-1"); + streamWorker = new StreamWorker(mockRecordBufferWriter, mockRecordConverter, mockSourceConfig, mockStreamAcknowledgementManager, + mockPartitionCheckpoint, mockPluginMetrics, 2, 1000, 10_000, 1_000); + } +} diff --git a/settings.gradle b/settings.gradle index f8fc52a4d3..9241061cda 100644 --- a/settings.gradle +++ b/settings.gradle @@ -189,4 +189,4 @@ include 'data-prepper-plugins:opensearch-api-source' include 'data-prepper-plugins:saas-source-plugins' include 'data-prepper-plugins:saas-source-plugins:source-crawler' include 'data-prepper-plugins:saas-source-plugins:jira-source' - +include 'data-prepper-plugins:neptune-source'