RATIS-1519. When DataStreamManagement#read an exception occurs, remove DataStream#596
Conversation
|
@szetszwo Please take a look. |
szetszwo
left a comment
There was a problem hiding this comment.
@guohao-rosicky , thanks for working on this. Some comments inlined.
| removeDataStream(buf); | ||
| buf.release(); |
There was a problem hiding this comment.
Let's have buf.release() first.
@@ -380,6 +392,7 @@ public class DataStreamManagement {
readImpl(request, ctx, buf, getStreams);
} catch (Throwable t) {
buf.release();
+ removeDataStream(request);
throw t;
}
}
| RaftClientRequest request = ClientProtoUtils.toRaftClientRequest( | ||
| RaftClientRequestProto.parseFrom(buf.nioBuffer())); |
There was a problem hiding this comment.
This works only if request.getType() == Type.STREAM_HEADER. The code should be:
private void removeDataStream(DataStreamRequestByteBuf request) {
try {
final ClientInvocationId invocationId = ClientInvocationId.valueOf(request.getClientId(), request.getStreamId());
final StreamInfo s = streams.remove(invocationId);
if (s != null) {
s.getDivision().getDataStreamMap().remove(invocationId);
}
} catch (IOException ignored) {
LOG.debug(this + ": Failed to removeDataStream for " + request, ignored);
}
}
| ClientInvocationId invocationId = ClientInvocationId.valueOf(request); | ||
| server.getDivision(request.getRaftGroupId()).getDataStreamMap().remove(invocationId); | ||
| } catch (Throwable e) { | ||
| throw new CompletionException(e); |
There was a problem hiding this comment.
Just print an error message as shown above.
|
@szetszwo modified. I removed |
szetszwo
left a comment
There was a problem hiding this comment.
@guohao-rosicky , good catch. Some comments inlined; see also https://issues.apache.org/jira/secure/attachment/13039740/596_review.patch
| final ClientInvocationId invocationId = ClientInvocationId.valueOf(request.getClientId(), request.getStreamId()); | ||
| final StreamInfo s = streams.remove(invocationId); | ||
| if (s != null) { | ||
| s.getDivision().getDataStreamMap().remove(invocationId); |
There was a problem hiding this comment.
Just found that we could avoid s.getDivision() to throw IOException; see https://issues.apache.org/jira/secure/attachment/13039740/596_review.patch
| try { | ||
| if (exception != null) { | ||
| streams.remove(key); | ||
| removeDataStream(request); |
There was a problem hiding this comment.
We should pass info in this case since it could be already removed from streams at
There was a problem hiding this comment.
Yes, I modify it
ef63a6f to
d2c486c
Compare
|
@guohao-rosicky , there are some NPEs in https://github.com/apache/ratis/runs/5103000516?check_suite_focus=true . Please take a look. We may have to update the tests? |
|
@guohao-rosicky , from the debug message you added, it is a Mock for RaftServer. It also showed "DataStream null". that We probably need to update the test. |
szetszwo
left a comment
There was a problem hiding this comment.
Please take a look, thanks.
|
in Can you help me find it. |
|
@guohao-rosicky , It seems the server behavior was changed. The primary did not forward the request to the next server as shown below that s1 returns null; see https://issues.apache.org/jira/secure/attachment/13039878/596_debug.patch Please check if the new behavior is correct. If yes, we can update the test. |
1ca5047 to
f5b0905
Compare
|
@szetszwo I apply the patch, I still have an NPE, I'll keep tracking the issues |
@guohao-rosicky , Do you mean https://issues.apache.org/jira/secure/attachment/13039878/596_debug.patch ? It just prints out some dubug messages. Please check if the new behavior is correct. If yes, we can update the test in order to avoid the NPE. |
|
@guohao-rosicky , there are some conflicts. Could you fix them? |
szetszwo
left a comment
There was a problem hiding this comment.
+1 the change looks good.
|
@guohao-rosicky , thanks for working not this! On top of the bug fix, the change from |
What changes were proposed in this pull request?
When DataStreamManagement#read an exception occurs, remove DataStream
What is the link to the Apache JIRA
https://issues.apache.org/jira/browse/RATIS-1519