-
Notifications
You must be signed in to change notification settings - Fork 1.5k
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
Make http client restart requests #1883
Conversation
@nyh , @tchaikov this is the restart implementation without magic numbers (but more hairy) as I mentioned here. Mind taking a look. Patches 1 through 5 are identical in both PRs. I do not yet have strong preference to either approach, both help S3 client recover from synchronously dropped connections |
691f3db
to
2f73829
Compare
upd:
|
sorry for the latency, will do in this week. |
@tchaikov , thank you for the review. I'd also like you to share your thoughts about which restarting mechanism you like more -- this one (that replays the request once over fresh new connection) or the #1847 approach (that implements backoff-retry and that may potentially use other connections from pool) |
sure. will do in this week. |
i'd prefer retrying with backoff. and i don't have strong preference without good reasoning. i'd go with a
a default implementation of |
ּSorry, I'll review it now. |
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.
Looks fairly good, but I have some comments and requests.
One bigger thing that bothers me is why the "automatic restart" of a request is not optional. I'm pretty sure that not all users of Seastar's HTTP client API will always have an idempotent use case - some will want to know when a request failed in the middle and not restart the same request - and instead create a new, potentially different, request.
I think you yourself told me once about a use case where if the request fails after a long time (e.g., imagine uploading a large file failed), the application needs to know this to sign a new request with the new current time - it can't just replay exactly the same request.
src/http/client.cc
Outdated
// be held in pool anyway. Still, this is unlikely, _nr_connections | ||
// decrease will happen in a waiting task and this place is chained | ||
// directly after the put_connection() | ||
return do_make_request(req, handle, expected, force_new_connection::yes); |
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 don't understand what's going on here. I understand that you want to close the existing connection (which apparently broke), and run the request again, probably on a different connection. But why does it need to be a new connection? Why can't we reuse an existing connection already lying in the pool and it must be a new connection? Why will this new connection "not be held in the pool anyway" - we'll only use it once?
Also, I don't understand the discussion about concurrent requests and limit. If a request is made, and there is no open connection, we have to create a new connection - we can't "limit" this number. We can limit the number of unused connections we cache, but how is it relevant here?
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.
But why does it need to be a new connection? Why can't we reuse an existing connection already lying in the pool and it must be a new connection? Why will this new connection "not be held in the pool anyway" - we'll only use it once?
That's exactly the question we're trying to resolve. There are currently two implementation of "retry failed http requests". This PR and the #1847 one. This PR retries request over fresh new connection. This approach is good, because it doesn't have any magic parameters from the caller (well, maybe a boolean flag whether to do it at all or not) -- it just retries the request once over new connection and that's it. THe #1847 implements request retrying in the "backoff-retry" manner. I.e. if a request fails it waits for a while, then picks up another connection (and in that case this another connection can be from pool or can be a new one, depending on the pool state) and tries request again. And does this several times. That approach needs configuration from the caller -- the number of retrying attempts and the pause duration between retries.
Also, I don't understand the discussion about concurrent requests and limit. If a request is made, and there is no open connection, we have to create a new connection - we can't "limit" this number.
We don't
We can limit the number of unused connections we cache, but how is it relevant here?
It just says that when the do_make_request() failed the connection that served that request is still alive at this place and it will only get closed few continuations after that. By the time this connection gets destroyed, request retrying would likely create a new connection without checking the limit thus breaking this limit for a short period of time. It's not critical, just a clarification comment.
include/seastar/http/client.hh
Outdated
template <typename Fn> | ||
SEASTAR_CONCEPT( requires std::invocable<Fn, connection&> ) | ||
auto with_connection(Fn&& fn); | ||
auto with_connection(Fn&& fn, force_new_connection force_new); |
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 didn't understand the need for the "force_new" option. I don't think it's needed - see my longer comment below.
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.
As I wrote there, there's another approach to restarting requests (#1847 ) in the backoff-retry manner. In it you asked why backoff-retry makes any sense, so here's another approach that just creates a new connection and retries via it.
Both approaches work, but in #1847 retrying a request via cached connection may fail several times in a row, so there's "maximum number of retry attempts" magic configuration constant. This approach doesn't need it. So now we need to decide which way to go -- this or that (or some other third one)
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.
Giving two options, this or #1847, is a https://en.wikipedia.org/wiki/False_dilemma - let's try to think if anything needs improving in this approach, instead of discussing whether it is better as a whole than some other as-a-whole approach.
The question of whether we need to retry just once or multiple times is, I think, completely orthogonal to where the connection comes from... A retried request may fail for a second time on a recycled connection or a brand-new connection, just the same.
But I think I finally understood now. Perhaps your point is not to complicate this API with the notion of a "pool" at all. So this API only gets a single connection, and if this connection fails, we need to create a new one. It doesn't have any notion of pool it can use.
If this is the explanation, I think the code could be simplified: It doesn't need a "force_new_connection" flag. Instead, before the recursive call to do_make_request() on retry, the caller can open the new connection - and hand the function the fresh connection - the callee doesn't need to care if it was called on a fresh connection or not.
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.
Giving two options, this or #1847, is a https://en.wikipedia.org/wiki/False_dilemma - let's try to think if anything needs improving in this approach, instead of discussing whether it is better as a whole than some other as-a-whole approach.
Sure
The question of whether we need to retry just once or multiple times is, I think, completely orthogonal to where the connection comes from... A retried request may fail for a second time on a recycled connection or a brand-new connection, just the same.
That's not so. At least from the statistical point of view. In my testing of S3 client, when a request fails in a connection from pool (i.e. this connection had served at least one successful requests before the failing one) chances that the next pooled connection fails too are pretty high. Neither 0, nor 1, but somewhere in between. Opposite to that, retrying a request over a fresh new connection never failed in my tests.
But I think I finally understood now. Perhaps your point is not to complicate this API with the notion of a "pool" at all.
The make_request() API doesn't reveal pool concept to the caller. And the force_new_connection thing is private type of the client class. My goal in this PR is not to complicate the make_request() API with "retry config" thing. If doing backoff-retry retrying once doesn't help in pretty large amount of cases, in my tests I had to take 1, 2 or sometimes 3 retries to make S3 uploading test pass. Retrying infinitely is also not good, so there must be some magic "maximum number of retries" configuration parameter.
However, I think you're right that we probably can get rid of the force_new_connection flag.
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.
The question of whether we need to retry just once or multiple times is, I think, completely orthogonal to where the connection comes from... A retried request may fail for a second time on a recycled connection or a brand-new connection, just the same.
That's not so. At least from the statistical point of view. In my testing of S3 client, when a request fails in a connection from pool (i.e. this connection had served at least one successful requests before the failing one) chances that the next pooled connection fails too are pretty high. Neither 0, nor 1, but somewhere in between. Opposite to that, retrying a request over a fresh new connection never failed in my tests.
I think I understand what might explain this statistical observation: Basically what you're saying that if you have in your pool connection A and B to the same server, and connection B broke in the middle, there is a chance that connection A broke too - e.g., perhaps because some load balancer in front of the server crashed and caused many different connections to fail.
I have two problems with that:
- It's very strange as a general (Seastar) feature, I don't think this mode of failure is something you'd expect to see in all use cases.
- But, if you do want to handle this case (assume that maybe all prior connections broke), please write about this - in a comment and commit message - explicitly. E.g., maybe something like my paragraph above. Otherwise, it's impossible to guess why you did all this request counting, pool special casing, and so on.
Finally, I have another question: If you have 100 connections in your pool and one broke, you open a new one for the retry because you think you can't trust any of the connections in the pool. But... You still have 99 other connections in the pool, and you don't trust them... So you didn't do the retry on one of these connections, but 99 future requests may try to use these connections and fail. So maybe, if you decided that the pool can't be trusted after a failure, maybe you need to close all of the connections? Or, alternatively, if you think it's too drastic to not reuse all these 99 connections, why was it important to not retry on one of these connections? Why are we worried that another retry will fail (and will be re-retried again!), more than we are worried about a new original request failing?
But I think I finally understood now. Perhaps your point is not to complicate this API with the notion of a "pool" at all.
The make_request() API doesn't reveal pool concept to the caller. And the force_new_connection thing is private type of the client class. My goal in this PR is not to complicate the make_request() API with "retry config" thing. If doing backoff-retry retrying once doesn't help in pretty large amount of cases, in my tests I had to take 1, 2 or sometimes 3 retries to make S3 uploading test pass. Retrying infinitely is also not good, so there must be some magic "maximum number of retries" configuration parameter.
I didn't understand - are you saying that this code retries only once, that it retries infinitely, or what?
And please don't refer me to other versions of this PR and what they do - I'm confused already as is :-) Let's discuss the features of this PR and whether it does the right thing or not, and if not how it can be improved incrementally, not replacing it by a completely different implementation.
However, I think you're right that we probably can get rid of the force_new_connection flag.
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 think I understand what might explain this statistical observation: Basically what you're saying that if you have in your pool connection A and B to the same server, and connection B broke in the middle, there is a chance that connection A broke too - e.g., perhaps because some load balancer in front of the server crashed and caused many different connections to fail.
Yes.
And important thing here -- "there's a chance". I'll get back to why this is important a bit later.
Finally, I have another question: If you have 100 connections in your pool and one broke, you open a new one for the retry because you think you can't trust any of the connections in the pool. But... You still have 99 other connections in the pool, and you don't trust them...
Not quite. As I write above -- "there's a chance", so ... (see below)
So you didn't do the retry on one of these connections, but 99 future requests may try to use these connections and fail. So maybe, if you decided that the pool can't be trusted after a failure, maybe you need to close all of the connections? Or, alternatively, if you think it's too drastic to not reuse all these 99 connections, why was it important to not retry on one of these connections? Why are we worried that another retry will fail (and will be re-retried again!), more than we are worried about a new original request failing?
So :) I still trust the rest of the pool. In its first version this "restart http requests on transport error" PR had another retry logic -- it retried requests over connections from pool, but in that case the 1st retry was sometimes enough and sometimes it wasn't. In the end of the day it could take 1, 2, 3, ... $pool_size retries to succeed (I didn't collect the historgram, sorry) and sometimes the pool became empty and a new connection was created. In other words -- retrying a request over some other connection from pool has non-zero chance to fail and non-zero chance to succeed and it takes time to find it out.
I didn't understand - are you saying that this code retries only once, that it retries infinitely, or what?
This code retries only once over new connection.
Previous version retried several times over connections from pool (or, if the pool exhausts, over new connection). In order not to retry infinitely I broke the look explicitly after N retries. There could be another loop breakage condition -- once the pool is over and we retry over new connection and it fails -- stop retrying.
And please don't refer me to other versions of this PR and what they do - I'm confused already as is :-)
I can't :) You're asking question like "what if we ..." and my answer is "we already did, and ...". But I'll try to
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.
@nyh , reminder ping
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 don't know what to do about this PR (let's try to get a second reviewer). My problem is that I am still not convinced that this approach is more than an ad-hoc band-aid for what you saw in S3 and really belongs in a general-purpose HTTP library in Seastar.
Moreover, I'm not sure I understand exactly what this PR is trying to do (you know the "it's hard to see the forest from the trees" problem) - can you please rewrite the cover letter to include a full explanation of what this implementation does, in a high level? Only now I understood for the first time that "this code retries only once over new connection" - I thought that this can happen more than once. As a mathematician who learned probability, I find it a bit odd (if there's a 1% chance that a request fails, then there's a %0.01 that the second request will fail too!). But I guess I can understand this simplification if it is documented well. But... I still don't understand the entire pool, statics, and so on, business. It sounds to me like like this code takes one connection an returns one connection - perhaps the same one, or perhaps a different one (if it closed a broken connection and opened a new one). Why does it need to keep counts?
Again, I suggest that instead of trying to answer my specific question, please write a longer and self-contained cover letter explaining what this PR does, exactly (by "exactly" I don't mean referring to code, but exactly what the algorithm is doing; By "self-contained" I mean assuming readers don't know exactly what other pieces like the pool do).
And/or, please get a second reviewer (@tchaikov) with a fresh perspective who maybe will understand this PR better.
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.
OK, I wrote a longer story in the description. I cannot avoid mentioning pool in it, because effectively the restart handles an issue that pops up on pooled connections
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.
As a mathematician who learned probability, I find it a bit odd (if there's a 1% chance that a request fails, then there's a %0.01 that the second request will fail too!)
This implies that the probability of all events is the same, but it's not. The probability of "request fails over connection from pool is 1%", while probability of "request fails over new connection" is different. Tying to "measure" it gives 0, which is, of course, a nonsense. Still it appears to be MUCH lower than for "... over connection from pool", provided there's no infrastructure problems
2f73829
to
085d805
Compare
upd:
|
085d805
to
0d5a4ae
Compare
upd:
|
1c20f8d
to
1461e16
Compare
upd:
|
@nyh , let's try to resurrect this PR. I rebased it, tuned up the description and resolved all the controversial comments from the previous iterations (the last one left is about the approach itself) |
Ok, I'll try to to fairly re-review this PR from scratch. Earlier in #1883 (comment) I suggested that a different reviewer should be brought in, with a fresh view, but at this point I don't remember what I said and what I didn't like so maybe my own view will be fresh as well ;-) |
Before reading the code, I want to point out that the new commit message was very helpful, and I feel like I finally understand what you're doing. It even makes a lot of sense if the only kind of error you want to handle is the case of the server closing an inactive connection without the client noticing, and in that case you convinced me that what had looked to me like arbitrary rules actually makes sense. Plus I like the fact that there seems to be fewer arbitrary rules here (no "first request on connection" special handling, etc.) What still bothers me a bit that this is not really the only kind of error that exists in the wild for an HTTP client. In a long request (e.g., some very long upload), there is a non-zero probability that the connection will break in the middle of the connection - not because we missed a closing (the scenario you considered), but really because of a server problem. So if there's a p=0.001% chance that a 1GB transfer will fail in the middle, there is a p^2= 0.000001% chance both the original transfer and its retry will also fail - which is small but not zero, so the decision to retry only once is less justified for this case. But I do agree that if the API can retry once and also respond with a failure and let the client retry even more times, that's fine too. Other APIs also have such limitations - for example the boto3 API for DynamoDB retries a temporary failure, by default, 3 times, without returning to the user. Why 3 and not 7? It doesn't matter. If it fails 3 retries it will return the failure to the user, the the user can try 3 more times if he wants. Now I'll start reading the code. |
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.
Looks good to me, I only have minor requests about some code comments.
Another thing I'm missing is any sort of testing (this PR seems to include none). You added new exception for the failure case, but didn't add a test for it. You're adding an elaborate retry mechanism, but don't have a test for it.
If you believe it's too hard to test in a unit test (this is not good, but sometimes it's the honest truth), I think you'll at least need to explain how you tested this new feature manually. The code looks correct to me, but who knows, does it actually work when encountering an error, or there is something missing that is hard to catch in a code review?
The method actualy shuts the socket down the hard way. There will appear a less disruptive call, so the existing one deserves some better name. Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
1461e16
to
26d6516
Compare
upd:
|
When a socket is closed after noticing the EOF message in its shared buffer, it doesn't resolve the "shutdown" future. This is not very nice, because socket is indeed dead, and no other code would resolve it, so it's good to do it on close(). In fact, it is the peer who has to resolve this future, but it should happen only after the queue with messages is drained and it complicates the logic. Resolving it on close is good enough. In the non-EOF case the socket is aborted, which also resolve the aforementioned future. Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
There's such a wrapper around loopback connection factory that fits it to http client needs. Several tests copy-and-pase it, new tests would appear soon, it's time to generalize it. Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
It was added by 15942b9 (http/client: Count and limit the number of connections) but wasn't documented, so do it now. Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
Currently consuming http response doesn't check if parser failed reading the response check. Add it. Also, when parser sees correct, but incomplete response it sets itself to .eof(), not .failed(). This case should be told from invalid, so the system error with ECONABORTED status is thrown instead. Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
This needs splitting connection::get_connection() into get_ and make_connection. Both to facilitate next patching Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
Currently request is passed by value all the way down to connection where it's moved to stable memory with do_with(std::move(req), ...) and then processing continues. Next patches are going to teach client restart requests over different connections, so the request should be moved into do_with-context at the client side. While at it -- keep the response handler in the stable context as well, next patching will appreciate this change. Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
Currently if client::make_request() fails, the error is propagated back to the caller whatever the error is. Sometimes the request fails because of connection glitch, which can happen and in that case it's common to try the same request again after a while in a hope that it was indeed some teporary glitch. Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
26d6516
to
3c5c7ea
Compare
upd:
|
_aborted = true; | ||
_q.abort(std::make_exception_ptr(std::system_error(EPIPE, std::system_category()))); | ||
} | ||
void shutdown() noexcept { |
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 got lost here. You renamed the old shutdown() to abort, and now you're re-introducing shutdown()? Or maybe it's a different overload with the same name?
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.
Well, yes, I'm introducing another shutdown(). Old shutdown (that got renamed to abort) now calls new shutdown and aborts the inner queue
@@ -220,7 +220,14 @@ public: | |||
* may re-use the sockets on its own | |||
* | |||
* \param f -- the factory pointer | |||
* \param max_connections -- maximum number of connection a client is allowed to maintain |
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 don't understand why we wanted such a feature... What happens if the user starts another request? It waits until one of the older request finishes? Why?
Anyway, you're just documenting a feature we already have, so ok.
I can understand a maximum number of connections cached in the pool, that's useful. But why do we want to artificially limit the total number of connections?
Although I guess you're just documenting this now,
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.
What happens if the user starts another request? It waits until one of the older request finishes? Why?
If max_connections
is already serving other requests (and pool is empty) then yes. Because this knob limits the number of sockets opened simultaneously. Limiting only the number of connections in pool is also useful, but I think concurrency control is also required for active connections (even though they are supposedly short-living)
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.
And yes, it's the pre-existing feature from #1652 (you approved it back then 😉 )
The new version looks really good (and well-tested), thanks. I just wrote a few comments for your consideration (I tentatively "approved" it). |
Real S3 server is known to actively close connections, thus breaking S3 storage backend at random places. The recent http client update is more robust against that, but the needed feature is OFF by default. refs: scylladb/seastar#1883 Signed-off-by: Pavel Emelyanov <xemul@scylladb.com>
Real S3 server is known to actively close connections, thus breaking S3 storage backend at random places. The recent http client update is more robust against that, but the needed feature is OFF by default. refs: scylladb/seastar#1883 Signed-off-by: Pavel Emelyanov <xemul@scylladb.com> Closes #19461
There's an issue with maintaining keep-alive sockets by http client.
Some prior art first.
The HTTP client uses seastar connections to send requests over and receive responses back. Typically the connection is a TCP socket (or TLS connection which is a wrapper over TCP socket too). Once request-response cycle is over the connection used for that is kept by a client in a "pool". Making another http request may then pick up the existing connection from the pool thus avoiding the extra latency of establishing new connection. Pool may thus accumulate more than one connection if the client user sends several requests in parallel.
HTTP servers may sometimes want to terminate the connections it keeps. This can happen in one of several ways.
The "gentle" way is when server adds the "connection: close" header to its response. In that case client would handle the response and will not put the connection back to pool, but instead would just close it. So next request would either pick some other connection from pool or would need to establish a new one.
Less gentle way a server may terminate a connection is by closing it, so the underlying TCP stack would communicate regular TCP FIN-s. On the client side there's connected_socket::wait_input_shutdown() method that returns a future that gets resolved when kernel terminates the connection by peers request. In case client's connection is kept in pool it will be closed and removed from the pool behind the scenes. Next request won't even notice that -- it will either pick some other connection from pool, or will establish a new one.
Sometimes more unpleasant situation happens. It can be either a race or deliberate server's "abort". In the former case, server closes the connection and TCP starts communicating FIN-s in parallel with client picking up a connection for its new request. In that case, even if kernel resolves "input-shutdown" event described above, due to seastar event loop and batch flusher, client would see that the connection is closed after it had picked it from pool and had chance to put some data into it. In the latter case server closes the connection in the middle of reading the request from the client or even in the middle of writing back the response. This is very unlikely, but still happens from time to time.
Having said the above, the problem -- when user calls
client::make_request()
and client steps on the "unpleasant" server-side connection closure, the request making resolves with exception and user has to do something about it. There are two options here -- handle the exception somehow or ask client to make the same request again (restart the request). This PR suggest that the latter choice can be implemented in the HTTP client code. If user doesn't want to restart, it may ask client not to do it, the new API allows for that.First (a side node) -- to tell the "server closed its socket" from other errors the exception from scoket IO is checked to be the system error with EPIPE or ECONABORTED code in it. EPIPE is reported when it comes from writing the request, ECONABORTED is reported when it comes from reading the response.
Next, there's only one way to replay the request -- client should get new connection somehow and repeat the request-response cycle. There are two options where to get new connection from -- from pool (if it's there) or establish a new one. While experimenting with HTTP client I noticed that picking up connection from pool often results in several "transport exception"-s in a row, as if server was closing connections in batches. So this PR makes a shortcut for restarts it always establishes new connection to server (which, after request-response cycle, is put back into pool normally).
And the last, but not least, restart only happens once, because the chance of observing "transport exception" from newly established connection is considered to be low enough not to care. Respectively, if a request is made over new connection (not over a connection from pool) and "transport exception" pops up it's not restarted.