-
Notifications
You must be signed in to change notification settings - Fork 208
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Add e2e acknowledgment and checkpointing to RDS source #4819
Changes from 7 commits
dafae15
fec5ee0
f719278
08f293b
31ffc71
847128b
e44ec9d
63db157
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -8,6 +8,8 @@ | |
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.acknowledgements.AcknowledgementSetManager; | ||
import org.opensearch.dataprepper.model.buffer.Buffer; | ||
import org.opensearch.dataprepper.model.codec.InputCodec; | ||
import org.opensearch.dataprepper.model.event.Event; | ||
|
@@ -32,6 +34,7 @@ | |
import java.util.concurrent.ExecutorService; | ||
import java.util.concurrent.Executors; | ||
import java.util.concurrent.atomic.AtomicInteger; | ||
import java.util.function.BiConsumer; | ||
|
||
import static org.opensearch.dataprepper.plugins.source.rds.RdsService.DATA_LOADER_MAX_JOB_COUNT; | ||
|
||
|
@@ -63,6 +66,7 @@ public class DataFileScheduler implements Runnable { | |
private final BufferAccumulator<Record<Event>> bufferAccumulator; | ||
private final ExportRecordConverter recordConverter; | ||
private final PluginMetrics pluginMetrics; | ||
private final AcknowledgementSetManager acknowledgementSetManager; | ||
|
||
private final Counter exportFileSuccessCounter; | ||
private final Counter exportFileErrorCounter; | ||
|
@@ -75,7 +79,8 @@ public DataFileScheduler(final EnhancedSourceCoordinator sourceCoordinator, | |
final S3Client s3Client, | ||
final EventFactory eventFactory, | ||
final Buffer<Record<Event>> buffer, | ||
final PluginMetrics pluginMetrics) { | ||
final PluginMetrics pluginMetrics, | ||
final AcknowledgementSetManager acknowledgementSetManager) { | ||
this.sourceCoordinator = sourceCoordinator; | ||
this.sourceConfig = sourceConfig; | ||
codec = new ParquetInputCodec(eventFactory); | ||
|
@@ -84,6 +89,7 @@ public DataFileScheduler(final EnhancedSourceCoordinator sourceCoordinator, | |
recordConverter = new ExportRecordConverter(); | ||
executor = Executors.newFixedThreadPool(DATA_LOADER_MAX_JOB_COUNT); | ||
this.pluginMetrics = pluginMetrics; | ||
this.acknowledgementSetManager = acknowledgementSetManager; | ||
|
||
this.exportFileSuccessCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_PROCESSED_COUNT); | ||
this.exportFileErrorCounter = pluginMetrics.counter(EXPORT_S3_OBJECTS_ERROR_COUNT); | ||
|
@@ -133,23 +139,39 @@ public void shutdown() { | |
} | ||
|
||
private void processDataFilePartition(DataFilePartition dataFilePartition) { | ||
// Create AcknowledgmentSet | ||
final boolean isAcknowledgmentsEnabled = sourceConfig.isAcknowledgmentsEnabled(); | ||
AcknowledgementSet acknowledgementSet = null; | ||
if (sourceConfig.isAcknowledgmentsEnabled()) { | ||
acknowledgementSet = acknowledgementSetManager.create((result) -> { | ||
if (result) { | ||
completeDataLoader(dataFilePartition).accept(null, null); | ||
LOG.info("Received acknowledgment of completion from sink for data file {}", dataFilePartition.getKey()); | ||
} else { | ||
exportFileErrorCounter.increment(); | ||
LOG.warn("Negative acknowledgment received for data file {}, retrying", dataFilePartition.getKey()); | ||
sourceCoordinator.giveUpPartition(dataFilePartition); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It me just me trying to understand the code here. Not a real comment about the code. I would assume the retry on this partition and also keeping track of retryCount is handled by the sourceCoordinator? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes. Once the datafile partition is given up, it will be available for acquire again and the file load will be retried by whichever node that picked up the partition. |
||
} | ||
}, sourceConfig.getDataFileAcknowledgmentTimeout()); | ||
} | ||
|
||
Runnable loader = DataFileLoader.create( | ||
dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter, pluginMetrics); | ||
dataFilePartition, codec, bufferAccumulator, objectReader, recordConverter, pluginMetrics, | ||
sourceCoordinator, acknowledgementSet, sourceConfig.getDataFileAcknowledgmentTimeout()); | ||
CompletableFuture runLoader = CompletableFuture.runAsync(loader, executor); | ||
|
||
runLoader.whenComplete((v, ex) -> { | ||
if (ex == null) { | ||
exportFileSuccessCounter.increment(); | ||
// Update global state so we know if all s3 files have been loaded | ||
updateLoadStatus(dataFilePartition.getExportTaskId(), DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT); | ||
sourceCoordinator.completePartition(dataFilePartition); | ||
} else { | ||
exportFileErrorCounter.increment(); | ||
LOG.error("There was an exception while processing an S3 data file", ex); | ||
sourceCoordinator.giveUpPartition(dataFilePartition); | ||
} | ||
numOfWorkers.decrementAndGet(); | ||
}); | ||
if (isAcknowledgmentsEnabled) { | ||
runLoader.whenComplete((v, ex) -> { | ||
if (ex != null) { | ||
exportFileErrorCounter.increment(); | ||
LOG.error("There was an exception while processing an S3 data file: {}", ex); | ||
sourceCoordinator.giveUpPartition(dataFilePartition); | ||
} | ||
numOfWorkers.decrementAndGet(); | ||
}); | ||
} else { | ||
runLoader.whenComplete(completeDataLoader(dataFilePartition)); | ||
} | ||
numOfWorkers.incrementAndGet(); | ||
} | ||
|
||
|
@@ -183,4 +205,20 @@ private void updateLoadStatus(String exportTaskId, Duration timeout) { | |
} | ||
} | ||
} | ||
|
||
private BiConsumer<Void, Throwable> completeDataLoader(DataFilePartition dataFilePartition) { | ||
return (v, ex) -> { | ||
if (ex == null) { | ||
exportFileSuccessCounter.increment(); | ||
// Update global state, so we know if all s3 files have been loaded | ||
updateLoadStatus(dataFilePartition.getExportTaskId(), DEFAULT_UPDATE_LOAD_STATUS_TIMEOUT); | ||
sourceCoordinator.completePartition(dataFilePartition); | ||
} else { | ||
exportFileErrorCounter.increment(); | ||
LOG.error("There was an exception while processing an S3 data file", ex); | ||
sourceCoordinator.giveUpPartition(dataFilePartition); | ||
} | ||
numOfWorkers.decrementAndGet(); | ||
}; | ||
} | ||
} |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
BufferAccumulator
is not thread safe. I would recommend to instantiate this set with in the run method itself instead of passing to this thread. Or avoid using Accumulator if it is not really required.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Good point! I moved the instantiation into the run method.