Skip to content
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

Remote exceptions #56

Merged
merged 15 commits into from
Aug 18, 2023
Merged

Remote exceptions #56

merged 15 commits into from
Aug 18, 2023

Conversation

aweisberg
Copy link
Contributor

No description provided.

@@ -99,12 +99,12 @@ public synchronized void accept(ReadNack reply, Throwable failure)
{
if (reply != null)
{
node.reply(replyTo, replyContext, reply);
node.reply(replyTo, replyContext, reply, null);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
node.reply(replyTo, replyContext, reply, null);
node.reply(replyTo, replyContext, reply, failure);

you didn't check if failure is non-null, so pass that on to the sink to check

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah makes sense to have it assert.

@@ -129,7 +129,9 @@ private void ack(@Nullable Ranges newUnavailable)
// and prevents races where we respond before dispatching all the required reads (if the reads are
// completing faster than the reads can be setup on all required shards)
if (-1 == --waitingOnCount)
reply(this.unavailable, data);
{
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

style: remove {}

{
if (waitingOnUpdater.decrementAndGet(this) == -1)
node.reply(replyTo, replyContext, WaitOnCommitOk.INSTANCE);
node.reply(replyTo, replyContext, fail != null ? WaitOnCommitOk.INSTANCE : null, fail);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this doesn't look right, if we see a failure will we actually decrement completely? Also wouldn't the exception be lost if it isn't the last event?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good catch. I will refactor it to check in accept and only send the failure response if it hasn't already sent a regular response and log if somehow we have failure but already sent a response.

if (fail != null)
{
checkState(success == null, "fail (%s) and success (%s) are both not null", fail, success);
// We don't really process errors for Recover here even though it is provided in the interface
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we should report back still, else the future can hang forever right?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can't report back because without the success there is no requestId to respond with. We can't use uncaught exception handler because the exception isn't typically uncaught tit's actually been passed by the caller of onRecover. onRecover is an odd one because it's kind of informative, but it isn't the callback for recover it's just some notification the agent gets.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We can't report back because without the success there is no requestId to respond with. We can't use uncaught exception handler because the exception isn't typically uncaught tit's actually been passed by to the callback of Recover. onRecover is an odd one because it's kind of informative, but it isn't the callback for recover it's just some notification the agent gets.

I need to look at why onRecover is involved in sending the reply at all. Why doesn't it get done in the callback for Recover? Why does it exist at all?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is only for the ListAgent, which is used only for testing purposes. We keep enough information to issue a response directly to clients for testing so we get more responses to analyse the correctness of.

@@ -110,38 +110,38 @@ public void accept(Result success, Throwable fail)
// TODO (desired, testing): error handling
if (success != null)
{
node.reply(client, replyContext, (ListResult) success);
node.reply(client, replyContext, (ListResult) success, null);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
node.reply(client, replyContext, (ListResult) success, null);
node.reply(client, replyContext, (ListResult) success, fail);

didn't check fail yet

@aweisberg aweisberg marked this pull request as ready for review August 16, 2023 16:18
if (send == null)
if (failure != null)
{
checkState(send == null, "fail (%s) and send (%s) are both not null", failure, send);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
checkState(send == null, "fail (%s) and send (%s) are both not null", failure, send);
if (send != null)
agent().onUncaughtException(new IllegalArgumentException(String.format("fail (%s) and send (%s) are both not null", failure, send)));

this situation would currently cause us to loose the real exception

int initialValue = waitingOnUpdater.get(this);
if (initialValue == -1)
{
logger.error("Had error in WaitOnCommit, but already replied so can't send failure response", failure);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
logger.error("Had error in WaitOnCommit, but already replied so can't send failure response", failure);
node.agent().onUncaughtException(new IllegalStateException("Had error in WaitOnCommit, but already replied so can't send failure response", failure));

if (throwable != null)
// TODO (expected, exceptions): should send exception to client, and consistency handle/propagate locally
logger.trace("{}: read failed for {}: {}", txnId, unsafeStore, throwable);
synchronized (ReadData.this)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you are not checking that throwable is non-null, so this could lead to an exception at fail.addSuppressed(throwable);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ouch yes.

@@ -149,14 +149,16 @@ void read(SafeCommandStore safeStore, Timestamp executeAt, PartialTxn txn)
Ranges unavailable = safeStore.ranges().unsafeToReadAt(executeAt);

txn.read(safeStore, executeAt).begin((next, throwable) -> {
if (throwable != null)
// TODO (expected, exceptions): should send exception to client, and consistency handle/propagate locally
logger.trace("{}: read failed for {}: {}", txnId, unsafeStore, throwable);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we don't actually know there is a read failure, we are a callback, so we see both success and failure! we have to first check which one we are

}
else
readComplete(unsafeStore, next, unavailable);
readComplete(unsafeStore, next, unavailable);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

if there is a failure do we really want to call readComplete? Feel like we should try to halt right away as the coordinator has already seen our failure

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I left it calling readComplete (exceptionally or otherwise) so that it would eventually send the response once all command stores responded with fail containing all the errors that occurred if there are more than one.

I am not married to that decision though. Sending back the first error is fine.

readComplete manipulates the waitingOn bit set so I left it at that level. readComplete does have one override that actually does something different, but since everything always goes through ack and ack checks for failures I was going to call it good enough.

It might be cleaner to put the waitingOn update into ack and call ack directly. WDYT?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ill look closer before replying directly to your comments, but one thing I am thinking about; when we lived with everyone needing to handle exceptions, no one did... this patch is trying to improve that.... so for this to hope all extending classes "do the right thing" might be good much to ask.... but ill reread this code and reflect on your comment

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The derived classes aren't the ones that send back the errors. ack does and that is private.

The derived classes can (and do) drop errors as part of their execution such as FetchRequest.readComplete which starts a new AsyncChain and adds a callback which passes the failure to uncaughtExceptionHandler.

I think the interface for listeners, task submission, and callbacks errors is possibly what is wrong. Every execution should occur with a context that specifies what the uncaught error handling should be, but you don't have to provide that context along with the thing you are executing. Or you do, but it's bespoke to that one thing you want to run (parameter to callback) rather than common across a variety of things that might be done to execute a request.

That would help with making it so can't get an error and then drop it or consume it in the wrong place because you would never handle it in the first place unless you specifically provided your own error handler rather than the one in the context. And when you do specify it should still hand off errors you don't want to deal with to the parent error handling context.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

before this patch this is what I see

  • accord.messages.ReadData#read would reply Error on the first exception and not call readComplete. This would not update waitingOn, but more importantly it would not call ack; which means if (-1 == --waitingOnCount) will never be true!
  • accord.impl.AbstractFetchCoordinator.FetchRequest#readComplete makes the read 2 operations:txn.read() and store.maxAppliedFor(...).onSuccess(a -> {this.maxApplied = a; super.readComplete(...)}) (I overly simplified). Even if a fetch had 2 reads (C* only has 1 atm), then we would still not see the ACK so would never complete the operation....
  • accord.messages.ReadTxnData#readComplete documents that it should unregister its listener... but because we call cancel we will schedule removal of the listener anyways... so thats prob fine.

I am not married to that decision though. Sending back the first error is fine.

I guess I would ask, what will the coordinator do differently with a list of failures? If a single one exists we will do the same as if 100 existed... so by only sending the first we have a smaller msg payload and can start handling the exception faster! We could detect this case and just log locally that these exceptions are getting ignored....

So, in short, I am in favor of failing fast and not pushing handling exceptions properly to each impl.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the interface for listeners, task submission, and callbacks errors is possibly what is wrong. Every execution should occur with a context that specifies what the uncaught error handling should be,

I agree with you on this point... AsyncChain and Futures are best when we "chain" them and centralize the error handling, but when we keep mixing and matching it gets hard to get right...

the common pattern is

// this is actually async, but to simplify the code I didn't do that
A value = null;
for store in stores.intersects(request):
  A local = store.process(request.map)
  value = request.reduce(value, local);
request.accept(value)

and in this case we try to add more state to make sure all map are called and accept is called (which is why we use -1 in -1 == --waitingOnCount)...

We could change this pattern to be

List<AsyncChain<A>> chains;
for store in stores.intersects(request):
  chains.add(store.process(request.map));
AsyncChains.reduce(chains, request.reduce)

this would allow us to start composing and chaining, rather than stoping chains, and forking new ones in different places...

Now, given how that is a structural change, I would prefer to do that as its own standalone work; adds far too much risk to this patch...

Comment on lines 336 to 348
V result = null;
boolean success = false;
try
{
trySetResult(callable.call(), null);
result = callable.call();
success = true;
}
catch (Throwable t)
{
trySetResult(null, t);
}
if (success)
trySetResult(result, null);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it would be good to document why the code is written this way, and we can avoid the boolean success if we rely on return when a failure is thrown

// There are two different type of exceptions: user function throws, listener throws.  To make sure this is clear,
            // make sure to catch the exception from the user function and set as failed, and let the listener exceptions bubble up.
            V call;
            try
            {
                call = callable.call();
            }
            catch (Throwable t)
            {
                setResult(null, t);
                return;
            }
            setResult(call, null);

@@ -44,14 +44,19 @@ public Task(Callable<T> fn)
@Override
public void run()
{
T result = null;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

to avoid copy/paste maybe we should extends AsyncResults.RunnableResult?

public static class Task<T> extends AsyncResults.RunnableResult<T> implements Pending, RunnableFuture<T>
    {
        public Task(Callable<T> fn)
        {
            super(fn);
        }

we can avoid the run method in this case completely

Copy link
Contributor

@dcapwell dcapwell left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am +1 once ReadData is resolved

@@ -153,7 +153,11 @@ void read(SafeCommandStore safeStore, Timestamp executeAt, PartialTxn txn)
{
// TODO (expected, exceptions): should send exception to client, and consistency handle/propagate locally
logger.trace("{}: read failed for {}: {}", txnId, unsafeStore, throwable);
node.reply(replyTo, replyContext, ReadNack.Error);
synchronized (ReadData.this)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why this lock? all impl delegate to CommandStores so wouldn't need locking.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You are right it probably isn't necessary. I reflexively added it because the existing call to cancel is from a synchronized method, but the actual implementations of cancel don't require it to be synchronized and are probably also idempotent so it's fine if multiple responses and cancellations happen at once. I'll remove it.

@@ -53,6 +52,7 @@ public abstract class ReadData extends AbstractEpochRequest<ReadData.ReadNack>
transient BitSet waitingOn;
transient int waitingOnCount;
transient Ranges unavailable;
transient Throwable fail;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

looks like dead code now?

{
if (fail != null)
{
node.reply(replyTo, replyContext, null, fail);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
node.reply(replyTo, replyContext, null, fail);
state = State.RETURNED;
node.reply(replyTo, replyContext, null, fail);

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't this also have the same illegal state logic as well? Maybe the fail handling should be in the switch below and handle obsolete as well.

@Override
public void replyWithFailure(Id replyingToNode, ReplyContext replyContext, Throwable failure)
{
reply(replyingToNode, replyContext, new FailureReply(failure));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
reply(replyingToNode, replyContext, new FailureReply(failure));
lookup.apply(self).agent().onUncaughtException(failure);
reply(replyingToNode, replyContext, new FailureReply(failure));

@aweisberg aweisberg merged commit 9133670 into apache:trunk Aug 18, 2023
@aweisberg aweisberg deleted the remote-exceptions branch August 18, 2023 20:48
@aweisberg aweisberg restored the remote-exceptions branch August 18, 2023 20:51
maedhroz pushed a commit to maedhroz/cassandra that referenced this pull request Sep 27, 2023
apache/cassandra-accord#56

Patch by Ariel Weisberg; Reviewed by David Capwell for CASSANDRA-18779
maedhroz pushed a commit to maedhroz/cassandra that referenced this pull request Sep 27, 2023
apache/cassandra-accord#56

Patch by Ariel Weisberg; Reviewed by David Capwell for CASSANDRA-18779
maedhroz pushed a commit to maedhroz/cassandra that referenced this pull request Sep 28, 2023
apache/cassandra-accord#56

Patch by Ariel Weisberg; Reviewed by David Capwell for CASSANDRA-18779
maedhroz pushed a commit to maedhroz/cassandra that referenced this pull request Sep 29, 2023
apache/cassandra-accord#56

Patch by Ariel Weisberg; Reviewed by David Capwell for CASSANDRA-18779
aweisberg added a commit to aweisberg/cassandra that referenced this pull request Dec 5, 2023
apache/cassandra-accord#56

Patch by Ariel Weisberg; Reviewed by David Capwell for CASSANDRA-18779
bdeggleston added a commit to bdeggleston/cassandra that referenced this pull request Mar 27, 2024
apache/cassandra-accord#56

Patch by Ariel Weisberg; Reviewed by David Capwell for CASSANDRA-18779
bdeggleston pushed a commit to bdeggleston/cassandra that referenced this pull request Mar 27, 2024
apache/cassandra-accord#56

Patch by Ariel Weisberg; Reviewed by David Capwell for CASSANDRA-18779
maedhroz pushed a commit to maedhroz/cassandra that referenced this pull request May 8, 2024
apache/cassandra-accord#56

Patch by Ariel Weisberg; Reviewed by David Capwell for CASSANDRA-18779
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
3 participants