Skip to content

Commit

Permalink
Properly handle redirects in the request retrier. Fix expected behavi…
Browse files Browse the repository at this point in the history
…or of client having round-robin uri selector.
  • Loading branch information
dtrejod committed Aug 11, 2022
1 parent 7e3a617 commit 96b410a
Show file tree
Hide file tree
Showing 12 changed files with 203 additions and 66 deletions.
49 changes: 30 additions & 19 deletions conjure-go-client/httpclient/client.go
Original file line number Diff line number Diff line change
Expand Up @@ -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),
Expand All @@ -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
}
}

Expand All @@ -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
Expand Down Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion conjure-go-client/httpclient/client_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
2 changes: 1 addition & 1 deletion conjure-go-client/httpclient/internal/balanced_selector.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
74 changes: 49 additions & 25 deletions conjure-go-client/httpclient/internal/request_retrier.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package internal

import (
"net/http"
"net/url"
"strings"

"github.com/palantir/pkg/retry"
Expand Down Expand Up @@ -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)
}
20 changes: 10 additions & 10 deletions conjure-go-client/httpclient/internal/request_retrier_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -31,27 +31,27 @@ 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)
}

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)
}

Expand All @@ -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)
}

Expand All @@ -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)
}

Expand Down
16 changes: 11 additions & 5 deletions conjure-go-client/httpclient/internal/retry.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand All @@ -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 {
Expand Down
4 changes: 2 additions & 2 deletions conjure-go-client/httpclient/internal/retry_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
49 changes: 49 additions & 0 deletions conjure-go-client/httpclient/internal/rr_selector.go
Original file line number Diff line number Diff line change
@@ -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)
}
Loading

0 comments on commit 96b410a

Please sign in to comment.