client-go: refactor retry logic so it can be reused

Kubernetes-commit: 5fdf196b4e9bbba036a43f4c3b5d9ed8af1672cc
This commit is contained in:
Abu Kashem 2021-05-20 16:54:19 -04:00 committed by Kubernetes Publisher
parent 228dada995
commit 239ec44d74
4 changed files with 797 additions and 98 deletions

View File

@ -93,7 +93,6 @@ type Request struct {
rateLimiter flowcontrol.RateLimiter
backoff BackoffManager
timeout time.Duration
maxRetries int
// generic components accessible via method setters
verb string
@ -110,8 +109,9 @@ type Request struct {
subresource string
// output
err error
body io.Reader
err error
body io.Reader
retry WithRetry
}
// NewRequest creates a new request helper object for accessing runtime.Objects on a server.
@ -142,7 +142,7 @@ func NewRequest(c *RESTClient) *Request {
backoff: backoff,
timeout: timeout,
pathPrefix: pathPrefix,
maxRetries: 10,
retry: &withRetry{maxRetries: 10},
warningHandler: c.warningHandler,
}
@ -408,10 +408,7 @@ func (r *Request) Timeout(d time.Duration) *Request {
// function is specifically called with a different value.
// A zero maxRetries prevent it from doing retires and return an error immediately.
func (r *Request) MaxRetries(maxRetries int) *Request {
if maxRetries < 0 {
maxRetries = 0
}
r.maxRetries = maxRetries
r.retry.SetMaxRetries(maxRetries)
return r
}
@ -842,6 +839,17 @@ func (r *Request) requestPreflightCheck() error {
return nil
}
func (r *Request) newHTTPRequest(ctx context.Context) (*http.Request, error) {
url := r.URL().String()
req, err := http.NewRequest(r.verb, url, r.body)
if err != nil {
return nil, err
}
req = req.WithContext(ctx)
req.Header = r.headers
return req, nil
}
// request connects to the server and invokes the provided function when a server response is
// received. It handles retry behavior and up front validation of requests. It will invoke
// fn at most once. It will return an error if a problem occurred prior to connecting to the
@ -881,27 +889,22 @@ func (r *Request) request(ctx context.Context, fn func(*http.Request, *http.Resp
}
// Right now we make about ten retry attempts if we get a Retry-After response.
retries := 0
var retryInfo string
var retryAfter *RetryAfter
for {
url := r.URL().String()
req, err := http.NewRequest(r.verb, url, r.body)
req, err := r.newHTTPRequest(ctx)
if err != nil {
return err
}
req = req.WithContext(ctx)
req.Header = r.headers
r.backoff.Sleep(r.backoff.CalculateBackoff(r.URL()))
if retries > 0 {
if retryAfter != nil {
// We are retrying the request that we already send to apiserver
// at least once before.
// This request should also be throttled with the client-internal rate limiter.
if err := r.tryThrottleWithInfo(ctx, retryInfo); err != nil {
if err := r.tryThrottleWithInfo(ctx, retryAfter.Reason); err != nil {
return err
}
retryInfo = ""
retryAfter = nil
}
resp, err := client.Do(req)
updateURLMetrics(ctx, r, resp, err)
@ -910,61 +913,46 @@ func (r *Request) request(ctx context.Context, fn func(*http.Request, *http.Resp
} else {
r.backoff.UpdateBackoff(r.URL(), err, resp.StatusCode)
}
if err != nil {
// "Connection reset by peer" or "apiserver is shutting down" are usually a transient errors.
// Thus in case of "GET" operations, we simply retry it.
// We are not automatically retrying "write" operations, as
// they are not idempotent.
if r.verb != "GET" {
return err
}
// For connection errors and apiserver shutdown errors retry.
if net.IsConnectionReset(err) || net.IsProbableEOF(err) {
// For the purpose of retry, we set the artificial "retry-after" response.
// TODO: Should we clean the original response if it exists?
resp = &http.Response{
StatusCode: http.StatusInternalServerError,
Header: http.Header{"Retry-After": []string{"1"}},
Body: ioutil.NopCloser(bytes.NewReader([]byte{})),
}
} else {
return err
}
}
done := func() bool {
// Ensure the response body is fully read and closed
// before we reconnect, so that we reuse the same TCP
// connection.
defer func() {
const maxBodySlurpSize = 2 << 10
if resp.ContentLength <= maxBodySlurpSize {
io.Copy(ioutil.Discard, &io.LimitedReader{R: resp.Body, N: maxBodySlurpSize})
}
resp.Body.Close()
}()
defer readAndCloseResponseBody(resp)
retries++
if seconds, wait := checkWait(resp); wait && retries <= r.maxRetries {
retryInfo = getRetryReason(retries, seconds, resp, err)
if seeker, ok := r.body.(io.Seeker); ok && r.body != nil {
_, err := seeker.Seek(0, 0)
if err != nil {
klog.V(4).Infof("Could not retry request, can't Seek() back to beginning of body for %T", r.body)
fn(req, resp)
return true
}
// if the the server returns an error in err, the response will be nil.
f := func(req *http.Request, resp *http.Response) {
if resp == nil {
return
}
fn(req, resp)
}
klog.V(4).Infof("Got a Retry-After %ds response for attempt %d to %v", seconds, retries, url)
r.backoff.Sleep(time.Duration(seconds) * time.Second)
var retry bool
retryAfter, retry = r.retry.NextRetry(req, resp, err, func(req *http.Request, err error) bool {
// "Connection reset by peer" or "apiserver is shutting down" are usually a transient errors.
// Thus in case of "GET" operations, we simply retry it.
// We are not automatically retrying "write" operations, as they are not idempotent.
if r.verb != "GET" {
return false
}
// For connection errors and apiserver shutdown errors retry.
if net.IsConnectionReset(err) || net.IsProbableEOF(err) {
return true
}
return false
})
if retry {
if err := r.retry.BeforeNextRetry(ctx, r.backoff, retryAfter, req.URL.String(), r.body); err != nil {
klog.V(4).Infof("Could not retry request - %v", err)
f(req, resp)
return true
}
return false
}
fn(req, resp)
f(req, resp)
return true
}()
if done {
return nil
return err
}
}
}
@ -1196,19 +1184,6 @@ func isTextResponse(resp *http.Response) bool {
return strings.HasPrefix(media, "text/")
}
// checkWait returns true along with a number of seconds if the server instructed us to wait
// before retrying.
func checkWait(resp *http.Response) (int, bool) {
switch r := resp.StatusCode; {
// any 500 error code and 429 can trigger a wait
case r == http.StatusTooManyRequests, r >= 500:
default:
return 0, false
}
i, ok := retryAfterSeconds(resp)
return i, ok
}
// retryAfterSeconds returns the value of the Retry-After header and true, or 0 and false if
// the header was missing or not a valid number.
func retryAfterSeconds(resp *http.Response) (int, bool) {
@ -1220,26 +1195,6 @@ func retryAfterSeconds(resp *http.Response) (int, bool) {
return 0, false
}
func getRetryReason(retries, seconds int, resp *http.Response, err error) string {
// priority and fairness sets the UID of the FlowSchema associated with a request
// in the following response Header.
const responseHeaderMatchedFlowSchemaUID = "X-Kubernetes-PF-FlowSchema-UID"
message := fmt.Sprintf("retries: %d, retry-after: %ds", retries, seconds)
switch {
case resp.StatusCode == http.StatusTooManyRequests:
// it is server-side throttling from priority and fairness
flowSchemaUID := resp.Header.Get(responseHeaderMatchedFlowSchemaUID)
return fmt.Sprintf("%s - retry-reason: due to server-side throttling, FlowSchema UID: %q", message, flowSchemaUID)
case err != nil:
// it's a retriable error
return fmt.Sprintf("%s - retry-reason: due to retriable error, error: %v", message, err)
default:
return fmt.Sprintf("%s - retry-reason: %d", message, resp.StatusCode)
}
}
// Result contains the result of calling Request.Do().
type Result struct {
body []byte

View File

@ -1093,6 +1093,7 @@ func TestRequestWatch(t *testing.T) {
for _, testCase := range testCases {
t.Run("", func(t *testing.T) {
testCase.Request.backoff = &NoBackoff{}
testCase.Request.retry = &withRetry{}
watch, err := testCase.Request.Watch(context.Background())
hasErr := err != nil
if hasErr != testCase.Err {
@ -1194,8 +1195,10 @@ func TestRequestStream(t *testing.T) {
},
},
}
for i, testCase := range testCases {
testCase.Request.backoff = &NoBackoff{}
testCase.Request.retry = &withRetry{maxRetries: 0}
body, err := testCase.Request.Stream(context.Background())
hasErr := err != nil
if hasErr != testCase.Err {
@ -1274,6 +1277,7 @@ func TestRequestDo(t *testing.T) {
}
for i, testCase := range testCases {
testCase.Request.backoff = &NoBackoff{}
testCase.Request.retry = &withRetry{}
body, err := testCase.Request.Do(context.Background()).Raw()
hasErr := err != nil
if hasErr != testCase.Err {
@ -1436,8 +1440,8 @@ func TestConnectionResetByPeerIsRetried(t *testing.T) {
return nil, &net.OpError{Err: syscall.ECONNRESET}
}),
},
backoff: backoff,
maxRetries: 10,
backoff: backoff,
retry: &withRetry{maxRetries: 10},
}
// We expect two retries of "connection reset by peer" and the success.
_, err := req.Do(context.Background()).Raw()
@ -2315,3 +2319,285 @@ func TestRequestMaxRetries(t *testing.T) {
})
}
}
type responseErr struct {
response *http.Response
err error
}
type seek struct {
offset int64
whence int
}
type count struct {
// keeps track of the number of Seek(offset, whence) calls.
seeks []seek
// how many times {Request|Response}.Body.Close() has been invoked
closes int
}
// used to track {Request|Response}.Body
type readTracker struct {
count *count
delegated io.Reader
}
func (r *readTracker) Seek(offset int64, whence int) (int64, error) {
if seeker, ok := r.delegated.(io.Seeker); ok {
r.count.seeks = append(r.count.seeks, seek{offset: offset, whence: whence})
return seeker.Seek(offset, whence)
}
return 0, io.EOF
}
func (r *readTracker) Read(p []byte) (n int, err error) {
return r.delegated.Read(p)
}
func (r *readTracker) Close() error {
if closer, ok := r.delegated.(io.Closer); ok {
r.count.closes++
return closer.Close()
}
return nil
}
func newReadTracker(count *count) *readTracker {
return &readTracker{
count: count,
}
}
func newCount() *count {
return &count{
closes: 0,
seeks: make([]seek, 0),
}
}
type readSeeker struct{ err error }
func (rs readSeeker) Read([]byte) (int, error) { return 0, rs.err }
func (rs readSeeker) Seek(int64, int) (int64, error) { return 0, rs.err }
func unWrap(err error) error {
if uerr, ok := err.(*url.Error); ok {
return uerr.Err
}
return err
}
// noSleepBackOff is a NoBackoff except it does not sleep,
// used for faster execution of the unit tests.
type noSleepBackOff struct {
*NoBackoff
}
func (n *noSleepBackOff) Sleep(d time.Duration) {}
func TestRequestWithRetry(t *testing.T) {
tests := []struct {
name string
body io.Reader
serverReturns responseErr
errExpected error
transformFuncInvokedExpected int
roundTripInvokedExpected int
}{
{
name: "server returns retry-after response, request body is not io.Seeker, retry goes ahead",
body: ioutil.NopCloser(bytes.NewReader([]byte{})),
serverReturns: responseErr{response: retryAfterResponse(), err: nil},
errExpected: nil,
transformFuncInvokedExpected: 1,
roundTripInvokedExpected: 2,
},
{
name: "server returns retry-after response, request body Seek returns error, retry aborted",
body: &readSeeker{err: io.EOF},
serverReturns: responseErr{response: retryAfterResponse(), err: nil},
errExpected: nil,
transformFuncInvokedExpected: 1,
roundTripInvokedExpected: 1,
},
{
name: "server returns retry-after response, request body Seek returns no error, retry goes ahead",
body: &readSeeker{err: nil},
serverReturns: responseErr{response: retryAfterResponse(), err: nil},
errExpected: nil,
transformFuncInvokedExpected: 1,
roundTripInvokedExpected: 2,
},
{
name: "server returns retryable err, request body is not io.Seek, retry goes ahead",
body: ioutil.NopCloser(bytes.NewReader([]byte{})),
serverReturns: responseErr{response: nil, err: io.ErrUnexpectedEOF},
errExpected: io.ErrUnexpectedEOF,
transformFuncInvokedExpected: 0,
roundTripInvokedExpected: 2,
},
{
name: "server returns retryable err, request body Seek returns error, retry aborted",
body: &readSeeker{err: io.EOF},
serverReturns: responseErr{response: nil, err: io.ErrUnexpectedEOF},
errExpected: io.ErrUnexpectedEOF,
transformFuncInvokedExpected: 0,
roundTripInvokedExpected: 1,
},
{
name: "server returns retryable err, request body Seek returns no err, retry goes ahead",
body: &readSeeker{err: nil},
serverReturns: responseErr{response: nil, err: io.ErrUnexpectedEOF},
errExpected: io.ErrUnexpectedEOF,
transformFuncInvokedExpected: 0,
roundTripInvokedExpected: 2,
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
var roundTripInvoked int
client := clientForFunc(func(req *http.Request) (*http.Response, error) {
roundTripInvoked++
return test.serverReturns.response, test.serverReturns.err
})
req := &Request{
verb: "GET",
body: test.body,
c: &RESTClient{
Client: client,
},
backoff: &noSleepBackOff{},
retry: &withRetry{maxRetries: 1},
}
var transformFuncInvoked int
err := req.request(context.Background(), func(request *http.Request, response *http.Response) {
transformFuncInvoked++
})
if test.roundTripInvokedExpected != roundTripInvoked {
t.Errorf("Expected RoundTrip to be invoked %d times, but got: %d", test.roundTripInvokedExpected, roundTripInvoked)
}
if test.transformFuncInvokedExpected != transformFuncInvoked {
t.Errorf("Expected transform func to be invoked %d times, but got: %d", test.transformFuncInvokedExpected, transformFuncInvoked)
}
if test.errExpected != unWrap(err) {
t.Errorf("Expected error: %v, but got: %v", test.errExpected, unWrap(err))
}
})
}
}
func TestRequestDoWithRetry(t *testing.T) {
testRequestWithRetry(t, func(ctx context.Context, r *Request) {
r.Do(ctx)
})
}
func TestRequestDORawWithRetry(t *testing.T) {
testRequestWithRetry(t, func(ctx context.Context, r *Request) {
r.DoRaw(ctx)
})
}
func testRequestWithRetry(t *testing.T, doFunc func(ctx context.Context, r *Request)) {
tests := []struct {
name string
verb string
body func() io.Reader
maxRetries int
serverReturns []responseErr
reqCountExpected *count
respCountExpected *count
}{
{
name: "server always returns retry-after response",
verb: "GET",
body: func() io.Reader { return bytes.NewReader([]byte{}) },
maxRetries: 2,
serverReturns: []responseErr{
{response: retryAfterResponse(), err: nil},
{response: retryAfterResponse(), err: nil},
{response: retryAfterResponse(), err: nil},
},
reqCountExpected: &count{closes: 0, seeks: make([]seek, 2)},
respCountExpected: &count{closes: 3, seeks: []seek{}},
},
{
name: "server always returns retryable error",
verb: "GET",
body: func() io.Reader { return bytes.NewReader([]byte{}) },
maxRetries: 2,
serverReturns: []responseErr{
{response: nil, err: io.EOF},
{response: nil, err: io.EOF},
{response: nil, err: io.EOF},
},
reqCountExpected: &count{closes: 0, seeks: make([]seek, 2)},
respCountExpected: &count{closes: 0, seeks: []seek{}},
},
{
name: "server returns success on the final retry",
verb: "GET",
body: func() io.Reader { return bytes.NewReader([]byte{}) },
maxRetries: 2,
serverReturns: []responseErr{
{response: retryAfterResponse(), err: nil},
{response: nil, err: io.EOF},
{response: &http.Response{StatusCode: http.StatusOK}, err: nil},
},
reqCountExpected: &count{closes: 0, seeks: make([]seek, 2)},
respCountExpected: &count{closes: 2, seeks: []seek{}},
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
respCountGot := newCount()
responseRecorder := newReadTracker(respCountGot)
var attempts int
client := clientForFunc(func(req *http.Request) (*http.Response, error) {
defer func() {
attempts++
}()
resp := test.serverReturns[attempts].response
if resp != nil {
responseRecorder.delegated = ioutil.NopCloser(bytes.NewReader([]byte{}))
resp.Body = responseRecorder
}
return resp, test.serverReturns[attempts].err
})
reqCountGot := newCount()
reqRecorder := newReadTracker(reqCountGot)
reqRecorder.delegated = test.body()
req := &Request{
verb: test.verb,
body: reqRecorder,
c: &RESTClient{
Client: client,
},
backoff: &noSleepBackOff{},
retry: &withRetry{maxRetries: test.maxRetries},
}
doFunc(context.Background(), req)
attemptsExpected := test.maxRetries + 1
if attemptsExpected != attempts {
t.Errorf("Expected retries: %d, but got: %d", attemptsExpected, attempts)
}
if !reflect.DeepEqual(test.reqCountExpected.seeks, reqCountGot.seeks) {
t.Errorf("Expected request body to have seek invocation: %v, but got: %v", test.reqCountExpected.seeks, reqCountGot.seeks)
}
if test.respCountExpected.closes != respCountGot.closes {
t.Errorf("Expected response body Close to be invoked %d times, but got: %d", test.respCountExpected.closes, respCountGot.closes)
}
})
}
}

228
rest/with_retry.go Normal file
View File

@ -0,0 +1,228 @@
/*
Copyright 2021 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rest
import (
"context"
"fmt"
"io"
"io/ioutil"
"net/http"
"time"
"k8s.io/klog/v2"
)
// IsRetryableErrorFunc allows the client to provide its own function
// that determines whether the specified err from the server is retryable.
//
// request: the original request sent to the server
// err: the server sent this error to us
//
// The function returns true if the error is retryable and the request
// can be retried, otherwise it returns false.
// We have four mode of communications - 'Stream', 'Watch', 'Do' and 'DoRaw', this
// function allows us to customize the retryability aspect of each.
type IsRetryableErrorFunc func(request *http.Request, err error) bool
func (r IsRetryableErrorFunc) IsErrorRetryable(request *http.Request, err error) bool {
return r(request, err)
}
// WithRetry allows the client to retry a request up to a certain number of times
// Note that WithRetry is not safe for concurrent use by multiple
// goroutines without additional locking or coordination.
type WithRetry interface {
// SetMaxRetries makes the request use the specified integer as a ceiling
// for retries upon receiving a 429 status code and the "Retry-After" header
// in the response.
// A zero maxRetries should prevent from doing any retry and return immediately.
SetMaxRetries(maxRetries int)
// NextRetry advances the retry counter appropriately and returns true if the
// request should be retried, otherwise it returns false if:
// - we have already reached the maximum retry threshold.
// - the error does not fall into the retryable category.
// - the server has not sent us a 429, or 5xx status code and the
// 'Retry-After' response header is not set with a value.
//
// if retry is set to true, retryAfter will contain the information
// regarding the next retry.
//
// request: the original request sent to the server
// resp: the response sent from the server, it is set if err is nil
// err: the server sent this error to us, if err is set then resp is nil.
// f: a IsRetryableErrorFunc function provided by the client that determines
// if the err sent by the server is retryable.
NextRetry(req *http.Request, resp *http.Response, err error, f IsRetryableErrorFunc) (*RetryAfter, bool)
// BeforeNextRetry is responsible for carrying out operations that need
// to be completed before the next retry is initiated:
// - if the request context is already canceled there is no need to
// retry, the function will return ctx.Err().
// - we need to seek to the beginning of the request body before we
// initiate the next retry, the function should return an error if
// it fails to do so.
// - we should wait the number of seconds the server has asked us to
// in the 'Retry-After' response header.
//
// If BeforeNextRetry returns an error the client should abort the retry,
// otherwise it is safe to initiate the next retry.
BeforeNextRetry(ctx context.Context, backoff BackoffManager, retryAfter *RetryAfter, url string, body io.Reader) error
}
// RetryAfter holds information associated with the next retry.
type RetryAfter struct {
// Wait is the duration the server has asked us to wait before
// the next retry is initiated.
// This is the value of the 'Retry-After' response header in seconds.
Wait time.Duration
// Attempt is the Nth attempt after which we have received a retryable
// error or a 'Retry-After' response header from the server.
Attempt int
// Reason describes why we are retrying the request
Reason string
}
type withRetry struct {
maxRetries int
attempts int
}
func (r *withRetry) SetMaxRetries(maxRetries int) {
if maxRetries < 0 {
maxRetries = 0
}
r.maxRetries = maxRetries
}
func (r *withRetry) NextRetry(req *http.Request, resp *http.Response, err error, f IsRetryableErrorFunc) (*RetryAfter, bool) {
if req == nil || (resp == nil && err == nil) {
// bad input, we do nothing.
return nil, false
}
r.attempts++
retryAfter := &RetryAfter{Attempt: r.attempts}
if r.attempts > r.maxRetries {
return retryAfter, false
}
// if the server returned an error, it takes precedence over the http response.
var errIsRetryable bool
if f != nil && err != nil && f.IsErrorRetryable(req, err) {
errIsRetryable = true
// we have a retryable error, for which we will create an
// artificial "Retry-After" response.
resp = retryAfterResponse()
}
if err != nil && !errIsRetryable {
return retryAfter, false
}
// if we are here, we have either a or b:
// a: we have a retryable error, for which we already
// have an artificial "Retry-After" response.
// b: we have a response from the server for which we
// need to check if it is retryable
seconds, wait := checkWait(resp)
if !wait {
return retryAfter, false
}
retryAfter.Wait = time.Duration(seconds) * time.Second
retryAfter.Reason = getRetryReason(r.attempts, seconds, resp, err)
return retryAfter, true
}
func (r *withRetry) BeforeNextRetry(ctx context.Context, backoff BackoffManager, retryAfter *RetryAfter, url string, body io.Reader) error {
// Ensure the response body is fully read and closed before
// we reconnect, so that we reuse the same TCP connection.
if ctx.Err() != nil {
return ctx.Err()
}
if seeker, ok := body.(io.Seeker); ok && body != nil {
if _, err := seeker.Seek(0, 0); err != nil {
return fmt.Errorf("can't Seek() back to beginning of body for %T", r)
}
}
klog.V(4).Infof("Got a Retry-After %s response for attempt %d to %v", retryAfter.Wait, retryAfter.Attempt, url)
if backoff != nil {
backoff.Sleep(retryAfter.Wait)
}
return nil
}
// checkWait returns true along with a number of seconds if
// the server instructed us to wait before retrying.
func checkWait(resp *http.Response) (int, bool) {
switch r := resp.StatusCode; {
// any 500 error code and 429 can trigger a wait
case r == http.StatusTooManyRequests, r >= 500:
default:
return 0, false
}
i, ok := retryAfterSeconds(resp)
return i, ok
}
func getRetryReason(retries, seconds int, resp *http.Response, err error) string {
// priority and fairness sets the UID of the FlowSchema
// associated with a request in the following response Header.
const responseHeaderMatchedFlowSchemaUID = "X-Kubernetes-PF-FlowSchema-UID"
message := fmt.Sprintf("retries: %d, retry-after: %ds", retries, seconds)
switch {
case resp.StatusCode == http.StatusTooManyRequests:
// it is server-side throttling from priority and fairness
flowSchemaUID := resp.Header.Get(responseHeaderMatchedFlowSchemaUID)
return fmt.Sprintf("%s - retry-reason: due to server-side throttling, FlowSchema UID: %q", message, flowSchemaUID)
case err != nil:
// it's a retryable error
return fmt.Sprintf("%s - retry-reason: due to retryable error, error: %v", message, err)
default:
return fmt.Sprintf("%s - retry-reason: %d", message, resp.StatusCode)
}
}
func readAndCloseResponseBody(resp *http.Response) {
if resp == nil {
return
}
// Ensure the response body is fully read and closed
// before we reconnect, so that we reuse the same TCP
// connection.
const maxBodySlurpSize = 2 << 10
defer resp.Body.Close()
if resp.ContentLength <= maxBodySlurpSize {
io.Copy(ioutil.Discard, &io.LimitedReader{R: resp.Body, N: maxBodySlurpSize})
}
}
func retryAfterResponse() *http.Response {
return &http.Response{
StatusCode: http.StatusInternalServerError,
Header: http.Header{"Retry-After": []string{"1"}},
}
}

230
rest/with_retry_test.go Normal file
View File

@ -0,0 +1,230 @@
/*
Copyright 2021 The Kubernetes Authors.
Licensed under the Apache License, Version 2.0 (the "License");
you may not use this file except in compliance with the License.
You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
*/
package rest
import (
"errors"
"net/http"
"reflect"
"testing"
"time"
"github.com/google/go-cmp/cmp"
)
var alwaysRetryError = IsRetryableErrorFunc(func(_ *http.Request, _ error) bool {
return true
})
var neverRetryError = IsRetryableErrorFunc(func(_ *http.Request, _ error) bool {
return false
})
func TestNextRetry(t *testing.T) {
fakeError := errors.New("fake error")
tests := []struct {
name string
attempts int
maxRetries int
request *http.Request
response *http.Response
err error
retryableErrFunc IsRetryableErrorFunc
retryExpected []bool
retryAfterExpected []*RetryAfter
}{
{
name: "bad input, response and err are nil",
maxRetries: 2,
attempts: 1,
request: &http.Request{},
response: nil,
err: nil,
retryExpected: []bool{false},
retryAfterExpected: []*RetryAfter{nil},
},
{
name: "zero maximum retry",
maxRetries: 0,
attempts: 1,
request: &http.Request{},
response: retryAfterResponse(),
err: nil,
retryExpected: []bool{false},
retryAfterExpected: []*RetryAfter{
{
Attempt: 1,
},
},
},
{
name: "server returned a retryable error",
maxRetries: 3,
attempts: 1,
request: &http.Request{},
response: nil,
err: fakeError,
retryableErrFunc: func(_ *http.Request, err error) bool {
if err == fakeError {
return true
}
return false
},
retryExpected: []bool{true},
retryAfterExpected: []*RetryAfter{
{
Attempt: 1,
Wait: time.Second,
Reason: "retries: 1, retry-after: 1s - retry-reason: due to retryable error, error: fake error",
},
},
},
{
name: "server returned a retryable HTTP 429 response",
maxRetries: 3,
attempts: 1,
request: &http.Request{},
response: &http.Response{
StatusCode: http.StatusTooManyRequests,
Header: http.Header{
"Retry-After": []string{"2"},
"X-Kubernetes-Pf-Flowschema-Uid": []string{"fs-1"},
},
},
err: nil,
retryExpected: []bool{true},
retryAfterExpected: []*RetryAfter{
{
Attempt: 1,
Wait: 2 * time.Second,
Reason: `retries: 1, retry-after: 2s - retry-reason: due to server-side throttling, FlowSchema UID: "fs-1"`,
},
},
},
{
name: "server returned a retryable HTTP 5xx response",
maxRetries: 3,
attempts: 1,
request: &http.Request{},
response: &http.Response{
StatusCode: http.StatusServiceUnavailable,
Header: http.Header{
"Retry-After": []string{"3"},
},
},
err: nil,
retryExpected: []bool{true},
retryAfterExpected: []*RetryAfter{
{
Attempt: 1,
Wait: 3 * time.Second,
Reason: "retries: 1, retry-after: 3s - retry-reason: 503",
},
},
},
{
name: "server returned a non response without without a Retry-After header",
maxRetries: 1,
attempts: 1,
request: &http.Request{},
response: &http.Response{
StatusCode: http.StatusTooManyRequests,
Header: http.Header{},
},
err: nil,
retryExpected: []bool{false},
retryAfterExpected: []*RetryAfter{
{
Attempt: 1,
},
},
},
{
name: "both response and err are set, err takes precedence",
maxRetries: 1,
attempts: 1,
request: &http.Request{},
response: retryAfterResponse(),
err: fakeError,
retryableErrFunc: func(_ *http.Request, err error) bool {
if err == fakeError {
return true
}
return false
},
retryExpected: []bool{true},
retryAfterExpected: []*RetryAfter{
{
Attempt: 1,
Wait: time.Second,
Reason: "retries: 1, retry-after: 1s - retry-reason: due to retryable error, error: fake error",
},
},
},
{
name: "all retries are exhausted",
maxRetries: 3,
attempts: 4,
request: &http.Request{},
response: nil,
err: fakeError,
retryableErrFunc: alwaysRetryError,
retryExpected: []bool{true, true, true, false},
retryAfterExpected: []*RetryAfter{
{
Attempt: 1,
Wait: time.Second,
Reason: "retries: 1, retry-after: 1s - retry-reason: due to retryable error, error: fake error",
},
{
Attempt: 2,
Wait: time.Second,
Reason: "retries: 2, retry-after: 1s - retry-reason: due to retryable error, error: fake error",
},
{
Attempt: 3,
Wait: time.Second,
Reason: "retries: 3, retry-after: 1s - retry-reason: due to retryable error, error: fake error",
},
{
Attempt: 4,
},
},
},
}
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
r := &withRetry{maxRetries: test.maxRetries}
retryGot := make([]bool, 0)
retryAfterGot := make([]*RetryAfter, 0)
for i := 0; i < test.attempts; i++ {
retryAfter, retry := r.NextRetry(test.request, test.response, test.err, test.retryableErrFunc)
retryGot = append(retryGot, retry)
retryAfterGot = append(retryAfterGot, retryAfter)
}
if !reflect.DeepEqual(test.retryExpected, retryGot) {
t.Errorf("Expected retry: %t, but got: %t", test.retryExpected, retryGot)
}
if !reflect.DeepEqual(test.retryAfterExpected, retryAfterGot) {
t.Errorf("Expected retry-after parameters to match, but got: %s", cmp.Diff(test.retryAfterExpected, retryAfterGot))
}
})
}
}