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

[FLINK-15416][network] add task manager netty client retry mechenism #11541

Conversation

HuangZhenQiu
Copy link
Contributor

What is the purpose of the change

Add retry logic for netty client creation in PartitionRequestClientFactory. It is useful to make flink runtime tolerant physical link issue in the switch.

Brief changelog

  • Add a netty client retry config
  • Add retry logic in PartitionRequestClientFactory for building new channel.

Verifying this change

This change is a trivial rework / code cleanup without any test coverage.

Does this pull request potentially affect one of the following parts:

  • Dependencies (does it add or upgrade a dependency): (no)
  • The public API, i.e., is any changed class annotated with @Public(Evolving): (no)
  • The serializers: (no)
  • The runtime per-record code paths (performance sensitive): (no )
  • Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (no)
  • The S3 file system connector: (no)

Documentation

  • Does this pull request introduce a new feature? (yes)
  • If yes, how is the feature documented? (docs)

@flinkbot
Copy link
Collaborator

Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
to review your pull request. We will use this comment to track the progress of the review.

Automated Checks

Last check on commit 3bc439b (Fri Mar 27 04:24:31 UTC 2020)

Warnings:

  • No documentation files were touched! Remember to keep the Flink docs up to date!
  • This pull request references an unassigned Jira ticket. According to the code contribution guide, tickets need to be assigned before starting with the implementation work.

Mention the bot in a comment to re-run the automated checks.

Review Progress

  • ❓ 1. The [description] looks good.
  • ❓ 2. There is [consensus] that the contribution should go into to Flink.
  • ❓ 3. Needs [attention] from.
  • ❓ 4. The change fits into the overall [architecture].
  • ❓ 5. Overall code [quality] is good.

Please see the Pull Request Review Guide for a full explanation of the review process.


The 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 commands
The @flinkbot bot supports the following commands:

  • @flinkbot approve description to approve one or more aspects (aspects: description, consensus, architecture and quality)
  • @flinkbot approve all to approve all aspects
  • @flinkbot approve-until architecture to approve everything until architecture
  • @flinkbot attention @username1 [@username2 ..] to require somebody's attention
  • @flinkbot disapprove architecture to remove an approval you gave earlier

@flinkbot
Copy link
Collaborator

flinkbot commented Mar 27, 2020

CI report:

Bot commands The @flinkbot bot supports the following commands:
  • @flinkbot run travis re-run the last Travis build
  • @flinkbot run azure re-run the last Azure build

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch 3 times, most recently from 3d6cba1 to b5ef192 Compare March 27, 2020 19:36
@HuangZhenQiu
Copy link
Contributor Author

@pnowojski

Would you please review this PR?

@pnowojski
Copy link
Contributor

This pull request references an unassigned Jira ticket. According to the code contribution guide, tickets need to be assigned before starting with the implementation work.

Please next time do not start work before being assigned to the ticket. Especially that there was my unresolved question in the Jira ticket whether we want to have such change or not.

In the end I think we can go in the direction of supporting retry mechanism in the network stack (including TCP connection loss in the middle of transfer), so we can just as well start with the reconnection retries as you are proposing.

I will give more detailed look at your PR later. For starter, could you try to provide unit tests for the PartitionRequestClientFactory class? For example by mocking NettyClient?

@HuangZhenQiu
Copy link
Contributor Author

@pnowojski
Thanks for the feedback. The network issue happens very often internally for us. Thus, we want to have a solution to mitigate at least internally. I will make sure to start to work after the ticket is assigned next week. For this PR, I will first add the test case for retry.

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch 2 times, most recently from 95069b3 to 8b0f177 Compare March 30, 2020 17:55
Copy link
Contributor

@pnowojski pnowojski left a comment

Choose a reason for hiding this comment

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

Thanks for the update. I've left couple of comments.

Comment on lines 119 to 84

try {
client = ((ConnectingChannel) old).waitForChannel();
} catch (IOException e) {
LOG.info("Continue to waitForChannel until the original thread has completed or failed");
}
Copy link
Contributor

Choose a reason for hiding this comment

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

What do you mean by this change/log message?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

As multiple subpartition will share the same channel. Let's say subpartition a, b both needs the channel. A goes to wait first, B will still wait here. If exception happens, both of them will be notified. But we don't want the exception throw in B as A can still retry.

Copy link
Contributor

Choose a reason for hiding this comment

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

But shouldn't B implement the same retry logic? Otherwise we are swallowing exceptions.

