forked from opensearch-project/data-prepper
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add AcknowledgementSet support to DocumentDB/MongoDB streams (opensea…
…rch-project#4379) * Add AcknowledgementSet support to DocumentDB/MongoDB streams Signed-off-by: Dinu John <[email protected]> * Update StreamAcknowledgementManagerTest Signed-off-by: Dinu John <[email protected]> * Add BackgroundThreadFactory that adds thread name prefix for debugging Signed-off-by: Dinu John <[email protected]> * Update unit test Signed-off-by: Dinu John <[email protected]> --------- Signed-off-by: Dinu John <[email protected]>
- Loading branch information
Showing
15 changed files
with
596 additions
and
94 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
45 changes: 45 additions & 0 deletions
45
...ongodb/src/main/java/org/opensearch/dataprepper/plugins/mongo/model/CheckpointStatus.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,45 @@ | ||
package org.opensearch.dataprepper.plugins.mongo.model; | ||
|
||
public class CheckpointStatus { | ||
private final String resumeToken; | ||
private final long recordCount; | ||
private boolean acknowledged; | ||
private final long createTimestamp; | ||
private Long acknowledgedTimestamp; | ||
|
||
public CheckpointStatus(final String resumeToken, final long recordCount, final long createTimestamp) { | ||
this.resumeToken = resumeToken; | ||
this.recordCount = recordCount; | ||
this.acknowledged = false; | ||
this.createTimestamp = createTimestamp; | ||
} | ||
|
||
public void setAcknowledgedTimestamp(final Long acknowledgedTimestamp) { | ||
this.acknowledgedTimestamp = acknowledgedTimestamp; | ||
} | ||
|
||
public void setAcknowledged(boolean acknowledged) { | ||
this.acknowledged = acknowledged; | ||
} | ||
|
||
public String getResumeToken() { | ||
return resumeToken; | ||
} | ||
public long getRecordCount() { | ||
return recordCount; | ||
} | ||
|
||
public boolean isAcknowledged() { | ||
return acknowledged; | ||
} | ||
|
||
public long getCreateTimestamp() { | ||
return createTimestamp; | ||
} | ||
|
||
public long getAcknowledgedTimestamp() { | ||
return acknowledgedTimestamp; | ||
} | ||
|
||
|
||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
127 changes: 127 additions & 0 deletions
127
...in/java/org/opensearch/dataprepper/plugins/mongo/stream/StreamAcknowledgementManager.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,127 @@ | ||
package org.opensearch.dataprepper.plugins.mongo.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.mongo.model.CheckpointStatus; | ||
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.function.Consumer; | ||
|
||
public class StreamAcknowledgementManager { | ||
private static final Logger LOG = LoggerFactory.getLogger(StreamAcknowledgementManager.class); | ||
private final ConcurrentLinkedQueue<CheckpointStatus> checkpoints = new ConcurrentLinkedQueue<>(); | ||
private final ConcurrentHashMap<String, CheckpointStatus> 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 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("mongodb-stream-ack-monitor")); | ||
} | ||
|
||
void init(final Consumer<Void> stopWorkerConsumer) { | ||
enableAcknowledgement = true; | ||
executorService.submit(() -> monitorAcknowledgment(executorService, stopWorkerConsumer)); | ||
} | ||
|
||
private void monitorAcknowledgment(final ExecutorService executorService, final Consumer<Void> stopWorkerConsumer) { | ||
long lastCheckpointTime = System.currentTimeMillis(); | ||
CheckpointStatus lastCheckpointStatus = null; | ||
while (!Thread.currentThread().isInterrupted()) { | ||
final CheckpointStatus checkpointStatus = checkpoints.peek(); | ||
if (checkpointStatus != null) { | ||
if (checkpointStatus.isAcknowledged()) { | ||
lastCheckpointStatus = checkpoints.poll(); | ||
ackStatus.remove(checkpointStatus.getResumeToken()); | ||
if (System.currentTimeMillis() - lastCheckpointTime >= checkPointIntervalInMs) { | ||
LOG.debug("Perform regular checkpointing for resume token {} at record count {}", checkpointStatus.getResumeToken(), checkpointStatus.getRecordCount()); | ||
partitionCheckpoint.checkpoint(checkpointStatus.getResumeToken(), checkpointStatus.getRecordCount()); | ||
lastCheckpointTime = System.currentTimeMillis(); | ||
} | ||
} else { | ||
LOG.debug("Checkpoint not complete for resume token {}", checkpointStatus.getResumeToken()); | ||
final Duration ackWaitDuration = Duration.between(Instant.ofEpochMilli(checkpointStatus.getCreateTimestamp()), Instant.now()); | ||
// Acknowledgement not received for the checkpoint after twice ack wait time | ||
if (ackWaitDuration.getSeconds() >= partitionAcknowledgmentTimeout.getSeconds() * 2) { | ||
// Give up partition and should interrupt parent thread to stop processing stream | ||
if (lastCheckpointStatus != null && lastCheckpointStatus.isAcknowledged()) { | ||
partitionCheckpoint.checkpoint(lastCheckpointStatus.getResumeToken(), lastCheckpointStatus.getRecordCount()); | ||
} | ||
LOG.warn("Acknowledgement not received for the checkpoint {} past wait time. Giving up partition.", checkpointStatus.getResumeToken()); | ||
partitionCheckpoint.giveUpPartition(); | ||
break; | ||
} | ||
} | ||
} | ||
|
||
try { | ||
Thread.sleep(acknowledgementMonitorWaitTimeInMs); | ||
} catch (InterruptedException ex) { | ||
break; | ||
} | ||
} | ||
stopWorkerConsumer.accept(null); | ||
executorService.shutdown(); | ||
} | ||
|
||
Optional<AcknowledgementSet> createAcknowledgementSet(final String resumeToken, final long recordNumber) { | ||
if (!enableAcknowledgement) { | ||
return Optional.empty(); | ||
} | ||
|
||
final CheckpointStatus checkpointStatus = new CheckpointStatus(resumeToken, recordNumber, Instant.now().toEpochMilli()); | ||
checkpoints.add(checkpointStatus); | ||
ackStatus.put(resumeToken, checkpointStatus); | ||
return Optional.of(acknowledgementSetManager.create((result) -> { | ||
if (result) { | ||
final CheckpointStatus ackCheckpointStatus = ackStatus.get(resumeToken); | ||
ackCheckpointStatus.setAcknowledgedTimestamp(Instant.now().toEpochMilli()); | ||
ackCheckpointStatus.setAcknowledged(true); | ||
LOG.debug("Received acknowledgment of completion from sink for checkpoint {}", resumeToken); | ||
} else { | ||
LOG.warn("Negative acknowledgment received for checkpoint {}, resetting checkpoint", resumeToken); | ||
// 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() { | ||
executorService.shutdown(); | ||
} | ||
|
||
@VisibleForTesting | ||
ConcurrentHashMap<String, CheckpointStatus> getAcknowledgementStatus() { | ||
return ackStatus; | ||
} | ||
|
||
@VisibleForTesting | ||
ConcurrentLinkedQueue<CheckpointStatus> getCheckpoints() { | ||
return checkpoints; | ||
} | ||
} |
Oops, something went wrong.