-
Notifications
You must be signed in to change notification settings - Fork 13.8k
[FLINK-24468][runtime] Wait for the channel activation before creating partition request client #17440
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
Conversation
|
Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community Automated ChecksLast check on commit 37cf758 (Fri Oct 08 15:42:04 UTC 2021) Warnings:
Mention the bot in a comment to re-run the automated checks. Review Progress
Please see the Pull Request Review Guide for a full explanation of the review process. DetailsThe Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required Bot commandsThe @flinkbot bot supports the following commands:
|
dawidwys
left a comment
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.
A higher level question. Do we really just stop debloating or do we trigger a failover?
Shouldn't the exception from the mailbox processor fail the task? Honestly, I am a bit hesitant to simply ignoring exceptions there and continue processing.
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
Outdated
Show resolved
Hide resolved
|
Another question. Why does it happen only for announcing the new buffer size? Shouldn't we have the same issue for all other methods of |
As I can see we just stopping debloating and no failover would be triggered
I was also surprised but it looks like we just ignore this error(it is exactly what should be figured out later).
if we have input data already then the this NPE is impossible, it is exactly what happens when we call other methods except for the new buffer size. So everything except the announcement of the buffer size is a reaction to the input data.
As you can see I made this NPE impossible by waiting for channel initialization on the start. So everything should work well now
I don't really sure that we need this try/catch block at all. My main fix is about waiting for the initialization but these changes I made just in case, but perhaps it is much better to understand the flink behavior when we have exceptions inside of the action.( But in general, I agree that |
|
I found the reason why the exception was swallowed. Please see: https://issues.apache.org/jira/browse/FLINK-24515. I think we should not swallow exceptions if debloating fails. Therefore I'd remove the As for the |
|
So I spent some time looking into the As the initial error shows it may take some time. I was rather thinking if we could return the future itself (I know we would've to pass it through a couple of layers). Then we could start buffer debloating once a future for all channels activation completes. |
|
I don't really sure about this proposal because we already create the client via clientBuilder, and then, by some reason, we need to wait on a future before the call of the one particular method. It looks pretty fragile because if tomorrow we will add one more method like that we will need to duplicate this waiting.
Only one open question for this solution - how to handle the interruptedException which is not typical for this place right now. I will try to come up with other ideas. |
|
Another idea to throw into the mix: Why do we even have the methods like: On the other hand we have the or (I am not sure about the threading model) I am sorry, I do not have much expertise on netty, so would be really nice to hear from @pnowojski on the idea above. |
|
Most likely we added I don't know why I haven't read the full thread, so I'm not sure. If we had changed the way you are proposing @dawidwys , would it solve/help solve the problems in this ticket? |
|
I understand I think it was added inside of the |
|
@dawidwys, I will try to implement it this way. |
|
Looking at the test results it seems to be working just fine (e2e failure unrelated). How do you feel about the solution @akalash ? If you see no obstacles, I'd suggest proceeding with it. |
|
@dawidwys, thanks for the idea. I have checked it and it looks like it should work without any problem and this implementation looks more correct than current one. So let's wait for tests for the last commits and then you can take a look at it again. |
dawidwys
left a comment
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.
Just some nitpicking on classes scope. I can fix it while merging. I'd just like to hear a confirmation.
...k-runtime/src/main/java/org/apache/flink/runtime/io/network/netty/ClientOutboundMessage.java
Outdated
Show resolved
Hide resolved
...ime/src/main/java/org/apache/flink/runtime/io/network/netty/NettyPartitionRequestClient.java
Outdated
Show resolved
Hide resolved
| } | ||
|
|
||
| private void sendToChannel(ClientOutboundMessage message) { | ||
| tcpChannel.eventLoop().execute(() -> tcpChannel.pipeline().fireUserEventTriggered(message)); |
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.
Did you investigate if we need to submit it through the eventLoop or would it be enough to call tcpChannel.pipeline().fireUserEventTriggered(message)?
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.
Yes, I did. Before these changes, we always submitted it through the evenLoop so I don't see a reason why we should change that
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.
I am fine merging it as is. However, I am curious myself if it was/is necessary (I could not find an answer, but I have not searched for long). But, I am not too familiar with how netty works.
BTW, I've just looked into it again and if I go into: ChannelPipeline#fireUserEventTriggered -> DefaultPipeline#fireUserEventTriggered -> AbstractChannelHandlerContext.invokeUserEventTriggered(this.head, event); I can see a code like this:
static void invokeUserEventTriggered(final AbstractChannelHandlerContext next, final Object event) {
ObjectUtil.checkNotNull(event, "event");
EventExecutor executor = next.executor();
if (executor.inEventLoop()) {
next.invokeUserEventTriggered(event);
} else {
executor.execute(new Runnable() {
public void run() {
next.invokeUserEventTriggered(event);
}
});
}
}
which would indicate submitting it explicitly from the eventLoop is unnecessary.
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.
Unfortunatelly, it is not possible to get rid of tcpChannel.eventLoop().execute because executor.inEventLoop() is always true since this call happens in the network thread but we still don't want to block the current execution and we want to send the massage in the other thread(or in the same thread but later).
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.
Actually, I can be wrong here and this problem can be related only to the test. But right now, I don't ready to answer more precisely.
…tworkClientHandler to NettyPartitioonRequestClient since it is not responsibility of handler to send them
…er size calculation to mailbox
dawidwys
left a comment
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.
LGTM
What is the purpose of the change
This PR fixes NPE which happens when we try to use channel context inside of ClientHandler before the channel was activated
Brief change log
Verifying this change
This change added tests and can be verified as follows:
Does this pull request potentially affect one of the following parts:
@Public(Evolving): (yes / no)Documentation