Copy link
Contributor

Choose a reason for hiding this comment

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

The current logic seems have some problems. When it exists exception while waiting, then the client is still null which is replaced into the map via below clients.replace(connectionId, old, client). In next loop, it would enter the above connectChannelWithRetry logic, that means the B might also encounter the same process as previous A to retry some times.

I guess the expectation we want is only let A retry connections and B is always waiting during A retries until final success or 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.

@pnowojski
The logic is always to let the first requester to retry. For the following requesters, will always go to wait until the first requester get the client or finished retry. Otherwise, there will be more retry times than expected.

@zhijiangW
I am not sure whether my understanding is correct or now. Once the ConnectingChannel is put into clients concurrentMap by the first requester, the object is never null after that. Thus, other requesters will go to line 97. So the process is what you described " let A retry connections and B is always waiting during A retries until final success or failure." Are you aligned?

Copy link
Contributor

Choose a reason for hiding this comment

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

Ok. I see, I missed the outer while (client == null) loop.

But as it is now, isn't it possible that B will loop indefinitely? Shouldn't it also follow the max retry count logic and if exceeded, throw an exception?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Agree. Apply the same retry logic for following channel request other than the first one.

@@ -130,6 +130,23 @@ public void run() {
}
}

@Test
public void testNettyClientConnectRetry() throws Exception {
NettyClient nettyClient = mock(NettyClient.class);
Copy link
Contributor

Choose a reason for hiding this comment

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

We stopped using mockito for tests (for various of reasons), could you try to provide a proper mock of NettyClient class? Maybe if you used NettyTestUtil you could just implement NettyClient#connect method that just throws an exception on the first invocation, but succeeds (returns real/correct ChannelFuture) on the second attempt?

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 think it will be better. But when I try to implement the TestingNettyClient, I found the SucceededChannelFuture only visible to package io.netty.channel. Thus, I need to create a new class of SucceededChannelFuture in Flink to return a success ChannelFuture. Do you think it is a right way to go?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes I noticed that and probably implementing ChannelFuture is not worth it, hence I suggested to use NettyTestUtil. It looks like you could just init a server (NettyTestUtil#initServer()) and NettyClient from #initClient() and you could use it directly, without a need to implement custom ChannelFuture? You can check usages of NettyTestUtil, like for example NettyClientServerSslTest#testClientUntrustedCertificate.

I might be missing something, but it looks like it could work quite easily.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes. I created a UnstableNettyClient to simulate the first n-1 connection failure and success in the last try.

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch 2 times, most recently from 6cdd6e4 to 3c0e3fb Compare April 3, 2020 08:11
@HuangZhenQiu
Copy link
Contributor Author

Thanks for the suggestions. I updated the PR accordingly. My major concern is to add a TestingNettyClient. It is a little bit overhead to create a SucceededChannelFuture to simulate the success path. Do you think it is the right way to proceed or we can have a simpler way?

Copy link
Contributor

@pnowojski pnowojski left a comment

Choose a reason for hiding this comment

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

Thanks for the update, I've posted couple of responses.

@@ -130,6 +130,23 @@ public void run() {
}
}

@Test
public void testNettyClientConnectRetry() throws Exception {
NettyClient nettyClient = mock(NettyClient.class);
Copy link
Contributor

Choose a reason for hiding this comment

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

Yes I noticed that and probably implementing ChannelFuture is not worth it, hence I suggested to use NettyTestUtil. It looks like you could just init a server (NettyTestUtil#initServer()) and NettyClient from #initClient() and you could use it directly, without a need to implement custom ChannelFuture? You can check usages of NettyTestUtil, like for example NettyClientServerSslTest#testClientUntrustedCertificate.

I might be missing something, but it looks like it could work quite easily.

Comment on lines 119 to 84

try {
client = ((ConnectingChannel) old).waitForChannel();
} catch (IOException e) {
LOG.info("Continue to waitForChannel until the original thread has completed or failed");
}
Copy link
Contributor

Choose a reason for hiding this comment

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

But shouldn't B implement the same retry logic? Otherwise we are swallowing exceptions.

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch 2 times, most recently from 9a02c82 to eeee4cd Compare April 7, 2020 04:19
Copy link
Contributor Author

@HuangZhenQiu HuangZhenQiu left a comment

Choose a reason for hiding this comment

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

@zhijiangW Thanks for joining the discussion. Fixed two obvious issues as you identified. I also replied to your concerns about the retry logic.

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch 3 times, most recently from 91c10df to baf2c55 Compare April 7, 2020 21:28
@pnowojski
Copy link
Contributor

pnowojski commented Jun 11, 2020

Thanks for the udpate @HuangZhenQiu

There is no test that is marked to ignore by this PR. Would you please double-check?

The whole PartitionRequestClientFactoryTest class was marked as @Ignore by Ufuk 5 years ago (line ~48). I don't know why and I don't know if the original reason/problem still exists or not. If the existing test is still unstable, I would be fine in completely dropping the unit test if it can not be fixed easily.

@HuangZhenQiu
Copy link
Contributor Author

@pnowojski
Oh, I see. I didn't check the class level to ignore. how about putting ignore only in the method testResourceReleaseAfterInterruptedConnect?

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch from f64492a to 5149176 Compare June 11, 2020 17:55
@pnowojski
Copy link
Contributor

pnowojski commented Jun 12, 2020

how about putting ignore only in the method testResourceReleaseAfterInterruptedConnect?

I haven't thought about that :) Yep, it's an obvious simple solution.

Test failure is unrelated and already fixed on master: FLINK-18239

@pnowojski
Copy link
Contributor

I think the testNettyClientConnectRetryMultipleThread test is unstable. It can deadlock once every ~50 runs when I loop whole PartitionRequestClientFactoryTest test on my local machine. If I try to execute just the testNettyClientConnectRetryMultipleThread it seems to not complete (deadlock?) even in the first attempt?

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch from 5149176 to 2ff2e89 Compare June 15, 2020 00:24
@HuangZhenQiu
Copy link
Contributor Author

@pnowojski
Yes, you are right. It is deadline for creating channel clients for the same connection id. To achieve a parallel connection building process for different connection id and also a synchronized way for the requests to the same connection id, I chose to synchronize connectionId in the createPartitionRequestClient function. It can guarantee the correctness with scarifying the scalability of the connection creation. Actually, we don't need to even use concurrent map. How do you think?

@pnowojski
Copy link
Contributor

pnowojski commented Jun 15, 2020

Can you explain what was the deadlock scenario?

I haven't got time to fully analyse your solution, but ConcurrentMap<...> clients is being used in other methods. It's also even in the same method where you added synchronized (connectionId), but before the synchronized section, so we can not simply change it to HashMap. Moreover your synchronized section wouldn't prevent two different ConnectionID requests from modifying same shared clients map.

Copy link
Contributor

@rkhachatryan rkhachatryan left a comment

Choose a reason for hiding this comment

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

Thanks for this PR @HuangZhenQiu, I also took a look at it and left some comments.

My major concern, however, is that Flink already has retry mechanism configured via taskmanager.network.request-backoff.xxx.
I guess it didn't work for you because it handles higher level errors, when we the connection is already established, right?

Did you consider modifying it?
I think it would be great for both users and developers to have a single retry mechanism for both kind of errors (besides complexity, it's not clear how would they interact with each other_.

Besides that, there is a deadlock in Azure (I'm also able to reproduce it locally).

clients.replace(connectionId, connectingChannel, client);
return client;
} catch (IOException | ChannelException e) {
LOG.error("Failed {} times to connect to {}", count, connectionId.getAddress(), e);
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 it will print 0 times for the 1st failure and 1 times for the 2nd. Should count + 1 be used?

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. There is why I syncronized on connectionId in line 74. It makes only one thread can goes into the connection for particular connectionId, it either success or fail after several times of retries.

} catch (IOException | ChannelException e) {
LOG.error("Failed {} times to connect to {}", count, connectionId.getAddress(), e);
ConnectingChannel newConnectingChannel = new ConnectingChannel(connectionId, this);
clients.replace(connectionId, connectingChannel, newConnectingChannel);
Copy link
Contributor

Choose a reason for hiding this comment

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

If the exception is thrown by connect and not waitForChannel then we use a new channel.
But other threads are waiting on the old channel - which will never be completed, right?

I guess this is the reason of the deadlock I see in Azure and locally.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes. It is the reason of deadlock before adding the synchronized on connectionId change.

@HuangZhenQiu
Copy link
Contributor Author

HuangZhenQiu commented Jun 19, 2020

@rkhachatryan
Thanks for giving feedback on the PR. I think the
PR is to tackle different things compare to taskmanager.network.request-backoff.xxx. taskmanager.network.request-backoff.xxx is translated into delayMs in NettyPartitionRequestClient to request sub Partition. It is after the TCP connection is built. The retry added here is to make TCP connection creation more reliable. It is not easy to combine it together. But it is a good suggestion, Let's wait for @pnowojski's feedback.

Yes, you are right. The deadline is due to the UnstableNettyClient can't simulate the netty bootstrap to notify the exception to ConnectingChannel. Currently, the exception is thrown in connect function, so that we need to explicitly notify it.

@HuangZhenQiu
Copy link
Contributor Author

HuangZhenQiu commented Jun 19, 2020

@pnowojski
Would you please help to trigger to Azure test?

@rkhachatryan
Copy link
Contributor

Did you test the latest version that has added synchronized on connectionID

Yes, I'm running a version with synchronized (connectionId) - 10..100 times testNettyClientConnectRetryMultipleThread.
I see from debug the cause is replace (If it's not my local issue, let's see if azure fails)

The Azure test has been not triggered it yet.

I referred to this failure of 2ff2e89 published by @flinkbot: https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=3462&view=results

@flinkbot run azure

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch 2 times, most recently from 4d30a9d to dcfd168 Compare June 19, 2020 18:24
Copy link
Contributor

@rkhachatryan rkhachatryan left a comment

Choose a reason for hiding this comment

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

I think I found the cause of deadlock, please see comments below.

Just a side note, I think we wouldn't have all these concurrency issues if we use ConcurrentHashMap "idiomatically" from the beginning, something like this:

ConcurrentMap<ConnectionID, Future<NettyPartitionRequestClient>> clients = new ConcurrentHashMap<>();

return clients.computeIfAbsent(connectionId, unused -> {
    int tried = 0;
    while (true) {
        try {
            ConnectingChannel connectingChannel = new ConnectingChannel(connectionId, this);
            nettyClient.connect(connectionId.getAddress()).addListener(connectingChannel);
            return connectingChannel.waitForChannel();
        } catch (IOException e) {
            if (++tried > retryNumber) {
                throw new CompletionException(e);
            }
        }
    }
}).get();

@@ -60,49 +71,41 @@ NettyPartitionRequestClient createPartitionRequestClient(ConnectionID connection
while (client == null) {
entry = clients.get(connectionId);
Copy link
Contributor

Choose a reason for hiding this comment

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

I found it to be the cause of deadlock:

  1. entry is placed by A
  2. entry is read by B
  3. A gets the connection and replaces entry - but A already holds an old entry
  4. B acquires lock and waits for channel indefinitely

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, you are right. Usually the exception can be notified to the old connecting channel by bootstrap. The test case actually triggers the exception within the connect function, thus we need to notify the exception explicitly.

nettyClient.connect(connectionId.getAddress()).addListener(connectingChannel);

client = connectingChannel.waitForChannel();
synchronized (connectionId) {
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 it's not safe to synchronize on this because it's passed from the outside.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, you are right. Java doesn't allow to synchronize on a variable.

@HuangZhenQiu
Copy link
Contributor Author

HuangZhenQiu commented Jun 19, 2020

@rkhachatryan
I think the solution you proposed is much better. Let me update it.

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch 3 times, most recently from f231abd to 3b7bc2e Compare June 19, 2020 19:09
@HuangZhenQiu
Copy link
Contributor Author

@pnowojski @rkhachatryan
I think what @rkhachatryan proposed is the most concise and scalable solution. I rebased master with some small changes. Please take a look.

@HuangZhenQiu
Copy link
Contributor Author

@flinkbot run azure

@HuangZhenQiu
Copy link
Contributor Author

@flinkbot run travis

@HuangZhenQiu HuangZhenQiu force-pushed the FLINK-15416-add-netty-connect-retry branch from 3b7bc2e to a8e0437 Compare June 21, 2020 05:40
@HuangZhenQiu
Copy link
Contributor Author

@flinkbot run travis

@HuangZhenQiu
Copy link
Contributor Author

@flinkbot run azure

}
else {
client = (NettyPartitionRequestClient) old;
client = (NettyPartitionRequestClient) clients.computeIfAbsent(connectionId, unused -> {
Copy link
Contributor

Choose a reason for hiding this comment

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

I indeed meant to use Futures as map values, because otherwise all operations on map (or it's part) are blocked for the duration of connection.
I implemented this in #12746 based on your changes. Please take a look.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
6 participants