jmckenzie-dev commented on code in PR #178:
URL:
https://github.com/apache/cassandra-analytics/pull/178#discussion_r2907401548
##########
cassandra-analytics-cdc-sidecar/src/main/java/org/apache/cassandra/cdc/sidecar/SidecarStatePersister.java:
##########
Review Comment:
> if flush fails, do we not allow restart of iterators to handle above
changes? If we don't allow restart, it will remain in the broken state forever.
Right now, if we get an OutOfMemoryError, we swallow it.
`java.io.IOException`? Swallow it.
This code here:
```
protected void persistToCassandra(boolean force)
{
// clean-up finished futures
activeFlush.removeIf(wrapper -> {
if (wrapper.allDone())
{
try
{
wrapper.await();
sidecarCdcStats.capturePersistSucceeded(System.nanoTime() -
wrapper.startTimeNanos);
}
catch (InterruptedException e)
{
LOGGER.warn("Persist failed with InterruptedException",
e);
Thread.currentThread().interrupt();
sidecarCdcStats.capturePersistFailed(e);
}
catch (Throwable throwable)
{
LOGGER.warn("Persist failed", throwable);
sidecarCdcStats.capturePersistFailed(throwable);
}
return true;
}
return false;
});
```
means that we can have all manner of nasty exceptions happen inside the
wrapped `persistToCassandra` calls. So right now, `.stop()` isn't greedy in the
passthrough to `flushActiveSafe`, but if anything goes haywire in the
`persistToCassandra` world we'll swallow it, log about it, and keep moving
along, allowing subsequent restart of the `SidecarStatePersister` since
`this.timerId` will be -1.
So my question really is: do we _always_ want the `SidecarStatePersister` in
a "restartable" state after calling `.stop()`, regardless of whether there are
serious underlying exceptions thrown by the `activeFlush` members?
`flushActiveSafe` has allowances for `ExecutionException` and
`InterruptedException` but otherwise will bubble exceptions up and leave the
object in a broken state if things otherwise fail in unexpected ways. Do we not
want to do the same for the state persistence?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]