From cd0953740a191a10555260ef4b2068cbb81f6acf Mon Sep 17 00:00:00 2001 From: Devin Trejo Date: Fri, 22 Jul 2022 15:09:51 -0400 Subject: [PATCH 1/9] improvement: Introduce new URIPool and URISelector interfaces. Refactors large bits of the internal httpclient.Client retry logic. Currently we don't store any state across the lifecycle of multiple requests. Instead the current retry logic relies on a RequestRetrier and a URI selection mechanism. However the interfaces between the two logic machines doesn't provide clear boundaries and also makes it difficult to use recovered response state across different requests. I propose introducing a two new interfaces -- URIPool and URISelector. The URIPool is an abstraction that tracks server side errors across all requests. It also is responsible for updating the list of available uris from the configuration refreshable. Additionally the URISelector is an abstraction that allow for implementing many types of scored request weighting algorithms. For example, least used connection, rendezvous routing, zone aware routing, ect. Lastly the RequestRetrier still determines at where requests in general should be retried and the associated backoff logic. Work here is still a WIP. --- conjure-go-client/httpclient/client.go | 54 +- .../httpclient/client_builder.go | 17 +- conjure-go-client/httpclient/client_params.go | 6 +- .../httpclient/internal/balanced_scorer.go | 129 ---- .../httpclient/internal/balanced_selector.go | 154 +++++ ...orer_test.go => balanced_selector_test.go} | 29 +- .../httpclient/internal/interface.go | 24 + .../{random_scorer.go => random_selector.go} | 46 +- ...scorer_test.go => random_selector_test.go} | 15 +- .../httpclient/internal/refreshable_scorer.go | 35 - .../httpclient/internal/request_retrier.go | 171 ++--- .../internal/request_retrier_test.go | 618 +++++++++--------- .../httpclient/internal/retry.go | 2 + .../httpclient/internal/stateful_uri_pool.go | 125 ++++ .../internal/stateful_uri_pool_test.go | 220 +++++++ 15 files changed, 962 insertions(+), 683 deletions(-) delete mode 100644 conjure-go-client/httpclient/internal/balanced_scorer.go create mode 100644 conjure-go-client/httpclient/internal/balanced_selector.go rename conjure-go-client/httpclient/internal/{balanced_scorer_test.go => balanced_selector_test.go} (71%) create mode 100644 conjure-go-client/httpclient/internal/interface.go rename conjure-go-client/httpclient/internal/{random_scorer.go => random_selector.go} (52%) rename conjure-go-client/httpclient/internal/{random_scorer_test.go => random_selector_test.go} (68%) delete mode 100644 conjure-go-client/httpclient/internal/refreshable_scorer.go create mode 100644 conjure-go-client/httpclient/internal/stateful_uri_pool.go create mode 100644 conjure-go-client/httpclient/internal/stateful_uri_pool_test.go diff --git a/conjure-go-client/httpclient/client.go b/conjure-go-client/httpclient/client.go index 136b9752..bdc681ce 100644 --- a/conjure-go-client/httpclient/client.go +++ b/conjure-go-client/httpclient/client.go @@ -55,7 +55,8 @@ type clientImpl struct { errorDecoderMiddleware Middleware recoveryMiddleware Middleware - uriScorer internal.RefreshableURIScoringMiddleware + uriPool internal.URIPool + uriSelector internal.URISelector maxAttempts refreshable.IntPtr // 0 means no limit. If nil, uses 2*len(uris). backoffOptions refreshingclient.RefreshableRetryParams bufferPool bytesbuffers.Pool @@ -82,12 +83,8 @@ func (c *clientImpl) Delete(ctx context.Context, params ...RequestParam) (*http. } func (c *clientImpl) Do(ctx context.Context, params ...RequestParam) (*http.Response, error) { - uris := c.uriScorer.CurrentURIScoringMiddleware().GetURIsInOrderOfIncreasingScore() - if len(uris) == 0 { - return nil, werror.ErrorWithContextParams(ctx, "no base URIs are configured") - } - - attempts := 2 * len(uris) + uriCount := c.uriPool.NumURIs() + attempts := 2 * uriCount if c.maxAttempts != nil { if confMaxAttempts := c.maxAttempts.CurrentIntPtr(); confMaxAttempts != nil { attempts = *confMaxAttempts @@ -95,18 +92,14 @@ func (c *clientImpl) Do(ctx context.Context, params ...RequestParam) (*http.Resp } var err error + retrier := internal.NewRequestRetrier(c.backoffOptions.CurrentRetryParams().Start(ctx), attempts) + var req *http.Request var resp *http.Response - - retrier := internal.NewRequestRetrier(uris, c.backoffOptions.CurrentRetryParams().Start(ctx), attempts) - for { - uri, isRelocated := retrier.GetNextURI(resp, err) - if uri == "" { - break - } + for retrier.Next(req, resp) { + req, resp, err = c.doOnce(ctx, params...) if err != nil { svc1log.FromContext(ctx).Debug("Retrying request", svc1log.Stacktrace(err)) } - resp, err = c.doOnce(ctx, uri, isRelocated, params...) } if err != nil { return nil, err @@ -114,13 +107,7 @@ func (c *clientImpl) Do(ctx context.Context, params ...RequestParam) (*http.Resp return resp, nil } -func (c *clientImpl) doOnce( - ctx context.Context, - baseURI string, - useBaseURIOnly bool, - params ...RequestParam, -) (*http.Response, error) { - +func (c *clientImpl) doOnce(ctx context.Context, params ...RequestParam) (*http.Request, *http.Response, error) { // 1. create the request b := &requestBuilder{ headers: make(http.Header), @@ -133,26 +120,26 @@ func (c *clientImpl) doOnce( continue } if err := p.apply(b); err != nil { - return nil, err + return nil, nil, err } } - if useBaseURIOnly { - b.path = "" - } for _, c := range b.configureCtx { ctx = c(ctx) } if b.method == "" { - return nil, werror.ErrorWithContextParams(ctx, "httpclient: use WithRequestMethod() to specify HTTP method") + return nil, nil, werror.ErrorWithContextParams(ctx, "httpclient: use WithRequestMethod() to specify HTTP method") + } + url, err := c.uriSelector.Select(c.uriPool.URIs(), b.headers) + if err != nil { + return nil, nil, werror.WrapWithContextParams(ctx, err, "failed to select uri") } - reqURI := joinURIAndPath(baseURI, b.path) - req, err := http.NewRequest(b.method, reqURI, nil) + req, err := http.NewRequestWithContext(ctx, b.method, url, nil) if err != nil { - return nil, werror.WrapWithContextParams(ctx, err, "failed to build new HTTP request") + return nil, nil, werror.WrapWithContextParams(ctx, err, "failed to build request") } - req = req.WithContext(ctx) + req.Header = b.headers if q := b.query.Encode(); q != "" { req.URL.RawQuery = q @@ -164,7 +151,8 @@ func (c *clientImpl) doOnce( transport := clientCopy.Transport // start with the client's transport configured with default middleware // must precede the error decoders to read the status code of the raw response. - transport = wrapTransport(transport, c.uriScorer.CurrentURIScoringMiddleware()) + transport = wrapTransport(transport, c.uriSelector) + transport = wrapTransport(transport, c.uriPool) // request decoder must precede the client decoder // must precede the body middleware to read the response body transport = wrapTransport(transport, b.errorDecoderMiddleware, c.errorDecoderMiddleware) @@ -188,7 +176,7 @@ func (c *clientImpl) doOnce( internal.DrainBody(resp) } - return resp, unwrapURLError(ctx, respErr) + return req, resp, unwrapURLError(ctx, respErr) } // unwrapURLError converts a *url.Error to a werror. We need this because all diff --git a/conjure-go-client/httpclient/client_builder.go b/conjure-go-client/httpclient/client_builder.go index 3779aabf..8338fc27 100644 --- a/conjure-go-client/httpclient/client_builder.go +++ b/conjure-go-client/httpclient/client_builder.go @@ -49,8 +49,8 @@ const ( type clientBuilder struct { HTTP *httpClientBuilder - URIs refreshable.StringSlice - URIScorerBuilder func([]string) internal.URIScoringMiddleware + URIs refreshable.StringSlice + URISelector internal.URISelector ErrorDecoder ErrorDecoder @@ -153,15 +153,14 @@ func newClient(ctx context.Context, b *clientBuilder, params ...ClientParam) (Cl if !b.HTTP.DisableRecovery { recovery = recoveryMiddleware{} } - uriScorer := internal.NewRefreshableURIScoringMiddleware(b.URIs, func(uris []string) internal.URIScoringMiddleware { - if b.URIScorerBuilder == nil { - return internal.NewRandomURIScoringMiddleware(uris, func() int64 { return time.Now().UnixNano() }) - } - return b.URIScorerBuilder(uris) - }) + uriPool := internal.NewStatefulURIPool(b.URIs) + if b.URISelector == nil { + b.URISelector = internal.NewRandomURISelector(func() int64 { return time.Now().UnixNano() }) + } return &clientImpl{ client: httpClient, - uriScorer: uriScorer, + uriPool: uriPool, + uriSelector: b.URISelector, maxAttempts: b.MaxAttempts, backoffOptions: b.RetryParams, middlewares: middleware, diff --git a/conjure-go-client/httpclient/client_params.go b/conjure-go-client/httpclient/client_params.go index e0cef56f..8184d9e4 100644 --- a/conjure-go-client/httpclient/client_params.go +++ b/conjure-go-client/httpclient/client_params.go @@ -537,11 +537,7 @@ func WithBasicAuth(username, password string) ClientParam { // and least recent errors. func WithBalancedURIScoring() ClientParam { return clientParamFunc(func(b *clientBuilder) error { - b.URIScorerBuilder = func(uris []string) internal.URIScoringMiddleware { - return internal.NewBalancedURIScoringMiddleware(uris, func() int64 { - return time.Now().UnixNano() - }) - } + b.URISelector = internal.NewBalancedURISelector(func() int64 { return time.Now().UnixNano() }) return nil }) } diff --git a/conjure-go-client/httpclient/internal/balanced_scorer.go b/conjure-go-client/httpclient/internal/balanced_scorer.go deleted file mode 100644 index cff197eb..00000000 --- a/conjure-go-client/httpclient/internal/balanced_scorer.go +++ /dev/null @@ -1,129 +0,0 @@ -// Copyright (c) 2021 Palantir Technologies. All rights reserved. -// -// 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 internal - -import ( - "math" - "math/rand" - "net/http" - "net/url" - "sort" - "sync/atomic" - "time" -) - -const ( - failureWeight = 10.0 - failureMemory = 30 * time.Second -) - -type URIScoringMiddleware interface { - GetURIsInOrderOfIncreasingScore() []string - RoundTrip(req *http.Request, next http.RoundTripper) (*http.Response, error) -} - -type balancedScorer struct { - uriInfos map[string]uriInfo -} - -type uriInfo struct { - inflight int32 - recentFailures CourseExponentialDecayReservoir -} - -// NewBalancedURIScoringMiddleware returns URI scoring middleware that tracks in-flight requests and recent failures -// for each URI configured on an HTTP client. URIs are scored based on fewest in-flight requests and recent errors, -// where client errors are weighted the same as 1/10 of an in-flight request, server errors are weighted as 10 -// in-flight requests, and errors are decayed using exponential decay with a half-life of 30 seconds. -// -// This implementation is based on Dialogue's BalancedScoreTracker: -// https://github.com/palantir/dialogue/blob/develop/dialogue-core/src/main/java/com/palantir/dialogue/core/BalancedScoreTracker.java -func NewBalancedURIScoringMiddleware(uris []string, nanoClock func() int64) URIScoringMiddleware { - uriInfos := make(map[string]uriInfo, len(uris)) - for _, uri := range uris { - uriInfos[uri] = uriInfo{ - recentFailures: NewCourseExponentialDecayReservoir(nanoClock, failureMemory), - } - } - return &balancedScorer{uriInfos} -} - -func (u *balancedScorer) GetURIsInOrderOfIncreasingScore() []string { - uris := make([]string, 0, len(u.uriInfos)) - scores := make(map[string]int32, len(u.uriInfos)) - for uri, info := range u.uriInfos { - uris = append(uris, uri) - scores[uri] = info.computeScore() - } - // Pre-shuffle to avoid overloading first URI when no request are in-flight - rand.Shuffle(len(uris), func(i, j int) { - uris[i], uris[j] = uris[j], uris[i] - }) - sort.Slice(uris, func(i, j int) bool { - return scores[uris[i]] < scores[uris[j]] - }) - return uris -} - -func (u *balancedScorer) RoundTrip(req *http.Request, next http.RoundTripper) (*http.Response, error) { - baseURI := getBaseURI(req.URL) - info, foundInfo := u.uriInfos[baseURI] - if foundInfo { - atomic.AddInt32(&info.inflight, 1) - defer atomic.AddInt32(&info.inflight, -1) - } - resp, err := next.RoundTrip(req) - if resp == nil || err != nil { - if foundInfo { - info.recentFailures.Update(failureWeight) - } - return nil, err - } - if foundInfo { - statusCode := resp.StatusCode - if isGlobalQosStatus(statusCode) || isServerErrorRange(statusCode) { - info.recentFailures.Update(failureWeight) - } else if isClientError(statusCode) { - info.recentFailures.Update(failureWeight / 100) - } - } - return resp, nil -} - -func (i *uriInfo) computeScore() int32 { - return atomic.LoadInt32(&i.inflight) + int32(math.Round(i.recentFailures.Get())) -} - -func getBaseURI(u *url.URL) string { - uCopy := url.URL{ - Scheme: u.Scheme, - Opaque: u.Opaque, - User: u.User, - Host: u.Host, - } - return uCopy.String() -} - -func isGlobalQosStatus(statusCode int) bool { - return statusCode == StatusCodeRetryOther || statusCode == StatusCodeUnavailable -} - -func isServerErrorRange(statusCode int) bool { - return statusCode/100 == 5 -} - -func isClientError(statusCode int) bool { - return statusCode/100 == 4 -} diff --git a/conjure-go-client/httpclient/internal/balanced_selector.go b/conjure-go-client/httpclient/internal/balanced_selector.go new file mode 100644 index 00000000..23efa50c --- /dev/null +++ b/conjure-go-client/httpclient/internal/balanced_selector.go @@ -0,0 +1,154 @@ +package internal + +import ( + "math" + "math/rand" + "net/http" + "net/url" + "sort" + "sync" + "sync/atomic" + "time" + + werror "github.com/palantir/witchcraft-go-error" +) + +const ( + failureWeight = 10.0 + failureMemory = 30 * time.Second +) + +// NewBalancedURISelector returns URI scoring middleware that tracks in-flight requests and recent failures +// for each URI configured on an HTTP client. URIs are scored based on fewest in-flight requests and recent errors, +// where client errors are weighted the same as 1/10 of an in-flight request, server errors are weighted as 10 +// in-flight requests, and errors are decayed using exponential decay with a half-life of 30 seconds. +// +// This implementation is based on Dialogue's BalancedScoreTracker: +// https://github.com/palantir/dialogue/blob/develop/dialogue-core/src/main/java/com/palantir/dialogue/core/BalancedScoreTracker.java +func NewBalancedURISelector(nanoClock func() int64) URISelector { + return &balancedSelector{ + nanoClock: nanoClock, + } +} + +type balancedSelector struct { + sync.Mutex + + nanoClock func() int64 + uriInfos map[string]uriInfo +} + +// Select implements estransport.Selector interface +func (s *balancedSelector) Select(uris []string, _ http.Header) (string, error) { + s.Lock() + defer s.Unlock() + + s.updateURIs(uris) + return s.next() +} + +func (s *balancedSelector) updateURIs(uris []string) { + uriInfos := make(map[string]uriInfo, len(uris)) + for _, uri := range uris { + if exisiting, ok := s.uriInfos[uri]; ok { + uriInfos[uri] = exisiting + continue + } + uriInfos[uri] = uriInfo{ + recentFailures: NewCourseExponentialDecayReservoir(s.nanoClock, failureMemory), + } + } + + s.uriInfos = uriInfos + return +} + +func (s *balancedSelector) next() (string, error) { + if len(s.uriInfos) == 0 { + return "", werror.Error("no valid connections available") + } + uris := make([]string, 0, len(s.uriInfos)) + scores := make(map[string]int32, len(s.uriInfos)) + for uri, info := range s.uriInfos { + uris = append(uris, uri) + scores[uri] = info.computeScore() + } + // Pre-shuffle to avoid overloading first URI when no request are in-flight + rand.Shuffle(len(uris), func(i, j int) { + uris[i], uris[j] = uris[j], uris[i] + }) + sort.Slice(uris, func(i, j int) bool { + return scores[uris[i]] < scores[uris[j]] + }) + return uris[0], nil +} + +func (s *balancedSelector) RoundTrip(req *http.Request, next http.RoundTripper) (*http.Response, error) { + baseURI := getBaseURI(req.URL) + s.updateInflight(baseURI, 1) + defer s.updateInflight(baseURI, -1) + + resp, err := next.RoundTrip(req) + if resp == nil || err != nil { + s.updateRecentFailures(baseURI, failureWeight) + return nil, err + } + statusCode := resp.StatusCode + if isGlobalQosStatus(statusCode) || isServerErrorRange(statusCode) { + s.updateRecentFailures(baseURI, failureWeight) + } else if isClientError(statusCode) { + s.updateRecentFailures(baseURI, failureWeight/100) + } + return resp, nil +} + +func (s *balancedSelector) updateInflight(uri string, score int32) { + s.Lock() + defer s.Unlock() + + info, ok := s.uriInfos[uri] + if ok { + atomic.AddInt32(&info.inflight, score) + } +} + +func (s *balancedSelector) updateRecentFailures(uri string, weight float64) { + s.Lock() + defer s.Unlock() + + info, ok := s.uriInfos[uri] + if ok { + info.recentFailures.Update(weight) + } +} + +type uriInfo struct { + inflight int32 + recentFailures CourseExponentialDecayReservoir +} + +func (i *uriInfo) computeScore() int32 { + return atomic.LoadInt32(&i.inflight) + int32(math.Round(i.recentFailures.Get())) +} + +func getBaseURI(u *url.URL) string { + uCopy := url.URL{ + Scheme: u.Scheme, + Opaque: u.Opaque, + User: u.User, + Host: u.Host, + } + return uCopy.String() +} + +func isGlobalQosStatus(statusCode int) bool { + return statusCode == StatusCodeRetryOther || statusCode == StatusCodeUnavailable +} + +func isServerErrorRange(statusCode int) bool { + return statusCode/100 == 5 +} + +func isClientError(statusCode int) bool { + return statusCode/100 == 4 +} diff --git a/conjure-go-client/httpclient/internal/balanced_scorer_test.go b/conjure-go-client/httpclient/internal/balanced_selector_test.go similarity index 71% rename from conjure-go-client/httpclient/internal/balanced_scorer_test.go rename to conjure-go-client/httpclient/internal/balanced_selector_test.go index 665ece5a..df23893e 100644 --- a/conjure-go-client/httpclient/internal/balanced_scorer_test.go +++ b/conjure-go-client/httpclient/internal/balanced_selector_test.go @@ -17,20 +17,21 @@ package internal import ( "net/http" "net/http/httptest" + "net/url" "testing" "github.com/stretchr/testify/assert" ) -func TestBalancedScorerRandomizesWithNoneInflight(t *testing.T) { +func TestBalancedSelectorRandomizesWithNoneInflight(t *testing.T) { uris := []string{"uri1", "uri2", "uri3", "uri4", "uri5"} - scorer := NewBalancedURIScoringMiddleware(uris, func() int64 { return 0 }) - scoredUris := scorer.GetURIsInOrderOfIncreasingScore() - assert.ElementsMatch(t, scoredUris, uris) - assert.NotEqual(t, scoredUris, uris) + scorer := NewBalancedURISelector(func() int64 { return 0 }) + scoredURI, err := scorer.Select(uris, nil) + assert.NoError(t, err) + assert.Contains(t, uris, scoredURI) } -func TestBalancedScoring(t *testing.T) { +func TestBalancedSelect(t *testing.T) { server200 := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) { rw.WriteHeader(http.StatusOK) })) @@ -44,15 +45,23 @@ func TestBalancedScoring(t *testing.T) { })) defer server503.Close() uris := []string{server503.URL, server429.URL, server200.URL} - scorer := NewBalancedURIScoringMiddleware(uris, func() int64 { return 0 }) + scorer := NewBalancedURISelector(func() int64 { return 0 }) for _, server := range []*httptest.Server{server200, server429, server503} { for i := 0; i < 10; i++ { - req, err := http.NewRequest("GET", server.URL, nil) + uri, err := scorer.Select(uris, nil) assert.NoError(t, err) + req, err := http.NewRequest("GET", uri, nil) + assert.NoError(t, err) + + url, err := url.Parse(uri) + assert.NoError(t, err) + req.URL = url _, err = scorer.RoundTrip(req, server.Client().Transport) assert.NoError(t, err) } } - scoredUris := scorer.GetURIsInOrderOfIncreasingScore() - assert.Equal(t, []string{server200.URL, server429.URL, server503.URL}, scoredUris) + + uri, err := scorer.Select(uris, nil) + assert.NoError(t, err) + assert.Equal(t, server200.URL, uri) } diff --git a/conjure-go-client/httpclient/internal/interface.go b/conjure-go-client/httpclient/internal/interface.go new file mode 100644 index 00000000..283e2c1f --- /dev/null +++ b/conjure-go-client/httpclient/internal/interface.go @@ -0,0 +1,24 @@ +package internal + +import ( + "net/http" +) + +// URISelector is used in combination with a URIPool to get the +// preferred next URL for a given request. +type URISelector interface { + Select([]string, http.Header) (string, error) + RoundTrip(*http.Request, http.RoundTripper) (*http.Response, error) +} + +// URIPool stores all possible URIs for a given connection. It can be use +// as http middleware in order to maintain state between requests. +type URIPool interface { + // NumURIs returns the overall count of URIs available in the + // connection pool. + NumURIs() int + // URIs returns the set of URIs that should be considered by a + // URISelector + URIs() []string + RoundTrip(*http.Request, http.RoundTripper) (*http.Response, error) +} diff --git a/conjure-go-client/httpclient/internal/random_scorer.go b/conjure-go-client/httpclient/internal/random_selector.go similarity index 52% rename from conjure-go-client/httpclient/internal/random_scorer.go rename to conjure-go-client/httpclient/internal/random_selector.go index 63ebc1d6..24708fa1 100644 --- a/conjure-go-client/httpclient/internal/random_scorer.go +++ b/conjure-go-client/httpclient/internal/random_selector.go @@ -17,31 +17,43 @@ package internal import ( "math/rand" "net/http" + "sync" + + werror "github.com/palantir/witchcraft-go-error" ) -type randomScorer struct { - uris []string +type randomSelector struct { + sync.Mutex nanoClock func() int64 } -func (n *randomScorer) GetURIsInOrderOfIncreasingScore() []string { - uris := make([]string, len(n.uris)) - copy(uris, n.uris) - rand.New(rand.NewSource(n.nanoClock())).Shuffle(len(uris), func(i, j int) { - uris[i], uris[j] = uris[j], uris[i] - }) - return uris +// NewRandomURISelector returns a URI scorer that randomizes the order of URIs when scoring using a rand.Rand +// seeded by the nanoClock function. The middleware no-ops on each request. +func NewRandomURISelector(nanoClock func() int64) URISelector { + return &randomSelector{ + nanoClock: nanoClock, + } } -func (n *randomScorer) RoundTrip(req *http.Request, next http.RoundTripper) (*http.Response, error) { - return next.RoundTrip(req) +// Select implements estransport.Selector interface +func (s *randomSelector) Select(uris []string, _ http.Header) (string, error) { + s.Lock() + defer s.Unlock() + + return s.next(uris) } -// NewRandomURIScoringMiddleware returns a URI scorer that randomizes the order of URIs when scoring using a rand.Rand -// seeded by the nanoClock function. The middleware no-ops on each request. -func NewRandomURIScoringMiddleware(uris []string, nanoClock func() int64) URIScoringMiddleware { - return &randomScorer{ - uris: uris, - nanoClock: nanoClock, +func (s *randomSelector) next(uris []string) (string, error) { + if len(uris) == 0 { + return "", werror.Error("no valid connections available") } + rand.New(rand.NewSource(s.nanoClock())).Shuffle(len(uris), func(i, j int) { + uris[i], uris[j] = uris[j], uris[i] + }) + + return uris[0], nil +} + +func (s *randomSelector) RoundTrip(req *http.Request, next http.RoundTripper) (*http.Response, error) { + return next.RoundTrip(req) } diff --git a/conjure-go-client/httpclient/internal/random_scorer_test.go b/conjure-go-client/httpclient/internal/random_selector_test.go similarity index 68% rename from conjure-go-client/httpclient/internal/random_scorer_test.go rename to conjure-go-client/httpclient/internal/random_selector_test.go index 43c332e4..ea902bdf 100644 --- a/conjure-go-client/httpclient/internal/random_scorer_test.go +++ b/conjure-go-client/httpclient/internal/random_selector_test.go @@ -21,11 +21,14 @@ import ( "github.com/stretchr/testify/assert" ) -func TestRandomScorerGetURIsRandomizes(t *testing.T) { +func TestRandomSelectorGetRandomURIs(t *testing.T) { uris := []string{"uri1", "uri2", "uri3", "uri4", "uri5"} - scorer := NewRandomURIScoringMiddleware(uris, func() int64 { return time.Now().UnixNano() }) - scoredUris1 := scorer.GetURIsInOrderOfIncreasingScore() - scoredUris2 := scorer.GetURIsInOrderOfIncreasingScore() - assert.ElementsMatch(t, scoredUris1, scoredUris2) - assert.NotEqual(t, scoredUris1, scoredUris2) + scorer := NewRandomURISelector(func() int64 { return time.Now().UnixNano() }) + uri, err := scorer.Select(uris, nil) + assert.NoError(t, err) + assert.Contains(t, uris, uri) + + uri2, err := scorer.Select(uris, nil) + assert.NoError(t, err) + assert.Contains(t, uris, uri2) } diff --git a/conjure-go-client/httpclient/internal/refreshable_scorer.go b/conjure-go-client/httpclient/internal/refreshable_scorer.go deleted file mode 100644 index 4ab94363..00000000 --- a/conjure-go-client/httpclient/internal/refreshable_scorer.go +++ /dev/null @@ -1,35 +0,0 @@ -// Copyright (c) 2021 Palantir Technologies. All rights reserved. -// -// 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 internal - -import ( - "github.com/palantir/pkg/refreshable" -) - -type RefreshableURIScoringMiddleware interface { - CurrentURIScoringMiddleware() URIScoringMiddleware -} - -func NewRefreshableURIScoringMiddleware(uris refreshable.StringSlice, constructor func([]string) URIScoringMiddleware) RefreshableURIScoringMiddleware { - return refreshableURIScoringMiddleware{uris.MapStringSlice(func(uris []string) interface{} { - return constructor(uris) - })} -} - -type refreshableURIScoringMiddleware struct{ refreshable.Refreshable } - -func (r refreshableURIScoringMiddleware) CurrentURIScoringMiddleware() URIScoringMiddleware { - return r.Current().(URIScoringMiddleware) -} diff --git a/conjure-go-client/httpclient/internal/request_retrier.go b/conjure-go-client/httpclient/internal/request_retrier.go index e7c738f7..4fcd57b9 100644 --- a/conjure-go-client/httpclient/internal/request_retrier.go +++ b/conjure-go-client/httpclient/internal/request_retrier.go @@ -16,44 +16,33 @@ package internal import ( "net/http" - "net/url" "strings" "github.com/palantir/pkg/retry" ) -const ( - meshSchemePrefix = "mesh-" -) - -// RequestRetrier manages URIs for an HTTP client, providing an API which determines whether requests should be retries -// and supplying the correct URL for the client to retry. -// In the case of servers in a service-mesh, requests will never be retried and the mesh URI will only be returned on the -// first call to GetNextURI +// RequestRetrier manages the lifecylce of a single request. It will tracks the +// backoff timing between subsequent requests. The retrier should only suggest +// a URI if the previous request returned a redirect or is a mesh URI. In the +// case of a mesh URI being detected, the request retrier will only attempt the +// request once. type RequestRetrier struct { - currentURI string - retrier retry.Retrier - uris []string - offset int - relocatedURIs map[string]struct{} - failedURIs map[string]struct{} - maxAttempts int - attemptCount int + retrier retry.Retrier + + maxAttempts int + attemptCount int } // NewRequestRetrier creates a new request retrier. // Regardless of maxAttempts, mesh URIs will never be retried. -func NewRequestRetrier(uris []string, retrier retry.Retrier, maxAttempts int) *RequestRetrier { - offset := 0 +func NewRequestRetrier( + retrier retry.Retrier, + maxAttempts int, +) *RequestRetrier { return &RequestRetrier{ - currentURI: uris[offset], - retrier: retrier, - uris: uris, - offset: offset, - relocatedURIs: map[string]struct{}{}, - failedURIs: map[string]struct{}{}, - maxAttempts: maxAttempts, - attemptCount: 0, + retrier: retrier, + maxAttempts: maxAttempts, + attemptCount: 0, } } @@ -65,119 +54,39 @@ func (r *RequestRetrier) attemptsRemaining() bool { return r.attemptCount < r.maxAttempts } -// GetNextURI returns the next URI a client should use, or empty string if no suitable URI remaining to retry. -// isRelocated is true when the URI comes from a redirect's Location header. In this case, it already includes the request path. -func (r *RequestRetrier) GetNextURI(resp *http.Response, respErr error) (uri string, isRelocated bool) { - defer func() { - r.attemptCount++ - }() - if r.attemptCount == 0 { - // First attempt is always successful. Trigger the first retry so later calls have backoff - // but ignore the returned value to ensure that the client can instrument the request even - // if the context is done. - r.retrier.Next() - return r.removeMeshSchemeIfPresent(r.currentURI), false - } - if !r.attemptsRemaining() { - // Retries exhausted - return "", false - } - if r.isMeshURI(r.currentURI) { - // Mesh uris don't get retried - return "", false - } - retryFn := r.getRetryFn(resp, respErr) - if retryFn == nil { - // The previous response was not retryable - return "", false - } - // Updates currentURI - if !retryFn() { - return "", false - } - return r.currentURI, r.isRelocatedURI(r.currentURI) -} - -func (r *RequestRetrier) getRetryFn(resp *http.Response, respErr error) func() bool { - errCode, _ := StatusCodeFromError(respErr) - if retryOther, _ := isThrottleResponse(resp, errCode); retryOther { - // 429: throttle - // Immediately backoff and select the next URI. - // TODO(whickman): use the retry-after header once #81 is resolved - return r.nextURIAndBackoff - } else if isUnavailableResponse(resp, errCode) { - // 503: go to next node - return r.nextURIOrBackoff - } else if shouldTryOther, otherURI := isRetryOtherResponse(resp, respErr, errCode); shouldTryOther { - // 307 or 308: go to next node, or particular node if provided. - if otherURI != nil { - return func() bool { - r.setURIAndResetBackoff(otherURI) - return true - } +// Next returns true if a subsequent request attempt should be attempted. If +// uses the previous requestURI to determine if the request should be +// attempted. If the returned value is true, the retrier will have waited the +// desired backoff interval before returning. +func (r *RequestRetrier) Next(prevReq *http.Request, prevResp *http.Response) bool { + defer func() { r.attemptCount++ }() + // check for bad requests + if prevResp != nil { + prevCode := prevResp.StatusCode + // succesfull response + if prevCode == http.StatusOK { + return false + } + if prevCode >= http.StatusBadRequest && prevCode < http.StatusInternalServerError { + return false } - return r.nextURIOrBackoff - } else if errCode >= http.StatusBadRequest && errCode < http.StatusInternalServerError { - return nil - } else if resp == nil { - // if we get a nil response, we can assume there is a problem with host and can move on to the next. - return r.nextURIOrBackoff } - return nil -} -func (r *RequestRetrier) setURIAndResetBackoff(otherURI *url.URL) { - // If the URI returned by relocation header is a relative path - // We will resolve it with the current URI - if !otherURI.IsAbs() { - if currentURI := parseLocationURL(r.currentURI); currentURI != nil { - otherURI = currentURI.ResolveReference(otherURI) + // don't retry mesh uris + if prevReq != nil { + prevURI := getBaseURI(prevReq.URL) + if r.isMeshURI(prevURI) { + return false } } - nextURI := otherURI.String() - r.relocatedURIs[otherURI.String()] = struct{}{} - r.retrier.Reset() - r.currentURI = nextURI -} -// If lastURI was already marked failed, we perform a backoff as determined by the retrier before returning the next URI and its offset. -// Otherwise, we add lastURI to failedURIs and return the next URI and its offset immediately. -func (r *RequestRetrier) nextURIOrBackoff() bool { - _, performBackoff := r.failedURIs[r.currentURI] - r.markFailedAndMoveToNextURI() - // If the URI has failed before, perform a backoff - if performBackoff || len(r.uris) == 1 { - return r.retrier.Next() + if !r.attemptsRemaining() { + // Retries exhausted + return false } - return true -} - -// Marks the current URI as failed, gets the next URI, and performs a backoff as determined by the retrier. -func (r *RequestRetrier) nextURIAndBackoff() bool { - r.markFailedAndMoveToNextURI() return r.retrier.Next() } -func (r *RequestRetrier) markFailedAndMoveToNextURI() { - r.failedURIs[r.currentURI] = struct{}{} - nextURIOffset := (r.offset + 1) % len(r.uris) - nextURI := r.uris[nextURIOffset] - r.currentURI = nextURI - r.offset = nextURIOffset -} - -func (r *RequestRetrier) removeMeshSchemeIfPresent(uri string) string { - if r.isMeshURI(uri) { - return strings.Replace(uri, meshSchemePrefix, "", 1) - } - return uri -} - -func (r *RequestRetrier) isMeshURI(uri string) bool { +func (*RequestRetrier) isMeshURI(uri string) bool { return strings.HasPrefix(uri, meshSchemePrefix) } - -func (r *RequestRetrier) isRelocatedURI(uri string) bool { - _, relocatedURI := r.relocatedURIs[uri] - return relocatedURI -} diff --git a/conjure-go-client/httpclient/internal/request_retrier_test.go b/conjure-go-client/httpclient/internal/request_retrier_test.go index ae830514..ee782069 100644 --- a/conjure-go-client/httpclient/internal/request_retrier_test.go +++ b/conjure-go-client/httpclient/internal/request_retrier_test.go @@ -21,352 +21,354 @@ import ( "time" "github.com/palantir/pkg/retry" - werror "github.com/palantir/witchcraft-go-error" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) -var _ retry.Retrier = &mockRetrier{} +//var _ retry.Retrier = &mockRetrier{} func TestRequestRetrier_HandleMeshURI(t *testing.T) { - r := NewRequestRetrier([]string{"mesh-http://example.com"}, retry.Start(context.Background()), 1) - uri, _ := r.GetNextURI(nil, nil) - require.Equal(t, uri, "http://example.com") - - respErr := werror.ErrorWithContextParams(context.Background(), "error", werror.SafeParam("statusCode", 429)) - uri, _ = r.GetNextURI(nil, respErr) - require.Empty(t, uri) + r := NewRequestRetrier(retry.Start(context.Background()), 1) + req, err := http.NewRequest("GET", "mesh-http://example.com", nil) + require.NoError(t, err) + shouldRetry := r.Next(req, &http.Response{}) + require.False(t, shouldRetry) } func TestRequestRetrier_AttemptCount(t *testing.T) { maxAttempts := 3 - r := NewRequestRetrier([]string{"https://example.com"}, retry.Start(context.Background()), maxAttempts) - // first request is not a retry - uri, _ := r.GetNextURI(nil, nil) - require.Equal(t, uri, "https://example.com") + r := NewRequestRetrier(retry.Start(context.Background()), maxAttempts) + // first request is not a retry, so it doesn't increment the overall count + shouldRetry := r.Next(nil, nil) + require.True(t, shouldRetry) for i := 0; i < maxAttempts-1; i++ { - uri, _ = r.GetNextURI(nil, nil) - require.Equal(t, uri, "https://example.com") + req, err := http.NewRequest("GET", "http://example.com", nil) + require.NoError(t, err) + shouldRetry = r.Next(req, &http.Response{}) + require.True(t, shouldRetry) } - uri, _ = r.GetNextURI(nil, nil) - require.Empty(t, uri) + + req, err := http.NewRequest("GET", "http://example.com", nil) + require.NoError(t, err) + shouldRetry = r.Next(req, &http.Response{}) + require.False(t, shouldRetry) } func TestRequestRetrier_UnlimitedAttempts(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second) defer cancel() - r := NewRequestRetrier([]string{"https://example.com"}, retry.Start(ctx, retry.WithInitialBackoff(50*time.Millisecond), retry.WithRandomizationFactor(0)), 0) + r := NewRequestRetrier(retry.Start(ctx, retry.WithInitialBackoff(50*time.Millisecond), retry.WithRandomizationFactor(0)), 0) startTime := time.Now() - uri, _ := r.GetNextURI(nil, nil) - require.Equal(t, uri, "https://example.com") + shouldRetry := r.Next(nil, nil) + require.True(t, shouldRetry) require.Lessf(t, time.Since(startTime), 49*time.Millisecond, "first GetNextURI should not have any delay") + req, err := http.NewRequest("GET", "http://example.com", nil) + require.NoError(t, err) + resp := &http.Response{} + startTime = time.Now() - uri, _ = r.GetNextURI(nil, nil) - require.Equal(t, uri, "https://example.com") + shouldRetry = r.Next(req, resp) + require.True(t, shouldRetry) assert.Greater(t, time.Since(startTime), 50*time.Millisecond, "delay should be at least 1 backoff") assert.Less(t, time.Since(startTime), 100*time.Millisecond, "delay should be less than 2 backoffs") startTime = time.Now() - uri, _ = r.GetNextURI(nil, nil) - require.Equal(t, uri, "https://example.com") + shouldRetry = r.Next(req, resp) + require.True(t, shouldRetry) assert.Greater(t, time.Since(startTime), 100*time.Millisecond, "delay should be at least 2 backoffs") assert.Less(t, time.Since(startTime), 200*time.Millisecond, "delay should be less than 3 backoffs") // Success should stop retries - uri, _ = r.GetNextURI(&http.Response{StatusCode: 200}, nil) - require.Empty(t, uri) + shouldRetry = r.Next(req, &http.Response{StatusCode: http.StatusOK}) + require.False(t, shouldRetry) } func TestRequestRetrier_ContextCanceled(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) cancel() - r := NewRequestRetrier([]string{"https://example.com"}, retry.Start(ctx), 0) - - // First attempt should return a URI to ensure that the client can instrument the request even - // if the context is done - uri, _ := r.GetNextURI(nil, nil) - require.Equal(t, uri, "https://example.com") - - // Subsequent attempt should stop retries - uri, _ = r.GetNextURI(nil, nil) - require.Empty(t, uri) -} - -func TestRequestRetrier_UsesLocationHeader(t *testing.T) { - respWithLocationHeader := &http.Response{ - StatusCode: StatusCodeRetryOther, - Header: http.Header{"Location": []string{"http://example.com"}}, - } - - r := NewRequestRetrier([]string{"a"}, retry.Start(context.Background()), 2) - uri, isRelocated := r.GetNextURI(nil, nil) - require.Equal(t, uri, "a") - require.False(t, isRelocated) - - uri, isRelocated = r.GetNextURI(respWithLocationHeader, nil) - require.Equal(t, uri, "http://example.com") - require.True(t, isRelocated) -} - -func TestRequestRetrier_UsesLocationFromErr(t *testing.T) { - r := NewRequestRetrier([]string{"http://example-1.com"}, retry.Start(context.Background()), 2) - respErr := werror.ErrorWithContextParams(context.Background(), "307", - werror.SafeParam("statusCode", 307), - werror.SafeParam("location", "http://example-2.com")) - - uri, isRelocated := r.GetNextURI(nil, nil) - require.Equal(t, uri, "http://example-1.com") - require.False(t, isRelocated) - - uri, isRelocated = r.GetNextURI(nil, respErr) - require.Equal(t, uri, "http://example-2.com") - require.True(t, isRelocated) -} - -func TestRequestRetrier_GetNextURI(t *testing.T) { - for _, tc := range []struct { - name string - resp *http.Response - respErr error - uris []string - shouldRetry bool - shouldRetrySameURI bool - shouldRetryBackoff bool - shouldRetryReset bool - }{ - { - name: "returns error if response exists and doesn't appear retryable", - resp: &http.Response{}, - respErr: nil, - uris: []string{"a", "b"}, - shouldRetry: false, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "returns error if error code not retryable", - resp: &http.Response{}, - respErr: nil, - uris: []string{"a", "b"}, - shouldRetry: false, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "returns a URI if response and error are nil", - resp: nil, - respErr: nil, - uris: []string{"a", "b"}, - shouldRetry: true, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "returns a URI if response and error are nil", - resp: nil, - respErr: nil, - uris: []string{"a", "b"}, - shouldRetry: true, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "retries and backs off the single URI if response and error are nil", - resp: nil, - respErr: nil, - uris: []string{"a"}, - shouldRetry: true, - shouldRetrySameURI: true, - shouldRetryBackoff: true, - shouldRetryReset: false, - }, - { - name: "returns a new URI if unavailable", - resp: nil, - respErr: werror.ErrorWithContextParams(context.Background(), "503", werror.SafeParam("statusCode", 503)), - uris: []string{"a", "b"}, - shouldRetry: true, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "retries and backs off the single URI if unavailable", - resp: nil, - respErr: werror.ErrorWithContextParams(context.Background(), "503", werror.SafeParam("statusCode", 503)), - uris: []string{"a"}, - shouldRetry: true, - shouldRetrySameURI: true, - shouldRetryBackoff: true, - shouldRetryReset: false, - }, - { - name: "returns a new URI and backs off if throttled", - resp: nil, - respErr: werror.ErrorWithContextParams(context.Background(), "429", werror.SafeParam("statusCode", 429)), - uris: []string{"a", "b"}, - shouldRetry: true, - shouldRetrySameURI: false, - shouldRetryBackoff: true, - shouldRetryReset: false, - }, - { - name: "retries single URI and backs off if throttled", - resp: nil, - respErr: werror.ErrorWithContextParams(context.Background(), "429", werror.SafeParam("statusCode", 429)), - uris: []string{"a"}, - shouldRetry: true, - shouldRetrySameURI: true, - shouldRetryBackoff: true, - shouldRetryReset: false, - }, - { - name: "retries another URI if gets retry other response without location", - resp: &http.Response{ - StatusCode: StatusCodeRetryOther, - }, - respErr: nil, - uris: []string{"a", "b"}, - shouldRetry: true, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "retries single URI and backs off if gets retry other response without location", - resp: &http.Response{ - StatusCode: StatusCodeRetryOther, - }, - respErr: nil, - uris: []string{"a"}, - shouldRetry: true, - shouldRetrySameURI: true, - shouldRetryBackoff: true, - shouldRetryReset: false, - }, - { - name: "retries another URI if gets retry temporary redirect response without location", - resp: &http.Response{ - StatusCode: StatusCodeRetryTemporaryRedirect, - }, - respErr: nil, - uris: []string{"a", "b"}, - shouldRetry: true, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "retries single URI and backs off if gets retry temporary redirect response without location", - resp: &http.Response{ - StatusCode: StatusCodeRetryTemporaryRedirect, - }, - respErr: nil, - uris: []string{"a"}, - shouldRetry: true, - shouldRetrySameURI: true, - shouldRetryBackoff: true, - shouldRetryReset: false, - }, - { - name: "does not retry 400 responses", - resp: &http.Response{ - StatusCode: 400, - }, - uris: []string{"a", "b"}, - shouldRetry: false, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "does not retry 404 responses", - resp: &http.Response{ - StatusCode: 404, - }, - uris: []string{"a", "b"}, - shouldRetry: false, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "does not retry 400 errors", - respErr: werror.ErrorWithContextParams(context.Background(), "400", werror.SafeParam("statusCode", 400)), - uris: []string{"a", "b"}, - shouldRetry: false, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - { - name: "does not retry 404s", - respErr: werror.ErrorWithContextParams(context.Background(), "404", werror.SafeParam("statusCode", 404)), - uris: []string{"a", "b"}, - shouldRetry: false, - shouldRetrySameURI: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, - } { - t.Run(tc.name, func(t *testing.T) { - retrier := newMockRetrier() - r := NewRequestRetrier(tc.uris, retrier, 2) - // first URI isn't a retry - firstURI, _ := r.GetNextURI(nil, nil) - require.NotEmpty(t, firstURI) - - retryURI, _ := r.GetNextURI(tc.resp, tc.respErr) - if tc.shouldRetry { - require.Contains(t, tc.uris, retryURI) - if tc.shouldRetrySameURI { - require.Equal(t, retryURI, firstURI) - } else { - require.NotEqual(t, retryURI, firstURI) - } - if tc.shouldRetryReset { - require.True(t, retrier.DidReset) - } - if tc.shouldRetryBackoff { - require.True(t, retrier.DidGetNext) - } - } else { - require.Empty(t, retryURI) - } - }) - } -} - -func newMockRetrier() *mockRetrier { - return &mockRetrier{ - DidGetNext: false, - DidReset: false, - } -} - -type mockRetrier struct { - DidGetNext bool - DidReset bool -} - -func (m *mockRetrier) Reset() { - m.DidReset = true -} + r := NewRequestRetrier(retry.Start(ctx), 0) -func (m *mockRetrier) Next() bool { - m.DidGetNext = true - return true + // No retries if context is candled + shouldRetry := r.Next(nil, nil) + require.False(t, shouldRetry) } -func (m *mockRetrier) CurrentAttempt() int { - return 0 -} +//func TestRequestRetrier_UsesLocationHeader(t *testing.T) { +// respWithLocationHeader := &http.Response{ +// StatusCode: StatusCodeRetryOther, +// Header: http.Header{"Location": []string{"http://example.com"}}, +// } +// +// r := NewRequestRetrier([]string{"a"}, retry.Start(context.Background()), 2) +// uri, isRelocated := r.GetNextURI(nil, nil) +// require.Equal(t, uri, "a") +// require.False(t, isRelocated) +// +// uri, isRelocated = r.GetNextURI(respWithLocationHeader, nil) +// require.Equal(t, uri, "http://example.com") +// require.True(t, isRelocated) +//} +// +//func TestRequestRetrier_UsesLocationFromErr(t *testing.T) { +// r := NewRequestRetrier([]string{"http://example-1.com"}, retry.Start(context.Background()), 2) +// respErr := werror.ErrorWithContextParams(context.Background(), "307", +// werror.SafeParam("statusCode", 307), +// werror.SafeParam("location", "http://example-2.com")) +// +// uri, isRelocated := r.GetNextURI(nil, nil) +// require.Equal(t, uri, "http://example-1.com") +// require.False(t, isRelocated) +// +// uri, isRelocated = r.GetNextURI(nil, respErr) +// require.Equal(t, uri, "http://example-2.com") +// require.True(t, isRelocated) +//} +// +//func TestRequestRetrier_GetNextURI(t *testing.T) { +// for _, tc := range []struct { +// name string +// resp *http.Response +// respErr error +// uris []string +// shouldRetry bool +// shouldRetrySameURI bool +// shouldRetryBackoff bool +// shouldRetryReset bool +// }{ +// { +// name: "returns error if response exists and doesn't appear retryable", +// resp: &http.Response{}, +// respErr: nil, +// uris: []string{"a", "b"}, +// shouldRetry: false, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "returns error if error code not retryable", +// resp: &http.Response{}, +// respErr: nil, +// uris: []string{"a", "b"}, +// shouldRetry: false, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "returns a URI if response and error are nil", +// resp: nil, +// respErr: nil, +// uris: []string{"a", "b"}, +// shouldRetry: true, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "returns a URI if response and error are nil", +// resp: nil, +// respErr: nil, +// uris: []string{"a", "b"}, +// shouldRetry: true, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "retries and backs off the single URI if response and error are nil", +// resp: nil, +// respErr: nil, +// uris: []string{"a"}, +// shouldRetry: true, +// shouldRetrySameURI: true, +// shouldRetryBackoff: true, +// shouldRetryReset: false, +// }, +// { +// name: "returns a new URI if unavailable", +// resp: nil, +// respErr: werror.ErrorWithContextParams(context.Background(), "503", werror.SafeParam("statusCode", 503)), +// uris: []string{"a", "b"}, +// shouldRetry: true, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "retries and backs off the single URI if unavailable", +// resp: nil, +// respErr: werror.ErrorWithContextParams(context.Background(), "503", werror.SafeParam("statusCode", 503)), +// uris: []string{"a"}, +// shouldRetry: true, +// shouldRetrySameURI: true, +// shouldRetryBackoff: true, +// shouldRetryReset: false, +// }, +// { +// name: "returns a new URI and backs off if throttled", +// resp: nil, +// respErr: werror.ErrorWithContextParams(context.Background(), "429", werror.SafeParam("statusCode", 429)), +// uris: []string{"a", "b"}, +// shouldRetry: true, +// shouldRetrySameURI: false, +// shouldRetryBackoff: true, +// shouldRetryReset: false, +// }, +// { +// name: "retries single URI and backs off if throttled", +// resp: nil, +// respErr: werror.ErrorWithContextParams(context.Background(), "429", werror.SafeParam("statusCode", 429)), +// uris: []string{"a"}, +// shouldRetry: true, +// shouldRetrySameURI: true, +// shouldRetryBackoff: true, +// shouldRetryReset: false, +// }, +// { +// name: "retries another URI if gets retry other response without location", +// resp: &http.Response{ +// StatusCode: StatusCodeRetryOther, +// }, +// respErr: nil, +// uris: []string{"a", "b"}, +// shouldRetry: true, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "retries single URI and backs off if gets retry other response without location", +// resp: &http.Response{ +// StatusCode: StatusCodeRetryOther, +// }, +// respErr: nil, +// uris: []string{"a"}, +// shouldRetry: true, +// shouldRetrySameURI: true, +// shouldRetryBackoff: true, +// shouldRetryReset: false, +// }, +// { +// name: "retries another URI if gets retry temporary redirect response without location", +// resp: &http.Response{ +// StatusCode: StatusCodeRetryTemporaryRedirect, +// }, +// respErr: nil, +// uris: []string{"a", "b"}, +// shouldRetry: true, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "retries single URI and backs off if gets retry temporary redirect response without location", +// resp: &http.Response{ +// StatusCode: StatusCodeRetryTemporaryRedirect, +// }, +// respErr: nil, +// uris: []string{"a"}, +// shouldRetry: true, +// shouldRetrySameURI: true, +// shouldRetryBackoff: true, +// shouldRetryReset: false, +// }, +// { +// name: "does not retry 400 responses", +// resp: &http.Response{ +// StatusCode: 400, +// }, +// uris: []string{"a", "b"}, +// shouldRetry: false, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "does not retry 404 responses", +// resp: &http.Response{ +// StatusCode: 404, +// }, +// uris: []string{"a", "b"}, +// shouldRetry: false, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "does not retry 400 errors", +// respErr: werror.ErrorWithContextParams(context.Background(), "400", werror.SafeParam("statusCode", 400)), +// uris: []string{"a", "b"}, +// shouldRetry: false, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// { +// name: "does not retry 404s", +// respErr: werror.ErrorWithContextParams(context.Background(), "404", werror.SafeParam("statusCode", 404)), +// uris: []string{"a", "b"}, +// shouldRetry: false, +// shouldRetrySameURI: false, +// shouldRetryBackoff: false, +// shouldRetryReset: false, +// }, +// } { +// t.Run(tc.name, func(t *testing.T) { +// retrier := newMockRetrier() +// r := NewRequestRetrier(tc.uris, retrier, 2) +// // first URI isn't a retry +// firstURI, _ := r.GetNextURI(nil, nil) +// require.NotEmpty(t, firstURI) +// +// retryURI, _ := r.GetNextURI(tc.resp, tc.respErr) +// if tc.shouldRetry { +// require.Contains(t, tc.uris, retryURI) +// if tc.shouldRetrySameURI { +// require.Equal(t, retryURI, firstURI) +// } else { +// require.NotEqual(t, retryURI, firstURI) +// } +// if tc.shouldRetryReset { +// require.True(t, retrier.DidReset) +// } +// if tc.shouldRetryBackoff { +// require.True(t, retrier.DidGetNext) +// } +// } else { +// require.Empty(t, retryURI) +// } +// }) +// } +//} +// +//func newMockRetrier() *mockRetrier { +// return &mockRetrier{ +// DidGetNext: false, +// DidReset: false, +// } +//} +// +//type mockRetrier struct { +// DidGetNext bool +// DidReset bool +//} +// +//func (m *mockRetrier) Reset() { +// m.DidReset = true +//} +// +//func (m *mockRetrier) Next() bool { +// m.DidGetNext = true +// return true +//} +// +//func (m *mockRetrier) CurrentAttempt() int { +// return 0 +//} +// diff --git a/conjure-go-client/httpclient/internal/retry.go b/conjure-go-client/httpclient/internal/retry.go index f4c6b484..c996e8c4 100644 --- a/conjure-go-client/httpclient/internal/retry.go +++ b/conjure-go-client/httpclient/internal/retry.go @@ -90,6 +90,8 @@ func parseLocationURL(locationStr string) *url.URL { return locationURL } +// isThrottleResponse returns true if the response a throttle response type. It +// also returns a duration after which the failed URI can be retried func isThrottleResponse(resp *http.Response, errCode int) (bool, time.Duration) { if errCode == StatusCodeThrottle { return true, 0 diff --git a/conjure-go-client/httpclient/internal/stateful_uri_pool.go b/conjure-go-client/httpclient/internal/stateful_uri_pool.go new file mode 100644 index 00000000..99cce25d --- /dev/null +++ b/conjure-go-client/httpclient/internal/stateful_uri_pool.go @@ -0,0 +1,125 @@ +package internal + +import ( + "net/http" + "net/url" + "sync" + "time" + + "github.com/palantir/pkg/refreshable" +) + +const ( + // defaultResurrectDuration is the amount of time after which + // we resurrect failed URIs + defaultResurrectDuration = time.Second * 60 + meshSchemePrefix = "mesh-" +) + +type statefulURIPool struct { + sync.RWMutex + + uris []string + failedURIs map[string]struct{} +} + +// NewStatefulURIPool returns a URIPool that keeps track of a +// refeshable set of possible URIs. It can be used as middleware to track +// server side request failures to future requests from hitting known bad servers. +func NewStatefulURIPool(uris refreshable.StringSlice) URIPool { + s := &statefulURIPool{} + s.updateURIs(uris.CurrentStringSlice()) + + _ = uris.SubscribeToStringSlice(s.updateURIs) + return s +} + +// NumURIs implements URIPool +func (s *statefulURIPool) NumURIs() int { + s.RLock() + defer s.RUnlock() + + return len(s.uris) +} + +// URIs implements URIPool +func (s *statefulURIPool) URIs() []string { + s.RLock() + defer s.RUnlock() + + uris := make([]string, 0, len(s.uris)) + for _, uri := range s.uris { + if _, ok := s.failedURIs[uri]; ok { + continue + } + uris = append(uris, uri) + } + // if all connections are "failed", then return them all + if len(uris) == 0 { + return s.uris + } + return uris +} + +// RoundTrip implements URIPool +func (s *statefulURIPool) RoundTrip(req *http.Request, next http.RoundTripper) (*http.Response, error) { + resp, respErr := next.RoundTrip(req) + errCode, _ := StatusCodeFromError(respErr) + + if isThrottle, ressurectAfter := isThrottleResponse(resp, errCode); isThrottle { + s.markBackoffURI(req, ressurectAfter) + } + if isUnavailableResponse(resp, errCode) { + // 503: go to next node + s.markBackoffURI(req, defaultResurrectDuration) + } + // TODO(dtrejo): Do we need to handle redirects or does the underlying http.Client do that for us? + //if isTryOther, isPermenant, otherURI := isRetryOtherResponse(resp, respErr, errCode); isTryOther { + // s.markBackoffURI(req, defaultResurrectDuration) + //} + if errCode >= http.StatusBadRequest && errCode < http.StatusInternalServerError { + // nothing to do + } + if resp == nil { + // if we get a nil response, we can assume there is a problem with host and can move on to the next. + s.markBackoffURI(req, defaultResurrectDuration) + } + + return resp, respErr +} + +func (s *statefulURIPool) updateURIs(uris []string) { + result := make([]string, 0, len(uris)) + for _, uri := range uris { + // validate URIs by parsing them + u, err := url.Parse(uri) + if err != nil { + // ignore invalid uris + continue + } + result = append(result, getBaseURI(u)) + } + + s.Lock() + defer s.Unlock() + s.uris = result + s.failedURIs = make(map[string]struct{}, len(uris)) +} + +func (s *statefulURIPool) markBackoffURI(req *http.Request, dur time.Duration) { + // if duration is equal to zero, then don't mark the URI as failed + if dur == 0 { + dur = defaultResurrectDuration + } + reqURL := getBaseURI(req.URL) + s.Lock() + defer s.Unlock() + + s.failedURIs[reqURL] = struct{}{} + + time.AfterFunc(dur, func() { + s.Lock() + defer s.Unlock() + delete(s.failedURIs, reqURL) + }) +} diff --git a/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go b/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go new file mode 100644 index 00000000..cb8d08cf --- /dev/null +++ b/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go @@ -0,0 +1,220 @@ +package internal + +import ( + "net/http" + "testing" + + "github.com/palantir/pkg/refreshable" + "github.com/stretchr/testify/assert" +) + +//func TestRequestRetrier_UsesLocationFromErr(t *testing.T) { +// r := NewRequestRetrier([]string{"http://example-1.com"}, retry.Start(context.Background()), 2) +// respErr := werror.ErrorWithContextParams(context.Background(), "307", +// werror.SafeParam("statusCode", 307), +// werror.SafeParam("location", "http://example-2.com")) +// +// uri, isRelocated := r.GetNextURI(nil, nil) +// require.Equal(t, uri, "http://example-1.com") +// require.False(t, isRelocated) +// +// uri, isRelocated = r.GetNextURI(nil, respErr) +// require.Equal(t, uri, "http://example-2.com") +// require.True(t, isRelocated) +//} + +func TestRequestRetrier_GetNextURIs(t *testing.T) { + for _, tc := range []struct { + name string + resp *http.Response + chosenURI string + beforeURIs []string + afterURIs []string + shouldRetry bool + shouldRetrySameURI bool + shouldRetryBackoff bool + shouldRetryReset bool + }{ + { + name: "preserves chosen URI if response doesn't contain a handled status code", + resp: &http.Response{}, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + afterURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + }, + { + name: "remove chosen URI if response is nil", + resp: nil, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + afterURIs: []string{"https://domain1.example.com"}, + }, + { + name: "removes chosen URI if response return code is 503", + resp: &http.Response{StatusCode: http.StatusServiceUnavailable}, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + afterURIs: []string{"https://domain1.example.com"}, + }, + { + name: "preserves chosen URI if response return code is 503 but it's a single URI", + resp: &http.Response{StatusCode: http.StatusServiceUnavailable}, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com"}, + afterURIs: []string{"https://domain0.example.com"}, + }, + { + name: "removes chosen URI if response return code is 429", + resp: &http.Response{StatusCode: http.StatusTooManyRequests}, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + afterURIs: []string{"https://domain1.example.com"}, + }, + { + name: "preserves chosen URI if response return code is 429 but it's a single URI", + resp: &http.Response{StatusCode: http.StatusTooManyRequests}, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com"}, + afterURIs: []string{"https://domain0.example.com"}, + }, + //{ + // name: "retries another URI if gets retry other response without location", + // resp: &http.Response{StatusCode: StatusCodeRetryOther}, + // resp: &http.Response{ + // StatusCode: StatusCodeRetryOther, + // }, + // respErr: nil, + // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, + // shouldRetry: true, + // shouldRetrySameURI: false, + // shouldRetryBackoff: false, + // shouldRetryReset: false, + //}, + //{ + // name: "retries single URI and backs off if gets retry other response without location", + // resp: &http.Response{ + // StatusCode: StatusCodeRetryOther, + // }, + // respErr: nil, + // uris: []string{"https://domain0.example.com"}, + // shouldRetry: true, + // shouldRetrySameURI: true, + // shouldRetryBackoff: true, + // shouldRetryReset: false, + //}, + //{ + // name: "retries another URI if gets retry temporary redirect response without location", + // resp: &http.Response{ + // StatusCode: StatusCodeRetryTemporaryRedirect, + // }, + // respErr: nil, + // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, + // shouldRetry: true, + // shouldRetrySameURI: false, + // shouldRetryBackoff: false, + // shouldRetryReset: false, + //}, + //{ + // name: "retries single URI and backs off if gets retry temporary redirect response without location", + // resp: &http.Response{ + // StatusCode: StatusCodeRetryTemporaryRedirect, + // }, + // respErr: nil, + // uris: []string{"https://domain0.example.com"}, + // shouldRetry: true, + // shouldRetrySameURI: true, + // shouldRetryBackoff: true, + // shouldRetryReset: false, + //}, + //{ + // name: "does not retry 400 responses", + // resp: &http.Response{ + // StatusCode: 400, + // }, + // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, + // shouldRetry: false, + // shouldRetrySameURI: false, + // shouldRetryBackoff: false, + // shouldRetryReset: false, + //}, + //{ + // name: "does not retry 404 responses", + // resp: &http.Response{ + // StatusCode: 404, + // }, + // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, + // shouldRetry: false, + // shouldRetrySameURI: false, + // shouldRetryBackoff: false, + // shouldRetryReset: false, + //}, + //{ + // name: "does not retry 400 errors", + // respErr: werror.ErrorWithContextParams(context.Background(), "400", werror.SafeParam("statusCode", 400)), + // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, + // shouldRetry: false, + // shouldRetrySameURI: false, + // shouldRetryBackoff: false, + // shouldRetryReset: false, + //}, + //{ + // name: "does not retry 404s", + // respErr: werror.ErrorWithContextParams(context.Background(), "404", werror.SafeParam("statusCode", 404)), + // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, + // shouldRetry: false, + // shouldRetrySameURI: false, + // shouldRetryBackoff: false, + // shouldRetryReset: false, + //}, + } { + t.Run(tc.name, func(t *testing.T) { + //retrier := newMockRetrier() + //r := NewRequestRetrier(tc.uris, retrier, 2) + //// first URI isn't a retry + //firstURI, _ := r.GetNextURI(nil, nil) + //require.NotEmpty(t, firstURI) + + ref := refreshable.NewDefaultRefreshable(tc.beforeURIs) + pool := NewStatefulURIPool(refreshable.NewStringSlice(ref)) + + req, err := http.NewRequest("GET", tc.chosenURI, nil) + assert.NoError(t, err) + + resp, err := pool.RoundTrip(req, newMockRoundTripper(tc.resp)) + assert.Equal(t, tc.resp, resp) + assert.NoError(t, err) + + assert.ElementsMatch(t, tc.afterURIs, pool.URIs()) + + //retryURI, _ := r.GetNextURI(tc.resp, tc.respErr) + //if tc.shouldRetry { + // require.Contains(t, tc.uris, retryURI) + // if tc.shouldRetrySameURI { + // require.Equal(t, retryURI, firstURI) + // } else { + // require.NotEqual(t, retryURI, firstURI) + // } + // if tc.shouldRetryReset { + // require.True(t, retrier.DidReset) + // } + // if tc.shouldRetryBackoff { + // require.True(t, retrier.DidGetNext) + // } + //} else { + // require.Empty(t, retryURI) + //} + }) + } +} + +func newMockRoundTripper(resp *http.Response) http.RoundTripper { + return &mockRoundTripper{resp: resp} +} + +type mockRoundTripper struct { + resp *http.Response +} + +func (m mockRoundTripper) RoundTrip(req *http.Request) (*http.Response, error) { + return m.resp, nil +} From 7e3a61739626ac9e0d085a9eca54f34b3cd0bf1e Mon Sep 17 00:00:00 2001 From: Devin Trejo Date: Fri, 22 Jul 2022 18:54:07 -0400 Subject: [PATCH 2/9] Doc strings. --- .../httpclient/internal/balanced_selector.go | 14 ++++++++++++++ conjure-go-client/httpclient/internal/interface.go | 14 ++++++++++++++ .../httpclient/internal/request_retrier.go | 10 ++++++---- .../httpclient/internal/stateful_uri_pool.go | 14 ++++++++++++++ .../httpclient/internal/stateful_uri_pool_test.go | 14 ++++++++++++++ 5 files changed, 62 insertions(+), 4 deletions(-) diff --git a/conjure-go-client/httpclient/internal/balanced_selector.go b/conjure-go-client/httpclient/internal/balanced_selector.go index 23efa50c..0ed7bebc 100644 --- a/conjure-go-client/httpclient/internal/balanced_selector.go +++ b/conjure-go-client/httpclient/internal/balanced_selector.go @@ -1,3 +1,17 @@ +// Copyright (c) 2022 Palantir Technologies. All rights reserved. +// +// 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 internal import ( diff --git a/conjure-go-client/httpclient/internal/interface.go b/conjure-go-client/httpclient/internal/interface.go index 283e2c1f..73d9eaa1 100644 --- a/conjure-go-client/httpclient/internal/interface.go +++ b/conjure-go-client/httpclient/internal/interface.go @@ -1,3 +1,17 @@ +// Copyright (c) 2022 Palantir Technologies. All rights reserved. +// +// 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 internal import ( diff --git a/conjure-go-client/httpclient/internal/request_retrier.go b/conjure-go-client/httpclient/internal/request_retrier.go index 4fcd57b9..ca008cf1 100644 --- a/conjure-go-client/httpclient/internal/request_retrier.go +++ b/conjure-go-client/httpclient/internal/request_retrier.go @@ -23,7 +23,7 @@ import ( // RequestRetrier manages the lifecylce of a single request. It will tracks the // backoff timing between subsequent requests. The retrier should only suggest -// a URI if the previous request returned a redirect or is a mesh URI. In the +// a retry if the previous request returned a redirect or is a mesh URI. In the // case of a mesh URI being detected, the request retrier will only attempt the // request once. type RequestRetrier struct { @@ -55,9 +55,9 @@ func (r *RequestRetrier) attemptsRemaining() bool { } // Next returns true if a subsequent request attempt should be attempted. If -// uses the previous requestURI to determine if the request should be -// attempted. If the returned value is true, the retrier will have waited the -// desired backoff interval before returning. +// uses the previous request/response (if provided) to determine if the request +// should be attempted. If the returned value is true, the retrier will have +// waited the desired backoff interval before returning. func (r *RequestRetrier) Next(prevReq *http.Request, prevResp *http.Response) bool { defer func() { r.attemptCount++ }() // check for bad requests @@ -80,6 +80,8 @@ func (r *RequestRetrier) Next(prevReq *http.Request, prevResp *http.Response) bo } } + // TODO (dtrejo): Handle redirects? + if !r.attemptsRemaining() { // Retries exhausted return false diff --git a/conjure-go-client/httpclient/internal/stateful_uri_pool.go b/conjure-go-client/httpclient/internal/stateful_uri_pool.go index 99cce25d..544a8140 100644 --- a/conjure-go-client/httpclient/internal/stateful_uri_pool.go +++ b/conjure-go-client/httpclient/internal/stateful_uri_pool.go @@ -1,3 +1,17 @@ +// Copyright (c) 2022 Palantir Technologies. All rights reserved. +// +// 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 internal import ( diff --git a/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go b/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go index cb8d08cf..33aa28f7 100644 --- a/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go +++ b/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go @@ -1,3 +1,17 @@ +// Copyright (c) 2022 Palantir Technologies. All rights reserved. +// +// 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 internal import ( From 96b410a3141650fc173534de44ac528b46259790 Mon Sep 17 00:00:00 2001 From: Devin Trejo Date: Thu, 11 Aug 2022 12:38:18 -0400 Subject: [PATCH 3/9] Properly handle redirects in the request retrier. Fix expected behavior of client having round-robin uri selector. --- conjure-go-client/httpclient/client.go | 49 +++++++----- .../httpclient/client_builder.go | 2 +- .../httpclient/internal/balanced_selector.go | 2 +- .../internal/random_selector_test.go | 2 +- .../httpclient/internal/request_retrier.go | 74 ++++++++++++------- .../internal/request_retrier_test.go | 20 ++--- .../httpclient/internal/retry.go | 16 ++-- .../httpclient/internal/retry_test.go | 4 +- .../httpclient/internal/rr_selector.go | 49 ++++++++++++ .../httpclient/internal/rr_selector_test.go | 44 +++++++++++ .../response_error_decoder_middleware.go | 5 +- .../response_error_decoder_middleware_test.go | 2 +- 12 files changed, 203 insertions(+), 66 deletions(-) create mode 100644 conjure-go-client/httpclient/internal/rr_selector.go create mode 100644 conjure-go-client/httpclient/internal/rr_selector_test.go diff --git a/conjure-go-client/httpclient/client.go b/conjure-go-client/httpclient/client.go index bdc681ce..f2b11bf9 100644 --- a/conjure-go-client/httpclient/client.go +++ b/conjure-go-client/httpclient/client.go @@ -83,31 +83,34 @@ func (c *clientImpl) Delete(ctx context.Context, params ...RequestParam) (*http. } func (c *clientImpl) Do(ctx context.Context, params ...RequestParam) (*http.Response, error) { - uriCount := c.uriPool.NumURIs() - attempts := 2 * uriCount + attempts := 2 * c.uriPool.NumURIs() if c.maxAttempts != nil { if confMaxAttempts := c.maxAttempts.CurrentIntPtr(); confMaxAttempts != nil { attempts = *confMaxAttempts } } + var resp *http.Response var err error retrier := internal.NewRequestRetrier(c.backoffOptions.CurrentRetryParams().Start(ctx), attempts) - var req *http.Request - var resp *http.Response - for retrier.Next(req, resp) { - req, resp, err = c.doOnce(ctx, params...) + for { + shouldRetry, retryURL := retrier.Next(resp, err) + if !shouldRetry { + break + } + resp, err = c.doOnce(ctx, retryURL, params...) if err != nil { svc1log.FromContext(ctx).Debug("Retrying request", svc1log.Stacktrace(err)) } } - if err != nil { - return nil, err - } - return resp, nil + return resp, err } -func (c *clientImpl) doOnce(ctx context.Context, params ...RequestParam) (*http.Request, *http.Response, error) { +func (c *clientImpl) doOnce( + ctx context.Context, + retryURL *url.URL, + params ...RequestParam, +) (*http.Response, error) { // 1. create the request b := &requestBuilder{ headers: make(http.Header), @@ -120,7 +123,7 @@ func (c *clientImpl) doOnce(ctx context.Context, params ...RequestParam) (*http. continue } if err := p.apply(b); err != nil { - return nil, nil, err + return nil, err } } @@ -129,15 +132,23 @@ func (c *clientImpl) doOnce(ctx context.Context, params ...RequestParam) (*http. } if b.method == "" { - return nil, nil, werror.ErrorWithContextParams(ctx, "httpclient: use WithRequestMethod() to specify HTTP method") + return nil, werror.ErrorWithContextParams(ctx, "httpclient: use WithRequestMethod() to specify HTTP method") } - url, err := c.uriSelector.Select(c.uriPool.URIs(), b.headers) - if err != nil { - return nil, nil, werror.WrapWithContextParams(ctx, err, "failed to select uri") + var uri string + if retryURL == nil { + var err error + uri, err = c.uriSelector.Select(c.uriPool.URIs(), b.headers) + if err != nil { + return nil, werror.WrapWithContextParams(ctx, err, "failed to select uri") + } + uri = joinURIAndPath(uri, b.path) + } else { + b.path = "" + uri = retryURL.String() } - req, err := http.NewRequestWithContext(ctx, b.method, url, nil) + req, err := http.NewRequestWithContext(ctx, b.method, uri, nil) if err != nil { - return nil, nil, werror.WrapWithContextParams(ctx, err, "failed to build request") + return nil, werror.WrapWithContextParams(ctx, err, "failed to build request") } req.Header = b.headers @@ -176,7 +187,7 @@ func (c *clientImpl) doOnce(ctx context.Context, params ...RequestParam) (*http. internal.DrainBody(resp) } - return req, resp, unwrapURLError(ctx, respErr) + return resp, unwrapURLError(ctx, respErr) } // unwrapURLError converts a *url.Error to a werror. We need this because all diff --git a/conjure-go-client/httpclient/client_builder.go b/conjure-go-client/httpclient/client_builder.go index 8338fc27..f371d341 100644 --- a/conjure-go-client/httpclient/client_builder.go +++ b/conjure-go-client/httpclient/client_builder.go @@ -155,7 +155,7 @@ func newClient(ctx context.Context, b *clientBuilder, params ...ClientParam) (Cl } uriPool := internal.NewStatefulURIPool(b.URIs) if b.URISelector == nil { - b.URISelector = internal.NewRandomURISelector(func() int64 { return time.Now().UnixNano() }) + b.URISelector = internal.NewRoundRobinURISelector(func() int64 { return time.Now().UnixNano() }) } return &clientImpl{ client: httpClient, diff --git a/conjure-go-client/httpclient/internal/balanced_selector.go b/conjure-go-client/httpclient/internal/balanced_selector.go index 0ed7bebc..1d500e71 100644 --- a/conjure-go-client/httpclient/internal/balanced_selector.go +++ b/conjure-go-client/httpclient/internal/balanced_selector.go @@ -52,7 +52,7 @@ type balancedSelector struct { uriInfos map[string]uriInfo } -// Select implements estransport.Selector interface +// Select implements Selector interface func (s *balancedSelector) Select(uris []string, _ http.Header) (string, error) { s.Lock() defer s.Unlock() diff --git a/conjure-go-client/httpclient/internal/random_selector_test.go b/conjure-go-client/httpclient/internal/random_selector_test.go index ea902bdf..063baf5f 100644 --- a/conjure-go-client/httpclient/internal/random_selector_test.go +++ b/conjure-go-client/httpclient/internal/random_selector_test.go @@ -21,7 +21,7 @@ import ( "github.com/stretchr/testify/assert" ) -func TestRandomSelectorGetRandomURIs(t *testing.T) { +func TestRandomSelector_Select(t *testing.T) { uris := []string{"uri1", "uri2", "uri3", "uri4", "uri5"} scorer := NewRandomURISelector(func() int64 { return time.Now().UnixNano() }) uri, err := scorer.Select(uris, nil) diff --git a/conjure-go-client/httpclient/internal/request_retrier.go b/conjure-go-client/httpclient/internal/request_retrier.go index ca008cf1..7d778bae 100644 --- a/conjure-go-client/httpclient/internal/request_retrier.go +++ b/conjure-go-client/httpclient/internal/request_retrier.go @@ -16,6 +16,7 @@ package internal import ( "net/http" + "net/url" "strings" "github.com/palantir/pkg/retry" @@ -54,41 +55,64 @@ func (r *RequestRetrier) attemptsRemaining() bool { return r.attemptCount < r.maxAttempts } -// Next returns true if a subsequent request attempt should be attempted. If -// uses the previous request/response (if provided) to determine if the request -// should be attempted. If the returned value is true, the retrier will have -// waited the desired backoff interval before returning. -func (r *RequestRetrier) Next(prevReq *http.Request, prevResp *http.Response) bool { +// Next returns true if a subsequent request attempt should be attempted. If uses the previous response/resp err (if +// provided) to determine if the request should be attempted. If the returned value is true, the retrier will have +// waited the desired backoff interval before returning when applicable. If the previous response was a redirect, the +// retrier will also return the URL that should be used for the new next request. +func (r *RequestRetrier) Next(resp *http.Response, err error) (bool, *url.URL) { defer func() { r.attemptCount++ }() - // check for bad requests - if prevResp != nil { - prevCode := prevResp.StatusCode - // succesfull response - if prevCode == http.StatusOK { - return false - } - if prevCode >= http.StatusBadRequest && prevCode < http.StatusInternalServerError { - return false - } + if r.isSuccess(resp) { + return false, nil } - // don't retry mesh uris - if prevReq != nil { - prevURI := getBaseURI(prevReq.URL) - if r.isMeshURI(prevURI) { - return false - } + if r.isNonRetryableClientError(resp, err) { + return false, nil + } + + // handle redirects + if tryOther, otherURI := isRetryOtherResponse(resp, err); tryOther && otherURI != nil { + return true, otherURI } - // TODO (dtrejo): Handle redirects? + // don't retry mesh uris + if r.isMeshURI(resp) { + return false, nil + } if !r.attemptsRemaining() { // Retries exhausted + return false, nil + } + return r.retrier.Next(), nil +} + +func (*RequestRetrier) isSuccess(resp *http.Response) bool { + if resp == nil { + return false + } + // Check for a 2XX status + return resp.StatusCode >= 200 && resp.StatusCode < 300 +} + +func (*RequestRetrier) isNonRetryableClientError(resp *http.Response, err error) bool { + errCode, _ := StatusCodeFromError(err) + // Check for a 4XX status parsed from the error or in the response + if isClientError(errCode) && errCode != StatusCodeThrottle { return false } - return r.retrier.Next() + if resp != nil && isClientError(resp.StatusCode) { + // 429 is retryable + if isThrottle, _ := isThrottleResponse(resp, errCode); !isThrottle { + return false + } + return true + } + return false } -func (*RequestRetrier) isMeshURI(uri string) bool { - return strings.HasPrefix(uri, meshSchemePrefix) +func (*RequestRetrier) isMeshURI(resp *http.Response) bool { + if resp == nil { + return false + } + return strings.HasPrefix(getBaseURI(resp.Request.URL), meshSchemePrefix) } diff --git a/conjure-go-client/httpclient/internal/request_retrier_test.go b/conjure-go-client/httpclient/internal/request_retrier_test.go index ee782069..537a76a8 100644 --- a/conjure-go-client/httpclient/internal/request_retrier_test.go +++ b/conjure-go-client/httpclient/internal/request_retrier_test.go @@ -31,7 +31,7 @@ func TestRequestRetrier_HandleMeshURI(t *testing.T) { r := NewRequestRetrier(retry.Start(context.Background()), 1) req, err := http.NewRequest("GET", "mesh-http://example.com", nil) require.NoError(t, err) - shouldRetry := r.Next(req, &http.Response{}) + shouldRetry, _ := r.Next(&http.Response{Request: req}, nil) require.False(t, shouldRetry) } @@ -39,19 +39,19 @@ func TestRequestRetrier_AttemptCount(t *testing.T) { maxAttempts := 3 r := NewRequestRetrier(retry.Start(context.Background()), maxAttempts) // first request is not a retry, so it doesn't increment the overall count - shouldRetry := r.Next(nil, nil) + shouldRetry, _ := r.Next(nil, nil) require.True(t, shouldRetry) for i := 0; i < maxAttempts-1; i++ { req, err := http.NewRequest("GET", "http://example.com", nil) require.NoError(t, err) - shouldRetry = r.Next(req, &http.Response{}) + shouldRetry, _ = r.Next(&http.Response{Request: req}, err) require.True(t, shouldRetry) } req, err := http.NewRequest("GET", "http://example.com", nil) require.NoError(t, err) - shouldRetry = r.Next(req, &http.Response{}) + shouldRetry, _ = r.Next(&http.Response{Request: req}, err) require.False(t, shouldRetry) } @@ -62,28 +62,28 @@ func TestRequestRetrier_UnlimitedAttempts(t *testing.T) { r := NewRequestRetrier(retry.Start(ctx, retry.WithInitialBackoff(50*time.Millisecond), retry.WithRandomizationFactor(0)), 0) startTime := time.Now() - shouldRetry := r.Next(nil, nil) + shouldRetry, _ := r.Next(nil, nil) require.True(t, shouldRetry) require.Lessf(t, time.Since(startTime), 49*time.Millisecond, "first GetNextURI should not have any delay") req, err := http.NewRequest("GET", "http://example.com", nil) require.NoError(t, err) - resp := &http.Response{} + resp := &http.Response{Request: req} startTime = time.Now() - shouldRetry = r.Next(req, resp) + shouldRetry, _ = r.Next(resp, err) require.True(t, shouldRetry) assert.Greater(t, time.Since(startTime), 50*time.Millisecond, "delay should be at least 1 backoff") assert.Less(t, time.Since(startTime), 100*time.Millisecond, "delay should be less than 2 backoffs") startTime = time.Now() - shouldRetry = r.Next(req, resp) + shouldRetry, _ = r.Next(resp, err) require.True(t, shouldRetry) assert.Greater(t, time.Since(startTime), 100*time.Millisecond, "delay should be at least 2 backoffs") assert.Less(t, time.Since(startTime), 200*time.Millisecond, "delay should be less than 3 backoffs") // Success should stop retries - shouldRetry = r.Next(req, &http.Response{StatusCode: http.StatusOK}) + shouldRetry, _ = r.Next(&http.Response{Request: req, StatusCode: http.StatusOK}, nil) require.False(t, shouldRetry) } @@ -94,7 +94,7 @@ func TestRequestRetrier_ContextCanceled(t *testing.T) { r := NewRequestRetrier(retry.Start(ctx), 0) // No retries if context is candled - shouldRetry := r.Next(nil, nil) + shouldRetry, _ := r.Next(nil, nil) require.False(t, shouldRetry) } diff --git a/conjure-go-client/httpclient/internal/retry.go b/conjure-go-client/httpclient/internal/retry.go index c996e8c4..4fe4247d 100644 --- a/conjure-go-client/httpclient/internal/retry.go +++ b/conjure-go-client/httpclient/internal/retry.go @@ -59,12 +59,15 @@ const ( StatusCodeUnavailable = http.StatusServiceUnavailable ) -func isRetryOtherResponse(resp *http.Response, err error, errCode int) (bool, *url.URL) { +func isRetryOtherResponse(resp *http.Response, err error) (bool, *url.URL) { + errCode, _ := StatusCodeFromError(err) + // prioritize redirect from werror first if errCode == StatusCodeRetryOther || errCode == StatusCodeRetryTemporaryRedirect { locationStr, ok := LocationFromError(err) - if ok { - return true, parseLocationURL(locationStr) + if !ok { + return true, nil } + return true, parseLocationURL(locationStr) } if resp == nil { @@ -74,8 +77,11 @@ func isRetryOtherResponse(resp *http.Response, err error, errCode int) (bool, *u resp.StatusCode != StatusCodeRetryTemporaryRedirect { return false, nil } - locationStr := resp.Header.Get("Location") - return true, parseLocationURL(locationStr) + location, err := resp.Location() + if err != nil { + return true, nil + } + return true, location } func parseLocationURL(locationStr string) *url.URL { diff --git a/conjure-go-client/httpclient/internal/retry_test.go b/conjure-go-client/httpclient/internal/retry_test.go index 78c90856..f5595ddb 100644 --- a/conjure-go-client/httpclient/internal/retry_test.go +++ b/conjure-go-client/httpclient/internal/retry_test.go @@ -129,14 +129,14 @@ func TestRetryResponseParsers(t *testing.T) { }, } { t.Run(test.Name, func(t *testing.T) { - errCode, _ := StatusCodeFromError(test.RespErr) - isRetryOther, retryOtherURL := isRetryOtherResponse(test.Response, test.RespErr, errCode) + isRetryOther, retryOtherURL := isRetryOtherResponse(test.Response, test.RespErr) if assert.Equal(t, test.IsRetryOther, isRetryOther) && test.RetryOtherURL != "" { if assert.NotNil(t, retryOtherURL) { assert.Equal(t, test.RetryOtherURL, retryOtherURL.String()) } } + errCode, _ := StatusCodeFromError(test.RespErr) isThrottle, throttleDur := isThrottleResponse(test.Response, errCode) if assert.Equal(t, test.IsThrottle, isThrottle) { assert.WithinDuration(t, time.Now().Add(test.ThrottleDuration), time.Now().Add(throttleDur), time.Second) diff --git a/conjure-go-client/httpclient/internal/rr_selector.go b/conjure-go-client/httpclient/internal/rr_selector.go new file mode 100644 index 00000000..fa4dccf8 --- /dev/null +++ b/conjure-go-client/httpclient/internal/rr_selector.go @@ -0,0 +1,49 @@ +// Copyright (c) 2022 Palantir Technologies. All rights reserved. +// +// 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 internal + +import ( + "net/http" + "sync" +) + +type roundRobinSelector struct { + sync.Mutex + nanoClock func() int64 + + offset int +} + +// NewRoundRobinURISelector returns a URI scorer that uses a round robin algorithm for selecting URIs when scoring +// using a rand.Rand seeded by the nanoClock function. The middleware no-ops on each request. +func NewRoundRobinURISelector(nanoClock func() int64) URISelector { + return &roundRobinSelector{ + nanoClock: nanoClock, + } +} + +// Select implements Selector interface +func (s *roundRobinSelector) Select(uris []string, _ http.Header) (string, error) { + s.Lock() + defer s.Unlock() + + s.offset = (s.offset + 1) % len(uris) + + return uris[s.offset], nil +} + +func (s *roundRobinSelector) RoundTrip(req *http.Request, next http.RoundTripper) (*http.Response, error) { + return next.RoundTrip(req) +} diff --git a/conjure-go-client/httpclient/internal/rr_selector_test.go b/conjure-go-client/httpclient/internal/rr_selector_test.go new file mode 100644 index 00000000..d1da545d --- /dev/null +++ b/conjure-go-client/httpclient/internal/rr_selector_test.go @@ -0,0 +1,44 @@ +// Copyright (c) 2022 Palantir Technologies. All rights reserved. +// +// 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 internal + +import ( + "testing" + "time" + + "github.com/stretchr/testify/assert" +) + +func TestRoundRobinSelector_Select(t *testing.T) { + uris := []string{"uri1", "uri2", "uri3", "uri4", "uri5"} + scorer := NewRoundRobinURISelector(func() int64 { return time.Now().UnixNano() }) + + const iterations = 100 + observed := make(map[string]int, iterations) + for i := 0; i < iterations; i++ { + uri, err := scorer.Select(uris, nil) + assert.NoError(t, err) + observed[uri] = observed[uri] + 1 + } + + occurences := make([]int, 0, len(observed)) + for _, count := range observed { + occurences = append(occurences, count) + } + + for _, v := range occurences { + assert.Equal(t, occurences[0], v) + } +} diff --git a/conjure-go-client/httpclient/response_error_decoder_middleware.go b/conjure-go-client/httpclient/response_error_decoder_middleware.go index 318dcaea..93d9a9cb 100644 --- a/conjure-go-client/httpclient/response_error_decoder_middleware.go +++ b/conjure-go-client/httpclient/response_error_decoder_middleware.go @@ -81,7 +81,10 @@ func (d restErrorDecoder) DecodeError(resp *http.Response) error { unsafeParams := map[string]interface{}{} if resp.StatusCode >= http.StatusTemporaryRedirect && resp.StatusCode < http.StatusBadRequest { - unsafeParams["location"] = resp.Header.Get("Location") + location, err := resp.Location() + if err == nil { + unsafeParams["location"] = location.String() + } } wSafeParams := werror.SafeParams(safeParams) wUnsafeParams := werror.UnsafeParams(unsafeParams) diff --git a/conjure-go-client/httpclient/response_error_decoder_middleware_test.go b/conjure-go-client/httpclient/response_error_decoder_middleware_test.go index 1e01dbbe..b15eb639 100644 --- a/conjure-go-client/httpclient/response_error_decoder_middleware_test.go +++ b/conjure-go-client/httpclient/response_error_decoder_middleware_test.go @@ -65,7 +65,7 @@ func TestErrorDecoderMiddlewares(t *testing.T) { assert.True(t, ok) assert.Equal(t, 307, code) location, ok := httpclient.LocationFromError(err) - assert.True(t, ok) + assert.False(t, ok) assert.Equal(t, "", location) }, }, From a060bc29ebba4ea010d94aecb15a56d76e3ccf0b Mon Sep 17 00:00:00 2001 From: svc-changelog Date: Thu, 28 Jul 2022 20:31:10 +0000 Subject: [PATCH 4/9] Add generated changelog entries --- changelog/@unreleased/pr-341.v2.yml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 changelog/@unreleased/pr-341.v2.yml diff --git a/changelog/@unreleased/pr-341.v2.yml b/changelog/@unreleased/pr-341.v2.yml new file mode 100644 index 00000000..9303234e --- /dev/null +++ b/changelog/@unreleased/pr-341.v2.yml @@ -0,0 +1,5 @@ +type: improvement +improvement: + description: Introduce new URIPool and URISelector interfaces. + links: + - https://github.com/palantir/conjure-go-runtime/pull/341 From a1a87f5fea6108a07c9fcedbbb83d601dfd6cd3b Mon Sep 17 00:00:00 2001 From: Devin Trejo Date: Mon, 15 Aug 2022 15:39:42 -0400 Subject: [PATCH 5/9] Minimize diff with develop. Uncomment tests for request retrier handling redirects. Uncomment relevant tests in stateful uri pool for failed uris. --- conjure-go-client/httpclient/client.go | 13 +- .../internal/balanced_selector_test.go | 4 - .../httpclient/internal/request_retrier.go | 2 +- .../internal/request_retrier_test.go | 497 ++++++++---------- .../httpclient/internal/stateful_uri_pool.go | 7 - .../internal/stateful_uri_pool_test.go | 170 ++---- 6 files changed, 262 insertions(+), 431 deletions(-) diff --git a/conjure-go-client/httpclient/client.go b/conjure-go-client/httpclient/client.go index f2b11bf9..9d439fcb 100644 --- a/conjure-go-client/httpclient/client.go +++ b/conjure-go-client/httpclient/client.go @@ -84,14 +84,17 @@ func (c *clientImpl) Delete(ctx context.Context, params ...RequestParam) (*http. func (c *clientImpl) Do(ctx context.Context, params ...RequestParam) (*http.Response, error) { attempts := 2 * c.uriPool.NumURIs() + if attempts == 0 { + return nil, werror.ErrorWithContextParams(ctx, "no base URIs are configured") + } if c.maxAttempts != nil { if confMaxAttempts := c.maxAttempts.CurrentIntPtr(); confMaxAttempts != nil { attempts = *confMaxAttempts } } - var resp *http.Response var err error + var resp *http.Response retrier := internal.NewRequestRetrier(c.backoffOptions.CurrentRetryParams().Start(ctx), attempts) for { shouldRetry, retryURL := retrier.Next(resp, err) @@ -99,9 +102,10 @@ func (c *clientImpl) Do(ctx context.Context, params ...RequestParam) (*http.Resp break } resp, err = c.doOnce(ctx, retryURL, params...) - if err != nil { - svc1log.FromContext(ctx).Debug("Retrying request", svc1log.Stacktrace(err)) + if err == nil { + break } + svc1log.FromContext(ctx).Debug("Retrying request", svc1log.Stacktrace(err)) } return resp, err } @@ -143,12 +147,11 @@ func (c *clientImpl) doOnce( } uri = joinURIAndPath(uri, b.path) } else { - b.path = "" uri = retryURL.String() } req, err := http.NewRequestWithContext(ctx, b.method, uri, nil) if err != nil { - return nil, werror.WrapWithContextParams(ctx, err, "failed to build request") + return nil, werror.WrapWithContextParams(ctx, err, "failed to build new HTTP request") } req.Header = b.headers diff --git a/conjure-go-client/httpclient/internal/balanced_selector_test.go b/conjure-go-client/httpclient/internal/balanced_selector_test.go index df23893e..18de4ada 100644 --- a/conjure-go-client/httpclient/internal/balanced_selector_test.go +++ b/conjure-go-client/httpclient/internal/balanced_selector_test.go @@ -17,7 +17,6 @@ package internal import ( "net/http" "net/http/httptest" - "net/url" "testing" "github.com/stretchr/testify/assert" @@ -53,9 +52,6 @@ func TestBalancedSelect(t *testing.T) { req, err := http.NewRequest("GET", uri, nil) assert.NoError(t, err) - url, err := url.Parse(uri) - assert.NoError(t, err) - req.URL = url _, err = scorer.RoundTrip(req, server.Client().Transport) assert.NoError(t, err) } diff --git a/conjure-go-client/httpclient/internal/request_retrier.go b/conjure-go-client/httpclient/internal/request_retrier.go index 7d778bae..446e448a 100644 --- a/conjure-go-client/httpclient/internal/request_retrier.go +++ b/conjure-go-client/httpclient/internal/request_retrier.go @@ -111,7 +111,7 @@ func (*RequestRetrier) isNonRetryableClientError(resp *http.Response, err error) } func (*RequestRetrier) isMeshURI(resp *http.Response) bool { - if resp == nil { + if resp == nil || resp.Request == nil { return false } return strings.HasPrefix(getBaseURI(resp.Request.URL), meshSchemePrefix) diff --git a/conjure-go-client/httpclient/internal/request_retrier_test.go b/conjure-go-client/httpclient/internal/request_retrier_test.go index 537a76a8..5c4dbd29 100644 --- a/conjure-go-client/httpclient/internal/request_retrier_test.go +++ b/conjure-go-client/httpclient/internal/request_retrier_test.go @@ -17,16 +17,16 @@ package internal import ( "context" "net/http" + "net/url" "testing" "time" "github.com/palantir/pkg/retry" + werror "github.com/palantir/witchcraft-go-error" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) -//var _ retry.Retrier = &mockRetrier{} - func TestRequestRetrier_HandleMeshURI(t *testing.T) { r := NewRequestRetrier(retry.Start(context.Background()), 1) req, err := http.NewRequest("GET", "mesh-http://example.com", nil) @@ -93,282 +93,225 @@ func TestRequestRetrier_ContextCanceled(t *testing.T) { r := NewRequestRetrier(retry.Start(ctx), 0) - // No retries if context is candled + // No retries if context is cancelled shouldRetry, _ := r.Next(nil, nil) require.False(t, shouldRetry) } -//func TestRequestRetrier_UsesLocationHeader(t *testing.T) { -// respWithLocationHeader := &http.Response{ -// StatusCode: StatusCodeRetryOther, -// Header: http.Header{"Location": []string{"http://example.com"}}, -// } -// -// r := NewRequestRetrier([]string{"a"}, retry.Start(context.Background()), 2) -// uri, isRelocated := r.GetNextURI(nil, nil) -// require.Equal(t, uri, "a") -// require.False(t, isRelocated) -// -// uri, isRelocated = r.GetNextURI(respWithLocationHeader, nil) -// require.Equal(t, uri, "http://example.com") -// require.True(t, isRelocated) -//} -// -//func TestRequestRetrier_UsesLocationFromErr(t *testing.T) { -// r := NewRequestRetrier([]string{"http://example-1.com"}, retry.Start(context.Background()), 2) -// respErr := werror.ErrorWithContextParams(context.Background(), "307", -// werror.SafeParam("statusCode", 307), -// werror.SafeParam("location", "http://example-2.com")) -// -// uri, isRelocated := r.GetNextURI(nil, nil) -// require.Equal(t, uri, "http://example-1.com") -// require.False(t, isRelocated) -// -// uri, isRelocated = r.GetNextURI(nil, respErr) -// require.Equal(t, uri, "http://example-2.com") -// require.True(t, isRelocated) -//} -// -//func TestRequestRetrier_GetNextURI(t *testing.T) { -// for _, tc := range []struct { -// name string -// resp *http.Response -// respErr error -// uris []string -// shouldRetry bool -// shouldRetrySameURI bool -// shouldRetryBackoff bool -// shouldRetryReset bool -// }{ -// { -// name: "returns error if response exists and doesn't appear retryable", -// resp: &http.Response{}, -// respErr: nil, -// uris: []string{"a", "b"}, -// shouldRetry: false, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "returns error if error code not retryable", -// resp: &http.Response{}, -// respErr: nil, -// uris: []string{"a", "b"}, -// shouldRetry: false, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "returns a URI if response and error are nil", -// resp: nil, -// respErr: nil, -// uris: []string{"a", "b"}, -// shouldRetry: true, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "returns a URI if response and error are nil", -// resp: nil, -// respErr: nil, -// uris: []string{"a", "b"}, -// shouldRetry: true, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "retries and backs off the single URI if response and error are nil", -// resp: nil, -// respErr: nil, -// uris: []string{"a"}, -// shouldRetry: true, -// shouldRetrySameURI: true, -// shouldRetryBackoff: true, -// shouldRetryReset: false, -// }, -// { -// name: "returns a new URI if unavailable", -// resp: nil, -// respErr: werror.ErrorWithContextParams(context.Background(), "503", werror.SafeParam("statusCode", 503)), -// uris: []string{"a", "b"}, -// shouldRetry: true, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "retries and backs off the single URI if unavailable", -// resp: nil, -// respErr: werror.ErrorWithContextParams(context.Background(), "503", werror.SafeParam("statusCode", 503)), -// uris: []string{"a"}, -// shouldRetry: true, -// shouldRetrySameURI: true, -// shouldRetryBackoff: true, -// shouldRetryReset: false, -// }, -// { -// name: "returns a new URI and backs off if throttled", -// resp: nil, -// respErr: werror.ErrorWithContextParams(context.Background(), "429", werror.SafeParam("statusCode", 429)), -// uris: []string{"a", "b"}, -// shouldRetry: true, -// shouldRetrySameURI: false, -// shouldRetryBackoff: true, -// shouldRetryReset: false, -// }, -// { -// name: "retries single URI and backs off if throttled", -// resp: nil, -// respErr: werror.ErrorWithContextParams(context.Background(), "429", werror.SafeParam("statusCode", 429)), -// uris: []string{"a"}, -// shouldRetry: true, -// shouldRetrySameURI: true, -// shouldRetryBackoff: true, -// shouldRetryReset: false, -// }, -// { -// name: "retries another URI if gets retry other response without location", -// resp: &http.Response{ -// StatusCode: StatusCodeRetryOther, -// }, -// respErr: nil, -// uris: []string{"a", "b"}, -// shouldRetry: true, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "retries single URI and backs off if gets retry other response without location", -// resp: &http.Response{ -// StatusCode: StatusCodeRetryOther, -// }, -// respErr: nil, -// uris: []string{"a"}, -// shouldRetry: true, -// shouldRetrySameURI: true, -// shouldRetryBackoff: true, -// shouldRetryReset: false, -// }, -// { -// name: "retries another URI if gets retry temporary redirect response without location", -// resp: &http.Response{ -// StatusCode: StatusCodeRetryTemporaryRedirect, -// }, -// respErr: nil, -// uris: []string{"a", "b"}, -// shouldRetry: true, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "retries single URI and backs off if gets retry temporary redirect response without location", -// resp: &http.Response{ -// StatusCode: StatusCodeRetryTemporaryRedirect, -// }, -// respErr: nil, -// uris: []string{"a"}, -// shouldRetry: true, -// shouldRetrySameURI: true, -// shouldRetryBackoff: true, -// shouldRetryReset: false, -// }, -// { -// name: "does not retry 400 responses", -// resp: &http.Response{ -// StatusCode: 400, -// }, -// uris: []string{"a", "b"}, -// shouldRetry: false, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "does not retry 404 responses", -// resp: &http.Response{ -// StatusCode: 404, -// }, -// uris: []string{"a", "b"}, -// shouldRetry: false, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "does not retry 400 errors", -// respErr: werror.ErrorWithContextParams(context.Background(), "400", werror.SafeParam("statusCode", 400)), -// uris: []string{"a", "b"}, -// shouldRetry: false, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// { -// name: "does not retry 404s", -// respErr: werror.ErrorWithContextParams(context.Background(), "404", werror.SafeParam("statusCode", 404)), -// uris: []string{"a", "b"}, -// shouldRetry: false, -// shouldRetrySameURI: false, -// shouldRetryBackoff: false, -// shouldRetryReset: false, -// }, -// } { -// t.Run(tc.name, func(t *testing.T) { -// retrier := newMockRetrier() -// r := NewRequestRetrier(tc.uris, retrier, 2) -// // first URI isn't a retry -// firstURI, _ := r.GetNextURI(nil, nil) -// require.NotEmpty(t, firstURI) -// -// retryURI, _ := r.GetNextURI(tc.resp, tc.respErr) -// if tc.shouldRetry { -// require.Contains(t, tc.uris, retryURI) -// if tc.shouldRetrySameURI { -// require.Equal(t, retryURI, firstURI) -// } else { -// require.NotEqual(t, retryURI, firstURI) -// } -// if tc.shouldRetryReset { -// require.True(t, retrier.DidReset) -// } -// if tc.shouldRetryBackoff { -// require.True(t, retrier.DidGetNext) -// } -// } else { -// require.Empty(t, retryURI) -// } -// }) -// } -//} -// -//func newMockRetrier() *mockRetrier { -// return &mockRetrier{ -// DidGetNext: false, -// DidReset: false, -// } -//} -// -//type mockRetrier struct { -// DidGetNext bool -// DidReset bool -//} -// -//func (m *mockRetrier) Reset() { -// m.DidReset = true -//} -// -//func (m *mockRetrier) Next() bool { -// m.DidGetNext = true -// return true -//} -// -//func (m *mockRetrier) CurrentAttempt() int { -// return 0 -//} -// +func TestRequestRetrier_UsesLocationHeader(t *testing.T) { + respWithLocationHeader := &http.Response{ + StatusCode: StatusCodeRetryOther, + Header: http.Header{"Location": []string{"http://example.com"}}, + } + + r := NewRequestRetrier(retry.Start(context.Background()), 2) + shouldRetry, uri := r.Next(nil, nil) + require.True(t, shouldRetry) + require.Nil(t, uri) + + shouldRetry, uri = r.Next(respWithLocationHeader, nil) + require.Equal(t, uri.String(), "http://example.com") + require.True(t, shouldRetry) +} + +func TestRequestRetrier_UsesLocationFromErr(t *testing.T) { + r := NewRequestRetrier(retry.Start(context.Background()), 2) + respErr := werror.ErrorWithContextParams(context.Background(), "307", + werror.SafeParam("statusCode", 307), + werror.SafeParam("location", "http://example-2.com")) + + shouldRetry, uri := r.Next(nil, respErr) + require.NotNil(t, uri) + require.Equal(t, uri.String(), "http://example-2.com") + require.True(t, shouldRetry) +} + +func TestRequestRetrier_Next(t *testing.T) { + for _, tc := range []struct { + name string + resp *http.Response + respErr error + retryURI *url.URL + shouldRetry bool + shouldRetryBackoff bool + shouldRetryReset bool + }{ + { + name: "returns error if response exists and doesn't appear retryable", + resp: &http.Response{}, + respErr: nil, + shouldRetry: false, + shouldRetryBackoff: false, + shouldRetryReset: false, + }, + { + name: "returns error if error code not retryable", + resp: &http.Response{}, + respErr: nil, + shouldRetry: false, + shouldRetryBackoff: false, + shouldRetryReset: false, + }, + { + name: "retries and backs off if response and error are nil", + resp: nil, + respErr: nil, + shouldRetry: true, + shouldRetryBackoff: true, + shouldRetryReset: false, + }, + { + name: "retries if unavailable", + resp: nil, + respErr: werror.ErrorWithContextParams(context.Background(), "503", werror.SafeParam("statusCode", 503)), + shouldRetry: true, + shouldRetryBackoff: false, + shouldRetryReset: false, + }, + { + name: "retries and backs off if throttled", + resp: nil, + respErr: werror.ErrorWithContextParams(context.Background(), "429", werror.SafeParam("statusCode", 429)), + shouldRetry: true, + shouldRetryBackoff: true, + shouldRetryReset: false, + }, + { + name: "retries with no backoff if gets retry other response without location", + resp: &http.Response{ + StatusCode: StatusCodeRetryOther, + }, + respErr: nil, + shouldRetry: true, + shouldRetryBackoff: false, + shouldRetryReset: false, + }, + { + name: "retries with no backoff if gets retry temporary redirect response without location", + resp: &http.Response{ + StatusCode: StatusCodeRetryTemporaryRedirect, + }, + respErr: nil, + shouldRetry: true, + shouldRetryBackoff: false, + shouldRetryReset: false, + }, + { + name: "retries with no backoff if gets retry temporary redirect response with a location", + resp: &http.Response{ + StatusCode: StatusCodeRetryTemporaryRedirect, + }, + respErr: werror.ErrorWithContextParams(context.Background(), + "307", + werror.SafeParam("statusCode", 307), + werror.SafeParam("location", "http://example-2.com")), + retryURI: mustNewURL("http://example-2.com"), + shouldRetry: true, + shouldRetryBackoff: true, + shouldRetryReset: false, + }, + { + name: "retries with no backoff if gets retry other redirect response with a location", + resp: &http.Response{ + StatusCode: StatusCodeRetryOther, + }, + respErr: werror.ErrorWithContextParams(context.Background(), + "308", + werror.SafeParam("statusCode", 308), + werror.SafeParam("location", "http://example-2.com")), + retryURI: mustNewURL("http://example-2.com"), + shouldRetry: true, + shouldRetryBackoff: true, + shouldRetryReset: false, + }, + { + name: "does not retry 400 responses", + resp: &http.Response{ + StatusCode: 400, + }, + shouldRetry: false, + shouldRetryBackoff: false, + shouldRetryReset: false, + }, + { + name: "does not retry 404 responses", + resp: &http.Response{ + StatusCode: 404, + }, + shouldRetry: false, + shouldRetryBackoff: false, + shouldRetryReset: false, + }, + { + name: "does not retry 400 errors", + respErr: werror.ErrorWithContextParams(context.Background(), "400", werror.SafeParam("statusCode", 400)), + shouldRetry: false, + shouldRetryBackoff: false, + shouldRetryReset: false, + }, + { + name: "does not retry 404s", + respErr: werror.ErrorWithContextParams(context.Background(), "404", werror.SafeParam("statusCode", 404)), + shouldRetry: false, + shouldRetryBackoff: false, + shouldRetryReset: false, + }, + } { + t.Run(tc.name, func(t *testing.T) { + retrier := newMockRetrier() + r := NewRequestRetrier(retrier, 2) + // first URI isn't a retry + shouldRetry, _ := r.Next(nil, nil) + require.True(t, shouldRetry) + + shouldRetry, retryURI := r.Next(tc.resp, tc.respErr) + assert.Equal(t, tc.shouldRetry, shouldRetry) + if tc.shouldRetry { + if tc.retryURI != nil { + require.Equal(t, tc.retryURI.String(), retryURI.String()) + } + if tc.shouldRetryReset { + require.True(t, retrier.DidReset) + } + if tc.shouldRetryBackoff { + require.True(t, retrier.DidGetNext) + } + } else { + require.Nil(t, retryURI) + } + }) + } +} + +func newMockRetrier() *mockRetrier { + return &mockRetrier{ + DidGetNext: false, + DidReset: false, + } +} + +type mockRetrier struct { + DidGetNext bool + DidReset bool +} + +func (m *mockRetrier) Reset() { + m.DidReset = true +} + +func (m *mockRetrier) Next() bool { + m.DidGetNext = true + return true +} + +func (m *mockRetrier) CurrentAttempt() int { + return 0 +} + +func mustNewURL(s string) *url.URL { + u, err := url.Parse(s) + if err != nil { + panic(err) + } + return u +} diff --git a/conjure-go-client/httpclient/internal/stateful_uri_pool.go b/conjure-go-client/httpclient/internal/stateful_uri_pool.go index 544a8140..56e49c7f 100644 --- a/conjure-go-client/httpclient/internal/stateful_uri_pool.go +++ b/conjure-go-client/httpclient/internal/stateful_uri_pool.go @@ -87,13 +87,6 @@ func (s *statefulURIPool) RoundTrip(req *http.Request, next http.RoundTripper) ( // 503: go to next node s.markBackoffURI(req, defaultResurrectDuration) } - // TODO(dtrejo): Do we need to handle redirects or does the underlying http.Client do that for us? - //if isTryOther, isPermenant, otherURI := isRetryOtherResponse(resp, respErr, errCode); isTryOther { - // s.markBackoffURI(req, defaultResurrectDuration) - //} - if errCode >= http.StatusBadRequest && errCode < http.StatusInternalServerError { - // nothing to do - } if resp == nil { // if we get a nil response, we can assume there is a problem with host and can move on to the next. s.markBackoffURI(req, defaultResurrectDuration) diff --git a/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go b/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go index 33aa28f7..db3ff72e 100644 --- a/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go +++ b/conjure-go-client/httpclient/internal/stateful_uri_pool_test.go @@ -22,32 +22,13 @@ import ( "github.com/stretchr/testify/assert" ) -//func TestRequestRetrier_UsesLocationFromErr(t *testing.T) { -// r := NewRequestRetrier([]string{"http://example-1.com"}, retry.Start(context.Background()), 2) -// respErr := werror.ErrorWithContextParams(context.Background(), "307", -// werror.SafeParam("statusCode", 307), -// werror.SafeParam("location", "http://example-2.com")) -// -// uri, isRelocated := r.GetNextURI(nil, nil) -// require.Equal(t, uri, "http://example-1.com") -// require.False(t, isRelocated) -// -// uri, isRelocated = r.GetNextURI(nil, respErr) -// require.Equal(t, uri, "http://example-2.com") -// require.True(t, isRelocated) -//} - func TestRequestRetrier_GetNextURIs(t *testing.T) { for _, tc := range []struct { - name string - resp *http.Response - chosenURI string - beforeURIs []string - afterURIs []string - shouldRetry bool - shouldRetrySameURI bool - shouldRetryBackoff bool - shouldRetryReset bool + name string + resp *http.Response + chosenURI string + beforeURIs []string + afterURIs []string }{ { name: "preserves chosen URI if response doesn't contain a handled status code", @@ -91,103 +72,36 @@ func TestRequestRetrier_GetNextURIs(t *testing.T) { beforeURIs: []string{"https://domain0.example.com"}, afterURIs: []string{"https://domain0.example.com"}, }, - //{ - // name: "retries another URI if gets retry other response without location", - // resp: &http.Response{StatusCode: StatusCodeRetryOther}, - // resp: &http.Response{ - // StatusCode: StatusCodeRetryOther, - // }, - // respErr: nil, - // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, - // shouldRetry: true, - // shouldRetrySameURI: false, - // shouldRetryBackoff: false, - // shouldRetryReset: false, - //}, - //{ - // name: "retries single URI and backs off if gets retry other response without location", - // resp: &http.Response{ - // StatusCode: StatusCodeRetryOther, - // }, - // respErr: nil, - // uris: []string{"https://domain0.example.com"}, - // shouldRetry: true, - // shouldRetrySameURI: true, - // shouldRetryBackoff: true, - // shouldRetryReset: false, - //}, - //{ - // name: "retries another URI if gets retry temporary redirect response without location", - // resp: &http.Response{ - // StatusCode: StatusCodeRetryTemporaryRedirect, - // }, - // respErr: nil, - // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, - // shouldRetry: true, - // shouldRetrySameURI: false, - // shouldRetryBackoff: false, - // shouldRetryReset: false, - //}, - //{ - // name: "retries single URI and backs off if gets retry temporary redirect response without location", - // resp: &http.Response{ - // StatusCode: StatusCodeRetryTemporaryRedirect, - // }, - // respErr: nil, - // uris: []string{"https://domain0.example.com"}, - // shouldRetry: true, - // shouldRetrySameURI: true, - // shouldRetryBackoff: true, - // shouldRetryReset: false, - //}, - //{ - // name: "does not retry 400 responses", - // resp: &http.Response{ - // StatusCode: 400, - // }, - // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, - // shouldRetry: false, - // shouldRetrySameURI: false, - // shouldRetryBackoff: false, - // shouldRetryReset: false, - //}, - //{ - // name: "does not retry 404 responses", - // resp: &http.Response{ - // StatusCode: 404, - // }, - // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, - // shouldRetry: false, - // shouldRetrySameURI: false, - // shouldRetryBackoff: false, - // shouldRetryReset: false, - //}, - //{ - // name: "does not retry 400 errors", - // respErr: werror.ErrorWithContextParams(context.Background(), "400", werror.SafeParam("statusCode", 400)), - // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, - // shouldRetry: false, - // shouldRetrySameURI: false, - // shouldRetryBackoff: false, - // shouldRetryReset: false, - //}, - //{ - // name: "does not retry 404s", - // respErr: werror.ErrorWithContextParams(context.Background(), "404", werror.SafeParam("statusCode", 404)), - // uris: []string{"https://domain0.example.com", "https://domain1.example.com"}, - // shouldRetry: false, - // shouldRetrySameURI: false, - // shouldRetryBackoff: false, - // shouldRetryReset: false, - //}, + { + name: "preserves chosen URI on permanent redirects", + resp: &http.Response{StatusCode: StatusCodeRetryOther}, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + afterURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + }, + { + name: "preserves chosen URI on temporary redirects", + resp: &http.Response{StatusCode: StatusCodeRetryOther}, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + afterURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + }, + { + name: "preserves chosen URI on 400 responses", + resp: &http.Response{StatusCode: http.StatusBadRequest}, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + afterURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + }, + { + name: "preserves chosen URI on 404 responses", + resp: &http.Response{StatusCode: http.StatusNotFound}, + chosenURI: "https://domain0.example.com", + beforeURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + afterURIs: []string{"https://domain0.example.com", "https://domain1.example.com"}, + }, } { t.Run(tc.name, func(t *testing.T) { - //retrier := newMockRetrier() - //r := NewRequestRetrier(tc.uris, retrier, 2) - //// first URI isn't a retry - //firstURI, _ := r.GetNextURI(nil, nil) - //require.NotEmpty(t, firstURI) - ref := refreshable.NewDefaultRefreshable(tc.beforeURIs) pool := NewStatefulURIPool(refreshable.NewStringSlice(ref)) @@ -199,24 +113,6 @@ func TestRequestRetrier_GetNextURIs(t *testing.T) { assert.NoError(t, err) assert.ElementsMatch(t, tc.afterURIs, pool.URIs()) - - //retryURI, _ := r.GetNextURI(tc.resp, tc.respErr) - //if tc.shouldRetry { - // require.Contains(t, tc.uris, retryURI) - // if tc.shouldRetrySameURI { - // require.Equal(t, retryURI, firstURI) - // } else { - // require.NotEqual(t, retryURI, firstURI) - // } - // if tc.shouldRetryReset { - // require.True(t, retrier.DidReset) - // } - // if tc.shouldRetryBackoff { - // require.True(t, retrier.DidGetNext) - // } - //} else { - // require.Empty(t, retryURI) - //} }) } } From cd32673a0006c9b82604b620294c247b20e46790 Mon Sep 17 00:00:00 2001 From: Devin Trejo Date: Mon, 15 Aug 2022 17:39:14 -0400 Subject: [PATCH 6/9] Properly handle all request retrier retriable requests. --- .../httpclient/internal/request_retrier.go | 14 +++++-- .../internal/request_retrier_test.go | 42 +++++++------------ 2 files changed, 25 insertions(+), 31 deletions(-) diff --git a/conjure-go-client/httpclient/internal/request_retrier.go b/conjure-go-client/httpclient/internal/request_retrier.go index 446e448a..4bce9158 100644 --- a/conjure-go-client/httpclient/internal/request_retrier.go +++ b/conjure-go-client/httpclient/internal/request_retrier.go @@ -70,7 +70,7 @@ func (r *RequestRetrier) Next(resp *http.Response, err error) (bool, *url.URL) { } // handle redirects - if tryOther, otherURI := isRetryOtherResponse(resp, err); tryOther && otherURI != nil { + if tryOther, otherURI := isRetryOtherResponse(resp, err); tryOther { return true, otherURI } @@ -79,6 +79,12 @@ func (r *RequestRetrier) Next(resp *http.Response, err error) (bool, *url.URL) { return false, nil } + // don't retry if we have a response, this isn't our first attempt and the err returned from the previous + // request is nil + if r.attemptCount != 0 && resp != nil && err == nil { + return false, nil + } + if !r.attemptsRemaining() { // Retries exhausted return false, nil @@ -95,10 +101,10 @@ func (*RequestRetrier) isSuccess(resp *http.Response) bool { } func (*RequestRetrier) isNonRetryableClientError(resp *http.Response, err error) bool { - errCode, _ := StatusCodeFromError(err) + errCode, ok := StatusCodeFromError(err) // Check for a 4XX status parsed from the error or in the response - if isClientError(errCode) && errCode != StatusCodeThrottle { - return false + if ok && isClientError(errCode) && errCode != StatusCodeThrottle { + return true } if resp != nil && isClientError(resp.StatusCode) { // 429 is retryable diff --git a/conjure-go-client/httpclient/internal/request_retrier_test.go b/conjure-go-client/httpclient/internal/request_retrier_test.go index 5c4dbd29..c2b1aa24 100644 --- a/conjure-go-client/httpclient/internal/request_retrier_test.go +++ b/conjure-go-client/httpclient/internal/request_retrier_test.go @@ -16,6 +16,7 @@ package internal import ( "context" + "errors" "net/http" "net/url" "testing" @@ -37,21 +38,19 @@ func TestRequestRetrier_HandleMeshURI(t *testing.T) { func TestRequestRetrier_AttemptCount(t *testing.T) { maxAttempts := 3 + err := errors.New("error") + r := NewRequestRetrier(retry.Start(context.Background()), maxAttempts) // first request is not a retry, so it doesn't increment the overall count - shouldRetry, _ := r.Next(nil, nil) + shouldRetry, _ := r.Next(nil, err) require.True(t, shouldRetry) for i := 0; i < maxAttempts-1; i++ { - req, err := http.NewRequest("GET", "http://example.com", nil) - require.NoError(t, err) - shouldRetry, _ = r.Next(&http.Response{Request: req}, err) + shouldRetry, _ = r.Next(nil, err) require.True(t, shouldRetry) } - req, err := http.NewRequest("GET", "http://example.com", nil) - require.NoError(t, err) - shouldRetry, _ = r.Next(&http.Response{Request: req}, err) + shouldRetry, _ = r.Next(nil, nil) require.False(t, shouldRetry) } @@ -59,6 +58,7 @@ func TestRequestRetrier_UnlimitedAttempts(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second) defer cancel() + err := errors.New("error") r := NewRequestRetrier(retry.Start(ctx, retry.WithInitialBackoff(50*time.Millisecond), retry.WithRandomizationFactor(0)), 0) startTime := time.Now() @@ -66,24 +66,20 @@ func TestRequestRetrier_UnlimitedAttempts(t *testing.T) { require.True(t, shouldRetry) require.Lessf(t, time.Since(startTime), 49*time.Millisecond, "first GetNextURI should not have any delay") - req, err := http.NewRequest("GET", "http://example.com", nil) - require.NoError(t, err) - resp := &http.Response{Request: req} - startTime = time.Now() - shouldRetry, _ = r.Next(resp, err) + shouldRetry, _ = r.Next(nil, err) require.True(t, shouldRetry) assert.Greater(t, time.Since(startTime), 50*time.Millisecond, "delay should be at least 1 backoff") assert.Less(t, time.Since(startTime), 100*time.Millisecond, "delay should be less than 2 backoffs") startTime = time.Now() - shouldRetry, _ = r.Next(resp, err) + shouldRetry, _ = r.Next(nil, err) require.True(t, shouldRetry) assert.Greater(t, time.Since(startTime), 100*time.Millisecond, "delay should be at least 2 backoffs") assert.Less(t, time.Since(startTime), 200*time.Millisecond, "delay should be less than 3 backoffs") // Success should stop retries - shouldRetry, _ = r.Next(&http.Response{Request: req, StatusCode: http.StatusOK}, nil) + shouldRetry, _ = r.Next(&http.Response{StatusCode: http.StatusOK}, nil) require.False(t, shouldRetry) } @@ -137,29 +133,21 @@ func TestRequestRetrier_Next(t *testing.T) { shouldRetryReset bool }{ { - name: "returns error if response exists and doesn't appear retryable", - resp: &http.Response{}, + name: "retries and backs off if response and error are nil", + resp: nil, respErr: nil, - shouldRetry: false, - shouldRetryBackoff: false, + shouldRetry: true, + shouldRetryBackoff: true, shouldRetryReset: false, }, { - name: "returns error if error code not retryable", + name: "no retries if response exists and doesn't appear retryable", resp: &http.Response{}, respErr: nil, shouldRetry: false, shouldRetryBackoff: false, shouldRetryReset: false, }, - { - name: "retries and backs off if response and error are nil", - resp: nil, - respErr: nil, - shouldRetry: true, - shouldRetryBackoff: true, - shouldRetryReset: false, - }, { name: "retries if unavailable", resp: nil, From dc1bf3bbea0bd8d5478cdd916cb9cf314e14cbcb Mon Sep 17 00:00:00 2001 From: Devin Trejo Date: Mon, 15 Aug 2022 17:53:04 -0400 Subject: [PATCH 7/9] Don't handle nil err responses in request retrier as it's an impossible scenario to hit. --- .../httpclient/internal/balanced_selector.go | 2 +- .../httpclient/internal/request_retrier.go | 11 +++-------- .../httpclient/internal/request_retrier_test.go | 16 ---------------- .../httpclient/internal/stateful_uri_pool.go | 6 +++--- 4 files changed, 7 insertions(+), 28 deletions(-) diff --git a/conjure-go-client/httpclient/internal/balanced_selector.go b/conjure-go-client/httpclient/internal/balanced_selector.go index 1d500e71..bc7205c1 100644 --- a/conjure-go-client/httpclient/internal/balanced_selector.go +++ b/conjure-go-client/httpclient/internal/balanced_selector.go @@ -113,7 +113,7 @@ func (s *balancedSelector) RoundTrip(req *http.Request, next http.RoundTripper) } else if isClientError(statusCode) { s.updateRecentFailures(baseURI, failureWeight/100) } - return resp, nil + return resp, err } func (s *balancedSelector) updateInflight(uri string, score int32) { diff --git a/conjure-go-client/httpclient/internal/request_retrier.go b/conjure-go-client/httpclient/internal/request_retrier.go index 4bce9158..114c4134 100644 --- a/conjure-go-client/httpclient/internal/request_retrier.go +++ b/conjure-go-client/httpclient/internal/request_retrier.go @@ -79,16 +79,12 @@ func (r *RequestRetrier) Next(resp *http.Response, err error) (bool, *url.URL) { return false, nil } - // don't retry if we have a response, this isn't our first attempt and the err returned from the previous - // request is nil - if r.attemptCount != 0 && resp != nil && err == nil { - return false, nil - } - if !r.attemptsRemaining() { // Retries exhausted return false, nil } + + // retry with backoff return r.retrier.Next(), nil } @@ -109,9 +105,8 @@ func (*RequestRetrier) isNonRetryableClientError(resp *http.Response, err error) if resp != nil && isClientError(resp.StatusCode) { // 429 is retryable if isThrottle, _ := isThrottleResponse(resp, errCode); !isThrottle { - return false + return true } - return true } return false } diff --git a/conjure-go-client/httpclient/internal/request_retrier_test.go b/conjure-go-client/httpclient/internal/request_retrier_test.go index c2b1aa24..a1542335 100644 --- a/conjure-go-client/httpclient/internal/request_retrier_test.go +++ b/conjure-go-client/httpclient/internal/request_retrier_test.go @@ -132,22 +132,6 @@ func TestRequestRetrier_Next(t *testing.T) { shouldRetryBackoff bool shouldRetryReset bool }{ - { - name: "retries and backs off if response and error are nil", - resp: nil, - respErr: nil, - shouldRetry: true, - shouldRetryBackoff: true, - shouldRetryReset: false, - }, - { - name: "no retries if response exists and doesn't appear retryable", - resp: &http.Response{}, - respErr: nil, - shouldRetry: false, - shouldRetryBackoff: false, - shouldRetryReset: false, - }, { name: "retries if unavailable", resp: nil, diff --git a/conjure-go-client/httpclient/internal/stateful_uri_pool.go b/conjure-go-client/httpclient/internal/stateful_uri_pool.go index 56e49c7f..5aed7e6a 100644 --- a/conjure-go-client/httpclient/internal/stateful_uri_pool.go +++ b/conjure-go-client/httpclient/internal/stateful_uri_pool.go @@ -77,8 +77,8 @@ func (s *statefulURIPool) URIs() []string { // RoundTrip implements URIPool func (s *statefulURIPool) RoundTrip(req *http.Request, next http.RoundTripper) (*http.Response, error) { - resp, respErr := next.RoundTrip(req) - errCode, _ := StatusCodeFromError(respErr) + resp, err := next.RoundTrip(req) + errCode, _ := StatusCodeFromError(err) if isThrottle, ressurectAfter := isThrottleResponse(resp, errCode); isThrottle { s.markBackoffURI(req, ressurectAfter) @@ -92,7 +92,7 @@ func (s *statefulURIPool) RoundTrip(req *http.Request, next http.RoundTripper) ( s.markBackoffURI(req, defaultResurrectDuration) } - return resp, respErr + return resp, err } func (s *statefulURIPool) updateURIs(uris []string) { From 5e2d60f261229b84178e305120ad85aab03a8066 Mon Sep 17 00:00:00 2001 From: Devin Trejo Date: Mon, 15 Aug 2022 18:33:31 -0400 Subject: [PATCH 8/9] Always try attempt on first try in order to get relevant context cancellation error. Cleanup err handling in balanced selector. Fallback to response status code if cgr error isn't defined. --- conjure-go-client/httpclient/client.go | 5 ++++- .../httpclient/internal/balanced_selector.go | 13 ++++++----- .../httpclient/internal/request_retrier.go | 18 ++++++++++----- .../internal/request_retrier_test.go | 22 ++++++++++++++----- .../httpclient/internal/stateful_uri_pool.go | 14 +++++++----- 5 files changed, 49 insertions(+), 23 deletions(-) diff --git a/conjure-go-client/httpclient/client.go b/conjure-go-client/httpclient/client.go index 9d439fcb..7f864106 100644 --- a/conjure-go-client/httpclient/client.go +++ b/conjure-go-client/httpclient/client.go @@ -107,7 +107,10 @@ func (c *clientImpl) Do(ctx context.Context, params ...RequestParam) (*http.Resp } svc1log.FromContext(ctx).Debug("Retrying request", svc1log.Stacktrace(err)) } - return resp, err + if err != nil { + return nil, err + } + return resp, nil } func (c *clientImpl) doOnce( diff --git a/conjure-go-client/httpclient/internal/balanced_selector.go b/conjure-go-client/httpclient/internal/balanced_selector.go index bc7205c1..e25e4c2c 100644 --- a/conjure-go-client/httpclient/internal/balanced_selector.go +++ b/conjure-go-client/httpclient/internal/balanced_selector.go @@ -103,14 +103,15 @@ func (s *balancedSelector) RoundTrip(req *http.Request, next http.RoundTripper) defer s.updateInflight(baseURI, -1) resp, err := next.RoundTrip(req) - if resp == nil || err != nil { - s.updateRecentFailures(baseURI, failureWeight) - return nil, err + errCode, ok := StatusCodeFromError(err) + // fall back to the status code from the response + if !ok && resp != nil { + errCode = resp.StatusCode } - statusCode := resp.StatusCode - if isGlobalQosStatus(statusCode) || isServerErrorRange(statusCode) { + + if isGlobalQosStatus(errCode) || isServerErrorRange(errCode) { s.updateRecentFailures(baseURI, failureWeight) - } else if isClientError(statusCode) { + } else if isClientError(errCode) { s.updateRecentFailures(baseURI, failureWeight/100) } return resp, err diff --git a/conjure-go-client/httpclient/internal/request_retrier.go b/conjure-go-client/httpclient/internal/request_retrier.go index 114c4134..e9ac970e 100644 --- a/conjure-go-client/httpclient/internal/request_retrier.go +++ b/conjure-go-client/httpclient/internal/request_retrier.go @@ -61,6 +61,19 @@ func (r *RequestRetrier) attemptsRemaining() bool { // retrier will also return the URL that should be used for the new next request. func (r *RequestRetrier) Next(resp *http.Response, err error) (bool, *url.URL) { defer func() { r.attemptCount++ }() + // should always try first request + if r.attemptCount == 0 { + // Trigger the first retry so later calls have backoff but ignore the returned value to ensure that the + // client can instrument the request even if the context is done. + r.retrier.Next() + return true, nil + } + + if !r.attemptsRemaining() { + // Retries exhausted + return false, nil + } + if r.isSuccess(resp) { return false, nil } @@ -79,11 +92,6 @@ func (r *RequestRetrier) Next(resp *http.Response, err error) (bool, *url.URL) { return false, nil } - if !r.attemptsRemaining() { - // Retries exhausted - return false, nil - } - // retry with backoff return r.retrier.Next(), nil } diff --git a/conjure-go-client/httpclient/internal/request_retrier_test.go b/conjure-go-client/httpclient/internal/request_retrier_test.go index a1542335..d8490533 100644 --- a/conjure-go-client/httpclient/internal/request_retrier_test.go +++ b/conjure-go-client/httpclient/internal/request_retrier_test.go @@ -30,9 +30,11 @@ import ( func TestRequestRetrier_HandleMeshURI(t *testing.T) { r := NewRequestRetrier(retry.Start(context.Background()), 1) - req, err := http.NewRequest("GET", "mesh-http://example.com", nil) - require.NoError(t, err) - shouldRetry, _ := r.Next(&http.Response{Request: req}, nil) + shouldRetry, _ := r.Next(&http.Response{}, nil) + require.True(t, shouldRetry) + + respErr := werror.ErrorWithContextParams(context.Background(), "error", werror.SafeParam("statusCode", 429)) + shouldRetry, _ = r.Next(&http.Response{}, respErr) require.False(t, shouldRetry) } @@ -89,8 +91,13 @@ func TestRequestRetrier_ContextCanceled(t *testing.T) { r := NewRequestRetrier(retry.Start(ctx), 0) - // No retries if context is cancelled + // First attempt should return a URI to ensure that the client can instrument the request even + // if the context is done shouldRetry, _ := r.Next(nil, nil) + require.True(t, shouldRetry) + + // Subsequent attempt should stop retries + shouldRetry, _ = r.Next(nil, nil) require.False(t, shouldRetry) } @@ -112,11 +119,16 @@ func TestRequestRetrier_UsesLocationHeader(t *testing.T) { func TestRequestRetrier_UsesLocationFromErr(t *testing.T) { r := NewRequestRetrier(retry.Start(context.Background()), 2) + + shouldRetry, uri := r.Next(nil, nil) + require.True(t, shouldRetry) + require.Nil(t, uri) + respErr := werror.ErrorWithContextParams(context.Background(), "307", werror.SafeParam("statusCode", 307), werror.SafeParam("location", "http://example-2.com")) - shouldRetry, uri := r.Next(nil, respErr) + shouldRetry, uri = r.Next(nil, respErr) require.NotNil(t, uri) require.Equal(t, uri.String(), "http://example-2.com") require.True(t, shouldRetry) diff --git a/conjure-go-client/httpclient/internal/stateful_uri_pool.go b/conjure-go-client/httpclient/internal/stateful_uri_pool.go index 5aed7e6a..ff19ca30 100644 --- a/conjure-go-client/httpclient/internal/stateful_uri_pool.go +++ b/conjure-go-client/httpclient/internal/stateful_uri_pool.go @@ -78,16 +78,18 @@ func (s *statefulURIPool) URIs() []string { // RoundTrip implements URIPool func (s *statefulURIPool) RoundTrip(req *http.Request, next http.RoundTripper) (*http.Response, error) { resp, err := next.RoundTrip(req) - errCode, _ := StatusCodeFromError(err) + errCode, ok := StatusCodeFromError(err) + // fall back to the status code from the response + if !ok && resp != nil { + errCode = resp.StatusCode + } if isThrottle, ressurectAfter := isThrottleResponse(resp, errCode); isThrottle { s.markBackoffURI(req, ressurectAfter) - } - if isUnavailableResponse(resp, errCode) { + } else if isUnavailableResponse(resp, errCode) { // 503: go to next node s.markBackoffURI(req, defaultResurrectDuration) - } - if resp == nil { + } else if resp == nil { // if we get a nil response, we can assume there is a problem with host and can move on to the next. s.markBackoffURI(req, defaultResurrectDuration) } @@ -116,7 +118,7 @@ func (s *statefulURIPool) updateURIs(uris []string) { func (s *statefulURIPool) markBackoffURI(req *http.Request, dur time.Duration) { // if duration is equal to zero, then don't mark the URI as failed if dur == 0 { - dur = defaultResurrectDuration + return } reqURL := getBaseURI(req.URL) s.Lock() From cdfd790b61f7ede35a4bd275580213dbde97a930 Mon Sep 17 00:00:00 2001 From: Devin Trejo Date: Mon, 15 Aug 2022 19:51:02 -0400 Subject: [PATCH 9/9] Add uriselector/pool middleware in client builder. Use defaultResurrectDuration if one isn't defined. --- conjure-go-client/httpclient/client.go | 9 +++---- .../httpclient/client_builder.go | 2 ++ .../internal/request_retrier_test.go | 27 +++++++++---------- .../httpclient/internal/stateful_uri_pool.go | 4 +-- 4 files changed, 20 insertions(+), 22 deletions(-) diff --git a/conjure-go-client/httpclient/client.go b/conjure-go-client/httpclient/client.go index 7f864106..399a65dc 100644 --- a/conjure-go-client/httpclient/client.go +++ b/conjure-go-client/httpclient/client.go @@ -102,10 +102,11 @@ func (c *clientImpl) Do(ctx context.Context, params ...RequestParam) (*http.Resp break } resp, err = c.doOnce(ctx, retryURL, params...) - if err == nil { - break + if err != nil { + svc1log.FromContext(ctx).Debug("Retrying request", svc1log.Stacktrace(err)) + continue } - svc1log.FromContext(ctx).Debug("Retrying request", svc1log.Stacktrace(err)) + break } if err != nil { return nil, err @@ -168,8 +169,6 @@ func (c *clientImpl) doOnce( transport := clientCopy.Transport // start with the client's transport configured with default middleware // must precede the error decoders to read the status code of the raw response. - transport = wrapTransport(transport, c.uriSelector) - transport = wrapTransport(transport, c.uriPool) // request decoder must precede the client decoder // must precede the body middleware to read the response body transport = wrapTransport(transport, b.errorDecoderMiddleware, c.errorDecoderMiddleware) diff --git a/conjure-go-client/httpclient/client_builder.go b/conjure-go-client/httpclient/client_builder.go index f371d341..d986d45e 100644 --- a/conjure-go-client/httpclient/client_builder.go +++ b/conjure-go-client/httpclient/client_builder.go @@ -157,6 +157,8 @@ func newClient(ctx context.Context, b *clientBuilder, params ...ClientParam) (Cl if b.URISelector == nil { b.URISelector = internal.NewRoundRobinURISelector(func() int64 { return time.Now().UnixNano() }) } + // append uriSelector and uriPool middlewares + middleware = append(middleware, uriPool, b.URISelector) return &clientImpl{ client: httpClient, uriPool: uriPool, diff --git a/conjure-go-client/httpclient/internal/request_retrier_test.go b/conjure-go-client/httpclient/internal/request_retrier_test.go index d8490533..ceffad8a 100644 --- a/conjure-go-client/httpclient/internal/request_retrier_test.go +++ b/conjure-go-client/httpclient/internal/request_retrier_test.go @@ -30,28 +30,25 @@ import ( func TestRequestRetrier_HandleMeshURI(t *testing.T) { r := NewRequestRetrier(retry.Start(context.Background()), 1) - shouldRetry, _ := r.Next(&http.Response{}, nil) + shouldRetry, _ := r.Next(nil, nil) require.True(t, shouldRetry) respErr := werror.ErrorWithContextParams(context.Background(), "error", werror.SafeParam("statusCode", 429)) - shouldRetry, _ = r.Next(&http.Response{}, respErr) + shouldRetry, _ = r.Next(nil, respErr) require.False(t, shouldRetry) } func TestRequestRetrier_AttemptCount(t *testing.T) { maxAttempts := 3 - err := errors.New("error") - r := NewRequestRetrier(retry.Start(context.Background()), maxAttempts) - // first request is not a retry, so it doesn't increment the overall count - shouldRetry, _ := r.Next(nil, err) + // first request is not a retry + shouldRetry, _ := r.Next(nil, nil) require.True(t, shouldRetry) for i := 0; i < maxAttempts-1; i++ { - shouldRetry, _ = r.Next(nil, err) + shouldRetry, _ = r.Next(nil, errors.New("error")) require.True(t, shouldRetry) } - shouldRetry, _ = r.Next(nil, nil) require.False(t, shouldRetry) } @@ -60,7 +57,6 @@ func TestRequestRetrier_UnlimitedAttempts(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second) defer cancel() - err := errors.New("error") r := NewRequestRetrier(retry.Start(ctx, retry.WithInitialBackoff(50*time.Millisecond), retry.WithRandomizationFactor(0)), 0) startTime := time.Now() @@ -69,13 +65,13 @@ func TestRequestRetrier_UnlimitedAttempts(t *testing.T) { require.Lessf(t, time.Since(startTime), 49*time.Millisecond, "first GetNextURI should not have any delay") startTime = time.Now() - shouldRetry, _ = r.Next(nil, err) + shouldRetry, _ = r.Next(nil, errors.New("error")) require.True(t, shouldRetry) assert.Greater(t, time.Since(startTime), 50*time.Millisecond, "delay should be at least 1 backoff") assert.Less(t, time.Since(startTime), 100*time.Millisecond, "delay should be less than 2 backoffs") startTime = time.Now() - shouldRetry, _ = r.Next(nil, err) + shouldRetry, _ = r.Next(nil, errors.New("error")) require.True(t, shouldRetry) assert.Greater(t, time.Since(startTime), 100*time.Millisecond, "delay should be at least 2 backoffs") assert.Less(t, time.Since(startTime), 200*time.Millisecond, "delay should be less than 3 backoffs") @@ -120,14 +116,15 @@ func TestRequestRetrier_UsesLocationHeader(t *testing.T) { func TestRequestRetrier_UsesLocationFromErr(t *testing.T) { r := NewRequestRetrier(retry.Start(context.Background()), 2) - shouldRetry, uri := r.Next(nil, nil) - require.True(t, shouldRetry) - require.Nil(t, uri) - respErr := werror.ErrorWithContextParams(context.Background(), "307", werror.SafeParam("statusCode", 307), werror.SafeParam("location", "http://example-2.com")) + // first request is not a retry + shouldRetry, uri := r.Next(nil, nil) + require.True(t, shouldRetry) + require.Nil(t, uri) + shouldRetry, uri = r.Next(nil, respErr) require.NotNil(t, uri) require.Equal(t, uri.String(), "http://example-2.com") diff --git a/conjure-go-client/httpclient/internal/stateful_uri_pool.go b/conjure-go-client/httpclient/internal/stateful_uri_pool.go index ff19ca30..6f4fb723 100644 --- a/conjure-go-client/httpclient/internal/stateful_uri_pool.go +++ b/conjure-go-client/httpclient/internal/stateful_uri_pool.go @@ -116,9 +116,9 @@ func (s *statefulURIPool) updateURIs(uris []string) { } func (s *statefulURIPool) markBackoffURI(req *http.Request, dur time.Duration) { - // if duration is equal to zero, then don't mark the URI as failed + // if duration is equal to zero, then use defaultResurrectDuration if dur == 0 { - return + dur = defaultResurrectDuration } reqURL := getBaseURI(req.URL) s.Lock()