Use gRPC ClientConn for otlpmetricgrpc client connection handling (#2425)
* Use gRPC ClientConn for otlpmetricgrpc conn handling * Update PR number * Update otlpmetrichttp to use retry package * Remove old commented code * Add all other external changes to changelog
This commit is contained in:
		
							parent
							
								
									a1f9c9772b
								
							
						
					
					
						commit
						f4b63f184d
					
				|  | @ -13,11 +13,19 @@ This project adheres to [Semantic Versioning](https://semver.org/spec/v2.0.0.htm | ||||||
| - Added an internal Logger. | - Added an internal Logger. | ||||||
|   This can be used by the SDK and API to provide users with feedback of the internal state. |   This can be used by the SDK and API to provide users with feedback of the internal state. | ||||||
|   To enable verbose logs configure the logger which will print V(1) logs. For debugging information configure to print V(5) logs. (#2343) |   To enable verbose logs configure the logger which will print V(1) logs. For debugging information configure to print V(5) logs. (#2343) | ||||||
|  | - Add the `WithRetry` `Option` and the `RetryConfig` type to the `go.opentelemetry.io/otel/exporter/otel/otlpmetric/otlpmetrichttp` package to specify retry behavior consistently. (#2425) | ||||||
| 
 | 
 | ||||||
| ### Changed | ### Changed | ||||||
| 
 | 
 | ||||||
| - The `"go.opentelemetry.io/otel/exporter/otel/otlptrace/otlptracegrpc".Client` now uses the underlying gRPC `ClientConn` to handle name resolution, TCP connection establishment (with retries and backoff) and TLS handshakes, and handling errors on established connections by re-resolving the name and reconnecting. (#2329) | - The `"go.opentelemetry.io/otel/exporter/otel/otlptrace/otlptracegrpc".Client` now uses the underlying gRPC `ClientConn` to handle name resolution, TCP connection establishment (with retries and backoff) and TLS handshakes, and handling errors on established connections by re-resolving the name and reconnecting. (#2329) | ||||||
| - Changed the project minimum supported Go version from 1.15 to 1.16. (#2412) | - Changed the project minimum supported Go version from 1.15 to 1.16. (#2412) | ||||||
|  | - The `"go.opentelemetry.io/otel/exporter/otel/otlpmetric/otlpmetricgrpc".Client` now uses the underlying gRPC `ClientConn` to handle name resolution, TCP connection establishment (with retries and backoff) and TLS handshakes, and handling errors on established connections by re-resolving the name and reconnecting. (#2425) | ||||||
|  | - The `"go.opentelemetry.io/otel/exporter/otel/otlpmetric/otlpmetricgrpc".RetrySettings` type is renamed to `RetryConfig`. (#2425) | ||||||
|  | 
 | ||||||
|  | ### Deprecated | ||||||
|  | 
 | ||||||
|  | - Deprecated the `"go.opentelemetry.io/otel/exporter/otel/otlpmetric/otlpmetrichttp".WithMaxAttempts` `Option`, use the new `WithRetry` `Option` instead. (#2425) | ||||||
|  | - Deprecated the `"go.opentelemetry.io/otel/exporter/otel/otlpmetric/otlpmetrichttp".WithBackoff` `Option`, use the new `WithRetry` `Option` instead. (#2425) | ||||||
| 
 | 
 | ||||||
| ### Removed | ### Removed | ||||||
| 
 | 
 | ||||||
|  |  | ||||||
|  | @ -12,7 +12,6 @@ require ( | ||||||
| 	go.opentelemetry.io/otel/sdk/export/metric v0.25.0 | 	go.opentelemetry.io/otel/sdk/export/metric v0.25.0 | ||||||
| 	go.opentelemetry.io/otel/sdk/metric v0.25.0 | 	go.opentelemetry.io/otel/sdk/metric v0.25.0 | ||||||
| 	go.opentelemetry.io/proto/otlp v0.11.0 | 	go.opentelemetry.io/proto/otlp v0.11.0 | ||||||
| 	google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013 |  | ||||||
| 	google.golang.org/grpc v1.42.0 | 	google.golang.org/grpc v1.42.0 | ||||||
| 	google.golang.org/protobuf v1.27.1 | 	google.golang.org/protobuf v1.27.1 | ||||||
| ) | ) | ||||||
|  |  | ||||||
|  | @ -1,38 +0,0 @@ | ||||||
| // Copyright The OpenTelemetry Authors
 |  | ||||||
| //
 |  | ||||||
| // Licensed under the Apache License, Version 2.0 (the "License");
 |  | ||||||
| // you may not use this file except in compliance with the License.
 |  | ||||||
| // You may obtain a copy of the License at
 |  | ||||||
| //
 |  | ||||||
| //     http://www.apache.org/licenses/LICENSE-2.0
 |  | ||||||
| //
 |  | ||||||
| // Unless required by applicable law or agreed to in writing, software
 |  | ||||||
| // distributed under the License is distributed on an "AS IS" BASIS,
 |  | ||||||
| // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 |  | ||||||
| // See the License for the specific language governing permissions and
 |  | ||||||
| // limitations under the License.
 |  | ||||||
| 
 |  | ||||||
| package connection |  | ||||||
| 
 |  | ||||||
| import ( |  | ||||||
| 	"os" |  | ||||||
| 	"testing" |  | ||||||
| 	"unsafe" |  | ||||||
| 
 |  | ||||||
| 	ottest "go.opentelemetry.io/otel/internal/internaltest" |  | ||||||
| ) |  | ||||||
| 
 |  | ||||||
| // Ensure struct alignment prior to running tests.
 |  | ||||||
| func TestMain(m *testing.M) { |  | ||||||
| 	fields := []ottest.FieldOffset{ |  | ||||||
| 		{ |  | ||||||
| 			Name:   "Connection.lastConnectErrPtr", |  | ||||||
| 			Offset: unsafe.Offsetof(Connection{}.lastConnectErrPtr), |  | ||||||
| 		}, |  | ||||||
| 	} |  | ||||||
| 	if !ottest.Aligned8Byte(fields, os.Stderr) { |  | ||||||
| 		os.Exit(1) |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	os.Exit(m.Run()) |  | ||||||
| } |  | ||||||
|  | @ -1,431 +0,0 @@ | ||||||
| // Copyright The OpenTelemetry Authors
 |  | ||||||
| //
 |  | ||||||
| // Licensed under the Apache License, Version 2.0 (the "License");
 |  | ||||||
| // you may not use this file except in compliance with the License.
 |  | ||||||
| // You may obtain a copy of the License at
 |  | ||||||
| //
 |  | ||||||
| //     http://www.apache.org/licenses/LICENSE-2.0
 |  | ||||||
| //
 |  | ||||||
| // Unless required by applicable law or agreed to in writing, software
 |  | ||||||
| // distributed under the License is distributed on an "AS IS" BASIS,
 |  | ||||||
| // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 |  | ||||||
| // See the License for the specific language governing permissions and
 |  | ||||||
| // limitations under the License.
 |  | ||||||
| 
 |  | ||||||
| package connection // import "go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/connection"
 |  | ||||||
| 
 |  | ||||||
| import ( |  | ||||||
| 	"context" |  | ||||||
| 	"fmt" |  | ||||||
| 	"math/rand" |  | ||||||
| 	"sync" |  | ||||||
| 	"sync/atomic" |  | ||||||
| 	"time" |  | ||||||
| 	"unsafe" |  | ||||||
| 
 |  | ||||||
| 	"github.com/cenkalti/backoff/v4" |  | ||||||
| 	"google.golang.org/genproto/googleapis/rpc/errdetails" |  | ||||||
| 	"google.golang.org/grpc" |  | ||||||
| 	"google.golang.org/grpc/codes" |  | ||||||
| 	"google.golang.org/grpc/encoding/gzip" |  | ||||||
| 	"google.golang.org/grpc/metadata" |  | ||||||
| 	"google.golang.org/grpc/status" |  | ||||||
| 
 |  | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpconfig" |  | ||||||
| ) |  | ||||||
| 
 |  | ||||||
| type Connection struct { |  | ||||||
| 	// Ensure pointer is 64-bit aligned for atomic operations on both 32 and 64 bit machines.
 |  | ||||||
| 	lastConnectErrPtr unsafe.Pointer |  | ||||||
| 
 |  | ||||||
| 	// mu protects the Connection as it is accessed by the
 |  | ||||||
| 	// exporter goroutines and background Connection goroutine
 |  | ||||||
| 	mu sync.Mutex |  | ||||||
| 	cc *grpc.ClientConn |  | ||||||
| 
 |  | ||||||
| 	// these fields are read-only after constructor is finished
 |  | ||||||
| 	cfg                  otlpconfig.Config |  | ||||||
| 	SCfg                 otlpconfig.SignalConfig |  | ||||||
| 	metadata             metadata.MD |  | ||||||
| 	newConnectionHandler func(cc *grpc.ClientConn) |  | ||||||
| 
 |  | ||||||
| 	// these channels are created once
 |  | ||||||
| 	disconnectedCh             chan bool |  | ||||||
| 	backgroundConnectionDoneCh chan struct{} |  | ||||||
| 	stopCh                     chan struct{} |  | ||||||
| 
 |  | ||||||
| 	// this is for tests, so they can replace the closing
 |  | ||||||
| 	// routine without a worry of modifying some global variable
 |  | ||||||
| 	// or changing it back to original after the test is done
 |  | ||||||
| 	closeBackgroundConnectionDoneCh func(ch chan struct{}) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func NewConnection(cfg otlpconfig.Config, sCfg otlpconfig.SignalConfig, handler func(cc *grpc.ClientConn)) *Connection { |  | ||||||
| 	c := new(Connection) |  | ||||||
| 	c.newConnectionHandler = handler |  | ||||||
| 	c.cfg = cfg |  | ||||||
| 	c.SCfg = sCfg |  | ||||||
| 	if len(c.SCfg.Headers) > 0 { |  | ||||||
| 		c.metadata = metadata.New(c.SCfg.Headers) |  | ||||||
| 	} |  | ||||||
| 	c.closeBackgroundConnectionDoneCh = func(ch chan struct{}) { |  | ||||||
| 		close(ch) |  | ||||||
| 	} |  | ||||||
| 	return c |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) StartConnection(ctx context.Context) error { |  | ||||||
| 	c.stopCh = make(chan struct{}) |  | ||||||
| 	c.disconnectedCh = make(chan bool, 1) |  | ||||||
| 	c.backgroundConnectionDoneCh = make(chan struct{}) |  | ||||||
| 
 |  | ||||||
| 	if err := c.connect(ctx); err == nil { |  | ||||||
| 		c.setStateConnected() |  | ||||||
| 	} else { |  | ||||||
| 		c.SetStateDisconnected(err) |  | ||||||
| 	} |  | ||||||
| 	go c.indefiniteBackgroundConnection() |  | ||||||
| 
 |  | ||||||
| 	// TODO: proper error handling when initializing connections.
 |  | ||||||
| 	// We can report permanent errors, e.g., invalid settings.
 |  | ||||||
| 	return nil |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) LastConnectError() error { |  | ||||||
| 	errPtr := (*error)(atomic.LoadPointer(&c.lastConnectErrPtr)) |  | ||||||
| 	if errPtr == nil { |  | ||||||
| 		return nil |  | ||||||
| 	} |  | ||||||
| 	return *errPtr |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) saveLastConnectError(err error) { |  | ||||||
| 	var errPtr *error |  | ||||||
| 	if err != nil { |  | ||||||
| 		errPtr = &err |  | ||||||
| 	} |  | ||||||
| 	atomic.StorePointer(&c.lastConnectErrPtr, unsafe.Pointer(errPtr)) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) SetStateDisconnected(err error) { |  | ||||||
| 	c.saveLastConnectError(err) |  | ||||||
| 	select { |  | ||||||
| 	case c.disconnectedCh <- true: |  | ||||||
| 	default: |  | ||||||
| 	} |  | ||||||
| 	c.newConnectionHandler(nil) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) setStateConnected() { |  | ||||||
| 	c.saveLastConnectError(nil) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) Connected() bool { |  | ||||||
| 	return c.LastConnectError() == nil |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| const defaultConnReattemptPeriod = 10 * time.Second |  | ||||||
| 
 |  | ||||||
| func (c *Connection) indefiniteBackgroundConnection() { |  | ||||||
| 	defer func() { |  | ||||||
| 		c.closeBackgroundConnectionDoneCh(c.backgroundConnectionDoneCh) |  | ||||||
| 	}() |  | ||||||
| 
 |  | ||||||
| 	connReattemptPeriod := c.cfg.ReconnectionPeriod |  | ||||||
| 	if connReattemptPeriod <= 0 { |  | ||||||
| 		connReattemptPeriod = defaultConnReattemptPeriod |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	// No strong seeding required, nano time can
 |  | ||||||
| 	// already help with pseudo uniqueness.
 |  | ||||||
| 	rng := rand.New(rand.NewSource(time.Now().UnixNano() + rand.Int63n(1024))) |  | ||||||
| 
 |  | ||||||
| 	// maxJitterNanos: 70% of the connectionReattemptPeriod
 |  | ||||||
| 	maxJitterNanos := int64(0.7 * float64(connReattemptPeriod)) |  | ||||||
| 
 |  | ||||||
| 	for { |  | ||||||
| 		// Otherwise these will be the normal scenarios to enable
 |  | ||||||
| 		// reconnection if we trip out.
 |  | ||||||
| 		// 1. If we've stopped, return entirely
 |  | ||||||
| 		// 2. Otherwise block until we are disconnected, and
 |  | ||||||
| 		//    then retry connecting
 |  | ||||||
| 		select { |  | ||||||
| 		case <-c.stopCh: |  | ||||||
| 			return |  | ||||||
| 
 |  | ||||||
| 		case <-c.disconnectedCh: |  | ||||||
| 			// Quickly check if we haven't stopped at the
 |  | ||||||
| 			// same time.
 |  | ||||||
| 			select { |  | ||||||
| 			case <-c.stopCh: |  | ||||||
| 				return |  | ||||||
| 
 |  | ||||||
| 			default: |  | ||||||
| 			} |  | ||||||
| 
 |  | ||||||
| 			// Normal scenario that we'll wait for
 |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		if err := c.connect(context.Background()); err == nil { |  | ||||||
| 			c.setStateConnected() |  | ||||||
| 		} else { |  | ||||||
| 			// this code is unreachable in most cases
 |  | ||||||
| 			// c.connect does not establish Connection
 |  | ||||||
| 			c.SetStateDisconnected(err) |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		// Apply some jitter to avoid lockstep retrials of other
 |  | ||||||
| 		// collector-exporters. Lockstep retrials could result in an
 |  | ||||||
| 		// innocent DDOS, by clogging the machine's resources and network.
 |  | ||||||
| 		jitter := time.Duration(rng.Int63n(maxJitterNanos)) |  | ||||||
| 		select { |  | ||||||
| 		case <-c.stopCh: |  | ||||||
| 			return |  | ||||||
| 		case <-time.After(connReattemptPeriod + jitter): |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) connect(ctx context.Context) error { |  | ||||||
| 	cc, err := c.dialToCollector(ctx) |  | ||||||
| 	if err != nil { |  | ||||||
| 		return err |  | ||||||
| 	} |  | ||||||
| 	c.setConnection(cc) |  | ||||||
| 	c.newConnectionHandler(cc) |  | ||||||
| 	return nil |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| // setConnection sets cc as the client Connection and returns true if
 |  | ||||||
| // the Connection state changed.
 |  | ||||||
| func (c *Connection) setConnection(cc *grpc.ClientConn) bool { |  | ||||||
| 	c.mu.Lock() |  | ||||||
| 	defer c.mu.Unlock() |  | ||||||
| 
 |  | ||||||
| 	// If previous clientConn is same as the current then just return.
 |  | ||||||
| 	// This doesn't happen right now as this func is only called with new ClientConn.
 |  | ||||||
| 	// It is more about future-proofing.
 |  | ||||||
| 	if c.cc == cc { |  | ||||||
| 		return false |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	// If the previous clientConn was non-nil, close it
 |  | ||||||
| 	if c.cc != nil { |  | ||||||
| 		_ = c.cc.Close() |  | ||||||
| 	} |  | ||||||
| 	c.cc = cc |  | ||||||
| 	return true |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) dialToCollector(ctx context.Context) (*grpc.ClientConn, error) { |  | ||||||
| 	if c.cfg.GRPCConn != nil { |  | ||||||
| 		return c.cfg.GRPCConn, nil |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	dialOpts := []grpc.DialOption{} |  | ||||||
| 	if c.cfg.ServiceConfig != "" { |  | ||||||
| 		dialOpts = append(dialOpts, grpc.WithDefaultServiceConfig(c.cfg.ServiceConfig)) |  | ||||||
| 	} |  | ||||||
| 	if c.SCfg.GRPCCredentials != nil { |  | ||||||
| 		dialOpts = append(dialOpts, grpc.WithTransportCredentials(c.SCfg.GRPCCredentials)) |  | ||||||
| 	} else if c.SCfg.Insecure { |  | ||||||
| 		dialOpts = append(dialOpts, grpc.WithInsecure()) |  | ||||||
| 	} |  | ||||||
| 	if c.SCfg.Compression == otlpconfig.GzipCompression { |  | ||||||
| 		dialOpts = append(dialOpts, grpc.WithDefaultCallOptions(grpc.UseCompressor(gzip.Name))) |  | ||||||
| 	} |  | ||||||
| 	if len(c.cfg.DialOptions) != 0 { |  | ||||||
| 		dialOpts = append(dialOpts, c.cfg.DialOptions...) |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	ctx, cancel := c.ContextWithStop(ctx) |  | ||||||
| 	defer cancel() |  | ||||||
| 	ctx = c.ContextWithMetadata(ctx) |  | ||||||
| 	return grpc.DialContext(ctx, c.SCfg.Endpoint, dialOpts...) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) ContextWithMetadata(ctx context.Context) context.Context { |  | ||||||
| 	if c.metadata.Len() > 0 { |  | ||||||
| 		return metadata.NewOutgoingContext(ctx, c.metadata) |  | ||||||
| 	} |  | ||||||
| 	return ctx |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) Shutdown(ctx context.Context) error { |  | ||||||
| 	close(c.stopCh) |  | ||||||
| 	// Ensure that the backgroundConnector returns
 |  | ||||||
| 	select { |  | ||||||
| 	case <-c.backgroundConnectionDoneCh: |  | ||||||
| 	case <-ctx.Done(): |  | ||||||
| 		return ctx.Err() |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	c.mu.Lock() |  | ||||||
| 	cc := c.cc |  | ||||||
| 	c.cc = nil |  | ||||||
| 	c.mu.Unlock() |  | ||||||
| 
 |  | ||||||
| 	if cc != nil { |  | ||||||
| 		return cc.Close() |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	return nil |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) ContextWithStop(ctx context.Context) (context.Context, context.CancelFunc) { |  | ||||||
| 	// Unify the parent context Done signal with the Connection's
 |  | ||||||
| 	// stop channel.
 |  | ||||||
| 	ctx, cancel := context.WithCancel(ctx) |  | ||||||
| 	go func(ctx context.Context, cancel context.CancelFunc) { |  | ||||||
| 		select { |  | ||||||
| 		case <-ctx.Done(): |  | ||||||
| 			// Nothing to do, either cancelled or deadline
 |  | ||||||
| 			// happened.
 |  | ||||||
| 		case <-c.stopCh: |  | ||||||
| 			cancel() |  | ||||||
| 		} |  | ||||||
| 	}(ctx, cancel) |  | ||||||
| 	return ctx, cancel |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (c *Connection) DoRequest(ctx context.Context, fn func(context.Context) error) error { |  | ||||||
| 	expBackoff := newExponentialBackoff(c.cfg.RetrySettings) |  | ||||||
| 
 |  | ||||||
| 	for { |  | ||||||
| 		err := fn(ctx) |  | ||||||
| 		if err == nil { |  | ||||||
| 			// request succeeded.
 |  | ||||||
| 			return nil |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		if !c.cfg.RetrySettings.Enabled { |  | ||||||
| 			return err |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		// We have an error, check gRPC status code.
 |  | ||||||
| 		st := status.Convert(err) |  | ||||||
| 		if st.Code() == codes.OK { |  | ||||||
| 			// Not really an error, still success.
 |  | ||||||
| 			return nil |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		// Now, this is this a real error.
 |  | ||||||
| 
 |  | ||||||
| 		if !shouldRetry(st.Code()) { |  | ||||||
| 			// It is not a retryable error, we should not retry.
 |  | ||||||
| 			return err |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		// Need to retry.
 |  | ||||||
| 
 |  | ||||||
| 		throttle := getThrottleDuration(st) |  | ||||||
| 
 |  | ||||||
| 		backoffDelay := expBackoff.NextBackOff() |  | ||||||
| 		if backoffDelay == backoff.Stop { |  | ||||||
| 			// throw away the batch
 |  | ||||||
| 			err = fmt.Errorf("max elapsed time expired: %w", err) |  | ||||||
| 			return err |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		var delay time.Duration |  | ||||||
| 
 |  | ||||||
| 		if backoffDelay > throttle { |  | ||||||
| 			delay = backoffDelay |  | ||||||
| 		} else { |  | ||||||
| 			if expBackoff.GetElapsedTime()+throttle > expBackoff.MaxElapsedTime { |  | ||||||
| 				err = fmt.Errorf("max elapsed time expired when respecting server throttle: %w", err) |  | ||||||
| 				return err |  | ||||||
| 			} |  | ||||||
| 
 |  | ||||||
| 			// Respect server throttling.
 |  | ||||||
| 			delay = throttle |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		// back-off, but get interrupted when shutting down or request is cancelled or timed out.
 |  | ||||||
| 		err = func() error { |  | ||||||
| 			dt := time.NewTimer(delay) |  | ||||||
| 			defer dt.Stop() |  | ||||||
| 
 |  | ||||||
| 			select { |  | ||||||
| 			case <-ctx.Done(): |  | ||||||
| 				return ctx.Err() |  | ||||||
| 			case <-c.stopCh: |  | ||||||
| 				return fmt.Errorf("interrupted due to shutdown: %w", err) |  | ||||||
| 			case <-dt.C: |  | ||||||
| 			} |  | ||||||
| 
 |  | ||||||
| 			return nil |  | ||||||
| 		}() |  | ||||||
| 
 |  | ||||||
| 		if err != nil { |  | ||||||
| 			return err |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func shouldRetry(code codes.Code) bool { |  | ||||||
| 	switch code { |  | ||||||
| 	case codes.OK: |  | ||||||
| 		// Success. This function should not be called for this code, the best we
 |  | ||||||
| 		// can do is tell the caller not to retry.
 |  | ||||||
| 		return false |  | ||||||
| 
 |  | ||||||
| 	case codes.Canceled, |  | ||||||
| 		codes.DeadlineExceeded, |  | ||||||
| 		codes.ResourceExhausted, |  | ||||||
| 		codes.Aborted, |  | ||||||
| 		codes.OutOfRange, |  | ||||||
| 		codes.Unavailable, |  | ||||||
| 		codes.DataLoss: |  | ||||||
| 		// These are retryable errors.
 |  | ||||||
| 		return true |  | ||||||
| 
 |  | ||||||
| 	case codes.Unknown, |  | ||||||
| 		codes.InvalidArgument, |  | ||||||
| 		codes.Unauthenticated, |  | ||||||
| 		codes.PermissionDenied, |  | ||||||
| 		codes.NotFound, |  | ||||||
| 		codes.AlreadyExists, |  | ||||||
| 		codes.FailedPrecondition, |  | ||||||
| 		codes.Unimplemented, |  | ||||||
| 		codes.Internal: |  | ||||||
| 		// These are fatal errors, don't retry.
 |  | ||||||
| 		return false |  | ||||||
| 
 |  | ||||||
| 	default: |  | ||||||
| 		// Don't retry on unknown codes.
 |  | ||||||
| 		return false |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func getThrottleDuration(status *status.Status) time.Duration { |  | ||||||
| 	// See if throttling information is available.
 |  | ||||||
| 	for _, detail := range status.Details() { |  | ||||||
| 		if t, ok := detail.(*errdetails.RetryInfo); ok { |  | ||||||
| 			if t.RetryDelay.Seconds > 0 || t.RetryDelay.Nanos > 0 { |  | ||||||
| 				// We are throttled. Wait before retrying as requested by the server.
 |  | ||||||
| 				return time.Duration(t.RetryDelay.Seconds)*time.Second + time.Duration(t.RetryDelay.Nanos)*time.Nanosecond |  | ||||||
| 			} |  | ||||||
| 			return 0 |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
| 	return 0 |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func newExponentialBackoff(rs otlpconfig.RetrySettings) *backoff.ExponentialBackOff { |  | ||||||
| 	// Do not use NewExponentialBackOff since it calls Reset and the code here must
 |  | ||||||
| 	// call Reset after changing the InitialInterval (this saves an unnecessary call to Now).
 |  | ||||||
| 	expBackoff := &backoff.ExponentialBackOff{ |  | ||||||
| 		InitialInterval:     rs.InitialInterval, |  | ||||||
| 		RandomizationFactor: backoff.DefaultRandomizationFactor, |  | ||||||
| 		Multiplier:          backoff.DefaultMultiplier, |  | ||||||
| 		MaxInterval:         rs.MaxInterval, |  | ||||||
| 		MaxElapsedTime:      rs.MaxElapsedTime, |  | ||||||
| 		Stop:                backoff.Stop, |  | ||||||
| 		Clock:               backoff.SystemClock, |  | ||||||
| 	} |  | ||||||
| 	expBackoff.Reset() |  | ||||||
| 
 |  | ||||||
| 	return expBackoff |  | ||||||
| } |  | ||||||
|  | @ -1,89 +0,0 @@ | ||||||
| // Copyright The OpenTelemetry Authors
 |  | ||||||
| //
 |  | ||||||
| // Licensed under the Apache License, Version 2.0 (the "License");
 |  | ||||||
| // you may not use this file except in compliance with the License.
 |  | ||||||
| // You may obtain a copy of the License at
 |  | ||||||
| //
 |  | ||||||
| //     http://www.apache.org/licenses/LICENSE-2.0
 |  | ||||||
| //
 |  | ||||||
| // Unless required by applicable law or agreed to in writing, software
 |  | ||||||
| // distributed under the License is distributed on an "AS IS" BASIS,
 |  | ||||||
| // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 |  | ||||||
| // See the License for the specific language governing permissions and
 |  | ||||||
| // limitations under the License.
 |  | ||||||
| 
 |  | ||||||
| package connection |  | ||||||
| 
 |  | ||||||
| import ( |  | ||||||
| 	"testing" |  | ||||||
| 	"time" |  | ||||||
| 
 |  | ||||||
| 	"github.com/stretchr/testify/require" |  | ||||||
| 	"google.golang.org/genproto/googleapis/rpc/errdetails" |  | ||||||
| 	"google.golang.org/grpc/codes" |  | ||||||
| 	"google.golang.org/grpc/status" |  | ||||||
| 	"google.golang.org/protobuf/types/known/durationpb" |  | ||||||
| ) |  | ||||||
| 
 |  | ||||||
| func TestGetThrottleDuration(t *testing.T) { |  | ||||||
| 	tts := []struct { |  | ||||||
| 		stsFn    func() (*status.Status, error) |  | ||||||
| 		throttle time.Duration |  | ||||||
| 	}{ |  | ||||||
| 		{ |  | ||||||
| 			stsFn: func() (*status.Status, error) { |  | ||||||
| 				return status.New( |  | ||||||
| 					codes.OK, |  | ||||||
| 					"status with no retry info", |  | ||||||
| 				), nil |  | ||||||
| 			}, |  | ||||||
| 			throttle: 0, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			stsFn: func() (*status.Status, error) { |  | ||||||
| 				st := status.New(codes.ResourceExhausted, "status with retry info") |  | ||||||
| 				return st.WithDetails( |  | ||||||
| 					&errdetails.RetryInfo{RetryDelay: durationpb.New(15 * time.Millisecond)}, |  | ||||||
| 				) |  | ||||||
| 			}, |  | ||||||
| 			throttle: 15 * time.Millisecond, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			stsFn: func() (*status.Status, error) { |  | ||||||
| 				st := status.New(codes.ResourceExhausted, "status with error info detail") |  | ||||||
| 				return st.WithDetails( |  | ||||||
| 					&errdetails.ErrorInfo{Reason: "no throttle detail"}, |  | ||||||
| 				) |  | ||||||
| 			}, |  | ||||||
| 			throttle: 0, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			stsFn: func() (*status.Status, error) { |  | ||||||
| 				st := status.New(codes.ResourceExhausted, "status with error info and retry info") |  | ||||||
| 				return st.WithDetails( |  | ||||||
| 					&errdetails.ErrorInfo{Reason: "no throttle detail"}, |  | ||||||
| 					&errdetails.RetryInfo{RetryDelay: durationpb.New(13 * time.Minute)}, |  | ||||||
| 				) |  | ||||||
| 			}, |  | ||||||
| 			throttle: 13 * time.Minute, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			stsFn: func() (*status.Status, error) { |  | ||||||
| 				st := status.New(codes.ResourceExhausted, "status with two retry info should take the first") |  | ||||||
| 				return st.WithDetails( |  | ||||||
| 					&errdetails.RetryInfo{RetryDelay: durationpb.New(13 * time.Minute)}, |  | ||||||
| 					&errdetails.RetryInfo{RetryDelay: durationpb.New(18 * time.Minute)}, |  | ||||||
| 				) |  | ||||||
| 			}, |  | ||||||
| 			throttle: 13 * time.Minute, |  | ||||||
| 		}, |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	for _, tt := range tts { |  | ||||||
| 		sts, _ := tt.stsFn() |  | ||||||
| 		t.Run(sts.Message(), func(t *testing.T) { |  | ||||||
| 			th := getThrottleDuration(sts) |  | ||||||
| 			require.Equal(t, tt.throttle, th) |  | ||||||
| 		}) |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  | @ -20,7 +20,11 @@ import ( | ||||||
| 	"time" | 	"time" | ||||||
| 
 | 
 | ||||||
| 	"google.golang.org/grpc" | 	"google.golang.org/grpc" | ||||||
|  | 	"google.golang.org/grpc/backoff" | ||||||
| 	"google.golang.org/grpc/credentials" | 	"google.golang.org/grpc/credentials" | ||||||
|  | 	"google.golang.org/grpc/encoding/gzip" | ||||||
|  | 
 | ||||||
|  | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/retry" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| const ( | const ( | ||||||
|  | @ -39,16 +43,6 @@ const ( | ||||||
| 	DefaultTimeout time.Duration = 10 * time.Second | 	DefaultTimeout time.Duration = 10 * time.Second | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| var ( |  | ||||||
| 	// defaultRetrySettings is a default settings for the retry policy.
 |  | ||||||
| 	defaultRetrySettings = RetrySettings{ |  | ||||||
| 		Enabled:         true, |  | ||||||
| 		InitialInterval: 5 * time.Second, |  | ||||||
| 		MaxInterval:     30 * time.Second, |  | ||||||
| 		MaxElapsedTime:  time.Minute, |  | ||||||
| 	} |  | ||||||
| ) |  | ||||||
| 
 |  | ||||||
| type ( | type ( | ||||||
| 	SignalConfig struct { | 	SignalConfig struct { | ||||||
| 		Endpoint    string | 		Endpoint    string | ||||||
|  | @ -67,16 +61,13 @@ type ( | ||||||
| 		// Signal specific configurations
 | 		// Signal specific configurations
 | ||||||
| 		Metrics SignalConfig | 		Metrics SignalConfig | ||||||
| 
 | 
 | ||||||
| 		// HTTP configurations
 | 		RetryConfig retry.Config | ||||||
| 		MaxAttempts int |  | ||||||
| 		Backoff     time.Duration |  | ||||||
| 
 | 
 | ||||||
| 		// gRPC configurations
 | 		// gRPC configurations
 | ||||||
| 		ReconnectionPeriod time.Duration | 		ReconnectionPeriod time.Duration | ||||||
| 		ServiceConfig      string | 		ServiceConfig      string | ||||||
| 		DialOptions        []grpc.DialOption | 		DialOptions        []grpc.DialOption | ||||||
| 		GRPCConn           *grpc.ClientConn | 		GRPCConn           *grpc.ClientConn | ||||||
| 		RetrySettings      RetrySettings |  | ||||||
| 	} | 	} | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
|  | @ -88,12 +79,46 @@ func NewDefaultConfig() Config { | ||||||
| 			Compression: NoCompression, | 			Compression: NoCompression, | ||||||
| 			Timeout:     DefaultTimeout, | 			Timeout:     DefaultTimeout, | ||||||
| 		}, | 		}, | ||||||
| 		RetrySettings: defaultRetrySettings, | 		RetryConfig: retry.DefaultConfig, | ||||||
| 	} | 	} | ||||||
| 
 | 
 | ||||||
| 	return c | 	return c | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | // NewGRPCConfig returns a new Config with all settings applied from opts and
 | ||||||
|  | // any unset setting using the default gRPC config values.
 | ||||||
|  | func NewGRPCConfig(opts ...GRPCOption) Config { | ||||||
|  | 	cfg := NewDefaultConfig() | ||||||
|  | 	ApplyGRPCEnvConfigs(&cfg) | ||||||
|  | 	for _, opt := range opts { | ||||||
|  | 		opt.ApplyGRPCOption(&cfg) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	if cfg.ServiceConfig != "" { | ||||||
|  | 		cfg.DialOptions = append(cfg.DialOptions, grpc.WithDefaultServiceConfig(cfg.ServiceConfig)) | ||||||
|  | 	} | ||||||
|  | 	if cfg.Metrics.GRPCCredentials != nil { | ||||||
|  | 		cfg.DialOptions = append(cfg.DialOptions, grpc.WithTransportCredentials(cfg.Metrics.GRPCCredentials)) | ||||||
|  | 	} else if cfg.Metrics.Insecure { | ||||||
|  | 		cfg.DialOptions = append(cfg.DialOptions, grpc.WithInsecure()) | ||||||
|  | 	} | ||||||
|  | 	if cfg.Metrics.Compression == GzipCompression { | ||||||
|  | 		cfg.DialOptions = append(cfg.DialOptions, grpc.WithDefaultCallOptions(grpc.UseCompressor(gzip.Name))) | ||||||
|  | 	} | ||||||
|  | 	if len(cfg.DialOptions) != 0 { | ||||||
|  | 		cfg.DialOptions = append(cfg.DialOptions, cfg.DialOptions...) | ||||||
|  | 	} | ||||||
|  | 	if cfg.ReconnectionPeriod != 0 { | ||||||
|  | 		p := grpc.ConnectParams{ | ||||||
|  | 			Backoff:           backoff.DefaultConfig, | ||||||
|  | 			MinConnectTimeout: cfg.ReconnectionPeriod, | ||||||
|  | 		} | ||||||
|  | 		cfg.DialOptions = append(cfg.DialOptions, grpc.WithConnectParams(p)) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return cfg | ||||||
|  | } | ||||||
|  | 
 | ||||||
| type ( | type ( | ||||||
| 	// GenericOption applies an option to the HTTP or gRPC driver.
 | 	// GenericOption applies an option to the HTTP or gRPC driver.
 | ||||||
| 	GenericOption interface { | 	GenericOption interface { | ||||||
|  | @ -218,9 +243,9 @@ func WithURLPath(urlPath string) GenericOption { | ||||||
| 	}) | 	}) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func WithRetry(settings RetrySettings) GenericOption { | func WithRetry(rc retry.Config) GenericOption { | ||||||
| 	return newGenericOption(func(cfg *Config) { | 	return newGenericOption(func(cfg *Config) { | ||||||
| 		cfg.RetrySettings = settings | 		cfg.RetryConfig = rc | ||||||
| 	}) | 	}) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | @ -255,15 +280,3 @@ func WithTimeout(duration time.Duration) GenericOption { | ||||||
| 		cfg.Metrics.Timeout = duration | 		cfg.Metrics.Timeout = duration | ||||||
| 	}) | 	}) | ||||||
| } | } | ||||||
| 
 |  | ||||||
| func WithMaxAttempts(maxAttempts int) GenericOption { |  | ||||||
| 	return newGenericOption(func(cfg *Config) { |  | ||||||
| 		cfg.MaxAttempts = maxAttempts |  | ||||||
| 	}) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func WithBackoff(duration time.Duration) GenericOption { |  | ||||||
| 	return newGenericOption(func(cfg *Config) { |  | ||||||
| 		cfg.Backoff = duration |  | ||||||
| 	}) |  | ||||||
| } |  | ||||||
|  |  | ||||||
|  | @ -55,6 +55,14 @@ func initializeExporter(t *testing.T, client otlpmetric.Client) *otlpmetric.Expo | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func testClientStopHonorsTimeout(t *testing.T, client otlpmetric.Client) { | func testClientStopHonorsTimeout(t *testing.T, client otlpmetric.Client) { | ||||||
|  | 	t.Cleanup(func() { | ||||||
|  | 		// The test is looking for a failed shut down. Call Stop a second time
 | ||||||
|  | 		// with an un-expired context to give the client a second chance at
 | ||||||
|  | 		// cleaning up. There is not guarantee from the Client interface this
 | ||||||
|  | 		// will succeed, therefore, no need to check the error (just give it a
 | ||||||
|  | 		// best try).
 | ||||||
|  | 		_ = client.Stop(context.Background()) | ||||||
|  | 	}) | ||||||
| 	e := initializeExporter(t, client) | 	e := initializeExporter(t, client) | ||||||
| 
 | 
 | ||||||
| 	innerCtx, innerCancel := context.WithTimeout(context.Background(), time.Microsecond) | 	innerCtx, innerCancel := context.WithTimeout(context.Background(), time.Microsecond) | ||||||
|  | @ -68,6 +76,14 @@ func testClientStopHonorsTimeout(t *testing.T, client otlpmetric.Client) { | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func testClientStopHonorsCancel(t *testing.T, client otlpmetric.Client) { | func testClientStopHonorsCancel(t *testing.T, client otlpmetric.Client) { | ||||||
|  | 	t.Cleanup(func() { | ||||||
|  | 		// The test is looking for a failed shut down. Call Stop a second time
 | ||||||
|  | 		// with an un-expired context to give the client a second chance at
 | ||||||
|  | 		// cleaning up. There is not guarantee from the Client interface this
 | ||||||
|  | 		// will succeed, therefore, no need to check the error (just give it a
 | ||||||
|  | 		// best try).
 | ||||||
|  | 		_ = client.Stop(context.Background()) | ||||||
|  | 	}) | ||||||
| 	e := initializeExporter(t, client) | 	e := initializeExporter(t, client) | ||||||
| 
 | 
 | ||||||
| 	ctx, innerCancel := context.WithCancel(context.Background()) | 	ctx, innerCancel := context.WithCancel(context.Background()) | ||||||
|  |  | ||||||
|  | @ -0,0 +1,137 @@ | ||||||
|  | // Copyright The OpenTelemetry Authors
 | ||||||
|  | //
 | ||||||
|  | // Licensed under the Apache License, Version 2.0 (the "License");
 | ||||||
|  | // you may not use this file except in compliance with the License.
 | ||||||
|  | // You may obtain a copy of the License at
 | ||||||
|  | //
 | ||||||
|  | //     http://www.apache.org/licenses/LICENSE-2.0
 | ||||||
|  | //
 | ||||||
|  | // Unless required by applicable law or agreed to in writing, software
 | ||||||
|  | // distributed under the License is distributed on an "AS IS" BASIS,
 | ||||||
|  | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 | ||||||
|  | // See the License for the specific language governing permissions and
 | ||||||
|  | // limitations under the License.
 | ||||||
|  | 
 | ||||||
|  | package retry // import "go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/retry"
 | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"context" | ||||||
|  | 	"fmt" | ||||||
|  | 	"time" | ||||||
|  | 
 | ||||||
|  | 	"github.com/cenkalti/backoff/v4" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | // DefaultConfig are the recommended defaults to use.
 | ||||||
|  | var DefaultConfig = Config{ | ||||||
|  | 	Enabled:         true, | ||||||
|  | 	InitialInterval: 5 * time.Second, | ||||||
|  | 	MaxInterval:     30 * time.Second, | ||||||
|  | 	MaxElapsedTime:  time.Minute, | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // Config defines configuration for retrying batches in case of export failure
 | ||||||
|  | // using an exponential backoff.
 | ||||||
|  | type Config struct { | ||||||
|  | 	// Enabled indicates whether to not retry sending batches in case of
 | ||||||
|  | 	// export failure.
 | ||||||
|  | 	Enabled bool | ||||||
|  | 	// InitialInterval the time to wait after the first failure before
 | ||||||
|  | 	// retrying.
 | ||||||
|  | 	InitialInterval time.Duration | ||||||
|  | 	// MaxInterval is the upper bound on backoff interval. Once this value is
 | ||||||
|  | 	// reached the delay between consecutive retries will always be
 | ||||||
|  | 	// `MaxInterval`.
 | ||||||
|  | 	MaxInterval time.Duration | ||||||
|  | 	// MaxElapsedTime is the maximum amount of time (including retries) spent
 | ||||||
|  | 	// trying to send a request/batch.  Once this value is reached, the data
 | ||||||
|  | 	// is discarded.
 | ||||||
|  | 	MaxElapsedTime time.Duration | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // RequestFunc wraps a request with retry logic.
 | ||||||
|  | type RequestFunc func(context.Context, func(context.Context) error) error | ||||||
|  | 
 | ||||||
|  | // EvaluateFunc returns if an error is retry-able and if an explicit throttle
 | ||||||
|  | // duration should be honored that was included in the error.
 | ||||||
|  | type EvaluateFunc func(error) (bool, time.Duration) | ||||||
|  | 
 | ||||||
|  | func (c Config) RequestFunc(evaluate EvaluateFunc) RequestFunc { | ||||||
|  | 	if !c.Enabled { | ||||||
|  | 		return func(ctx context.Context, fn func(context.Context) error) error { | ||||||
|  | 			return fn(ctx) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// Do not use NewExponentialBackOff since it calls Reset and the code here
 | ||||||
|  | 	// must call Reset after changing the InitialInterval (this saves an
 | ||||||
|  | 	// unnecessary call to Now).
 | ||||||
|  | 	b := &backoff.ExponentialBackOff{ | ||||||
|  | 		InitialInterval:     c.InitialInterval, | ||||||
|  | 		RandomizationFactor: backoff.DefaultRandomizationFactor, | ||||||
|  | 		Multiplier:          backoff.DefaultMultiplier, | ||||||
|  | 		MaxInterval:         c.MaxInterval, | ||||||
|  | 		MaxElapsedTime:      c.MaxElapsedTime, | ||||||
|  | 		Stop:                backoff.Stop, | ||||||
|  | 		Clock:               backoff.SystemClock, | ||||||
|  | 	} | ||||||
|  | 	b.Reset() | ||||||
|  | 
 | ||||||
|  | 	return func(ctx context.Context, fn func(context.Context) error) error { | ||||||
|  | 		for { | ||||||
|  | 			err := fn(ctx) | ||||||
|  | 			if err == nil { | ||||||
|  | 				return nil | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			retryable, throttle := evaluate(err) | ||||||
|  | 			if !retryable { | ||||||
|  | 				return err | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			bOff := b.NextBackOff() | ||||||
|  | 			if bOff == backoff.Stop { | ||||||
|  | 				return fmt.Errorf("max retry time elapsed: %w", err) | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			// Wait for the greater of the backoff or throttle delay.
 | ||||||
|  | 			var delay time.Duration | ||||||
|  | 			if bOff > throttle { | ||||||
|  | 				delay = bOff | ||||||
|  | 			} else { | ||||||
|  | 				elapsed := b.GetElapsedTime() | ||||||
|  | 				if b.MaxElapsedTime != 0 && elapsed+throttle > b.MaxElapsedTime { | ||||||
|  | 					return fmt.Errorf("max retry time would elapse: %w", err) | ||||||
|  | 				} | ||||||
|  | 				delay = throttle | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			if err := waitFunc(ctx, delay); err != nil { | ||||||
|  | 				return err | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // Allow override for testing.
 | ||||||
|  | var waitFunc = wait | ||||||
|  | 
 | ||||||
|  | func wait(ctx context.Context, delay time.Duration) error { | ||||||
|  | 	timer := time.NewTimer(delay) | ||||||
|  | 	defer timer.Stop() | ||||||
|  | 
 | ||||||
|  | 	select { | ||||||
|  | 	case <-ctx.Done(): | ||||||
|  | 		// Handle the case where the timer and context deadline end
 | ||||||
|  | 		// simultaneously by prioritizing the timer expiration nil value
 | ||||||
|  | 		// response.
 | ||||||
|  | 		select { | ||||||
|  | 		case <-timer.C: | ||||||
|  | 		default: | ||||||
|  | 			return ctx.Err() | ||||||
|  | 		} | ||||||
|  | 	case <-timer.C: | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  | @ -0,0 +1,197 @@ | ||||||
|  | // Copyright The OpenTelemetry Authors
 | ||||||
|  | //
 | ||||||
|  | // Licensed under the Apache License, Version 2.0 (the "License");
 | ||||||
|  | // you may not use this file except in compliance with the License.
 | ||||||
|  | // You may obtain a copy of the License at
 | ||||||
|  | //
 | ||||||
|  | //     http://www.apache.org/licenses/LICENSE-2.0
 | ||||||
|  | //
 | ||||||
|  | // Unless required by applicable law or agreed to in writing, software
 | ||||||
|  | // distributed under the License is distributed on an "AS IS" BASIS,
 | ||||||
|  | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 | ||||||
|  | // See the License for the specific language governing permissions and
 | ||||||
|  | // limitations under the License.
 | ||||||
|  | 
 | ||||||
|  | package retry | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"context" | ||||||
|  | 	"errors" | ||||||
|  | 	"testing" | ||||||
|  | 	"time" | ||||||
|  | 
 | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | func TestWait(t *testing.T) { | ||||||
|  | 	tests := []struct { | ||||||
|  | 		ctx      context.Context | ||||||
|  | 		delay    time.Duration | ||||||
|  | 		expected error | ||||||
|  | 	}{ | ||||||
|  | 		{ | ||||||
|  | 			ctx:      context.Background(), | ||||||
|  | 			delay:    time.Duration(0), | ||||||
|  | 			expected: nil, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			ctx:      context.Background(), | ||||||
|  | 			delay:    time.Duration(1), | ||||||
|  | 			expected: nil, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			ctx:      context.Background(), | ||||||
|  | 			delay:    time.Duration(-1), | ||||||
|  | 			expected: nil, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			ctx: func() context.Context { | ||||||
|  | 				ctx, cancel := context.WithCancel(context.Background()) | ||||||
|  | 				cancel() | ||||||
|  | 				return ctx | ||||||
|  | 			}(), | ||||||
|  | 			// Ensure the timer and context do not end simultaneously.
 | ||||||
|  | 			delay:    1 * time.Hour, | ||||||
|  | 			expected: context.Canceled, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for _, test := range tests { | ||||||
|  | 		assert.Equal(t, test.expected, wait(test.ctx, test.delay)) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestNonRetryableError(t *testing.T) { | ||||||
|  | 	ev := func(error) (bool, time.Duration) { return false, 0 } | ||||||
|  | 
 | ||||||
|  | 	reqFunc := Config{ | ||||||
|  | 		Enabled:         true, | ||||||
|  | 		InitialInterval: 1 * time.Nanosecond, | ||||||
|  | 		MaxInterval:     1 * time.Nanosecond, | ||||||
|  | 		// Never stop retrying.
 | ||||||
|  | 		MaxElapsedTime: 0, | ||||||
|  | 	}.RequestFunc(ev) | ||||||
|  | 	ctx := context.Background() | ||||||
|  | 	assert.NoError(t, reqFunc(ctx, func(context.Context) error { | ||||||
|  | 		return nil | ||||||
|  | 	})) | ||||||
|  | 	assert.ErrorIs(t, reqFunc(ctx, func(context.Context) error { | ||||||
|  | 		return assert.AnError | ||||||
|  | 	}), assert.AnError) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestThrottledRetry(t *testing.T) { | ||||||
|  | 	// Ensure the throttle delay is used by making longer than backoff delay.
 | ||||||
|  | 	throttleDelay, backoffDelay := time.Second, time.Nanosecond | ||||||
|  | 
 | ||||||
|  | 	ev := func(error) (bool, time.Duration) { | ||||||
|  | 		// Retry everything with a throttle delay.
 | ||||||
|  | 		return true, throttleDelay | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	reqFunc := Config{ | ||||||
|  | 		Enabled:         true, | ||||||
|  | 		InitialInterval: backoffDelay, | ||||||
|  | 		MaxInterval:     backoffDelay, | ||||||
|  | 		// Never stop retrying.
 | ||||||
|  | 		MaxElapsedTime: 0, | ||||||
|  | 	}.RequestFunc(ev) | ||||||
|  | 
 | ||||||
|  | 	origWait := waitFunc | ||||||
|  | 	var done bool | ||||||
|  | 	waitFunc = func(_ context.Context, delay time.Duration) error { | ||||||
|  | 		assert.Equal(t, throttleDelay, delay, "retry not throttled") | ||||||
|  | 		// Try twice to ensure call is attempted again after delay.
 | ||||||
|  | 		if done { | ||||||
|  | 			return assert.AnError | ||||||
|  | 		} | ||||||
|  | 		done = true | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  | 	defer func() { waitFunc = origWait }() | ||||||
|  | 
 | ||||||
|  | 	ctx := context.Background() | ||||||
|  | 	assert.ErrorIs(t, reqFunc(ctx, func(context.Context) error { | ||||||
|  | 		return errors.New("not this error") | ||||||
|  | 	}), assert.AnError) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestBackoffRetry(t *testing.T) { | ||||||
|  | 	ev := func(error) (bool, time.Duration) { return true, 0 } | ||||||
|  | 
 | ||||||
|  | 	delay := time.Nanosecond | ||||||
|  | 	reqFunc := Config{ | ||||||
|  | 		Enabled:         true, | ||||||
|  | 		InitialInterval: delay, | ||||||
|  | 		MaxInterval:     delay, | ||||||
|  | 		// Never stop retrying.
 | ||||||
|  | 		MaxElapsedTime: 0, | ||||||
|  | 	}.RequestFunc(ev) | ||||||
|  | 
 | ||||||
|  | 	origWait := waitFunc | ||||||
|  | 	var done bool | ||||||
|  | 	waitFunc = func(_ context.Context, d time.Duration) error { | ||||||
|  | 		assert.Equal(t, delay, d, "retry not backoffed") | ||||||
|  | 		// Try twice to ensure call is attempted again after delay.
 | ||||||
|  | 		if done { | ||||||
|  | 			return assert.AnError | ||||||
|  | 		} | ||||||
|  | 		done = true | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  | 	defer func() { waitFunc = origWait }() | ||||||
|  | 
 | ||||||
|  | 	ctx := context.Background() | ||||||
|  | 	assert.ErrorIs(t, reqFunc(ctx, func(context.Context) error { | ||||||
|  | 		return errors.New("not this error") | ||||||
|  | 	}), assert.AnError) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestThrottledRetryGreaterThanMaxElapsedTime(t *testing.T) { | ||||||
|  | 	// Ensure the throttle delay is used by making longer than backoff delay.
 | ||||||
|  | 	tDelay, bDelay := time.Hour, time.Nanosecond | ||||||
|  | 	ev := func(error) (bool, time.Duration) { return true, tDelay } | ||||||
|  | 	reqFunc := Config{ | ||||||
|  | 		Enabled:         true, | ||||||
|  | 		InitialInterval: bDelay, | ||||||
|  | 		MaxInterval:     bDelay, | ||||||
|  | 		MaxElapsedTime:  tDelay - (time.Nanosecond), | ||||||
|  | 	}.RequestFunc(ev) | ||||||
|  | 
 | ||||||
|  | 	ctx := context.Background() | ||||||
|  | 	assert.Contains(t, reqFunc(ctx, func(context.Context) error { | ||||||
|  | 		return assert.AnError | ||||||
|  | 	}).Error(), "max retry time would elapse: ") | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestMaxElapsedTime(t *testing.T) { | ||||||
|  | 	ev := func(error) (bool, time.Duration) { return true, 0 } | ||||||
|  | 	delay := time.Nanosecond | ||||||
|  | 	reqFunc := Config{ | ||||||
|  | 		Enabled: true, | ||||||
|  | 		// InitialInterval > MaxElapsedTime means immediate return.
 | ||||||
|  | 		InitialInterval: 2 * delay, | ||||||
|  | 		MaxElapsedTime:  delay, | ||||||
|  | 	}.RequestFunc(ev) | ||||||
|  | 
 | ||||||
|  | 	ctx := context.Background() | ||||||
|  | 	assert.Contains(t, reqFunc(ctx, func(context.Context) error { | ||||||
|  | 		return assert.AnError | ||||||
|  | 	}).Error(), "max retry time elapsed: ") | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestRetryNotEnabled(t *testing.T) { | ||||||
|  | 	ev := func(error) (bool, time.Duration) { | ||||||
|  | 		t.Error("evaluated retry when not enabled") | ||||||
|  | 		return false, 0 | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	reqFunc := Config{}.RequestFunc(ev) | ||||||
|  | 	ctx := context.Background() | ||||||
|  | 	assert.NoError(t, reqFunc(ctx, func(context.Context) error { | ||||||
|  | 		return nil | ||||||
|  | 	})) | ||||||
|  | 	assert.ErrorIs(t, reqFunc(ctx, func(context.Context) error { | ||||||
|  | 		return assert.AnError | ||||||
|  | 	}), assert.AnError) | ||||||
|  | } | ||||||
|  | @ -17,91 +17,259 @@ package otlpmetricgrpc // import "go.opentelemetry.io/otel/exporters/otlp/otlpme | ||||||
| import ( | import ( | ||||||
| 	"context" | 	"context" | ||||||
| 	"errors" | 	"errors" | ||||||
| 	"fmt" |  | ||||||
| 	"sync" | 	"sync" | ||||||
|  | 	"time" | ||||||
| 
 | 
 | ||||||
|  | 	"google.golang.org/genproto/googleapis/rpc/errdetails" | ||||||
| 	"google.golang.org/grpc" | 	"google.golang.org/grpc" | ||||||
|  | 	"google.golang.org/grpc/codes" | ||||||
|  | 	"google.golang.org/grpc/metadata" | ||||||
|  | 	"google.golang.org/grpc/status" | ||||||
| 
 | 
 | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric" | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric" | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/connection" |  | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpconfig" | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpconfig" | ||||||
|  | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/retry" | ||||||
| 	colmetricpb "go.opentelemetry.io/proto/otlp/collector/metrics/v1" | 	colmetricpb "go.opentelemetry.io/proto/otlp/collector/metrics/v1" | ||||||
| 	metricpb "go.opentelemetry.io/proto/otlp/metrics/v1" | 	metricpb "go.opentelemetry.io/proto/otlp/metrics/v1" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| type client struct { | type client struct { | ||||||
| 	connection *connection.Connection | 	endpoint      string | ||||||
|  | 	dialOpts      []grpc.DialOption | ||||||
|  | 	metadata      metadata.MD | ||||||
|  | 	exportTimeout time.Duration | ||||||
|  | 	requestFunc   retry.RequestFunc | ||||||
| 
 | 
 | ||||||
| 	lock          sync.Mutex | 	// stopCtx is used as a parent context for all exports. Therefore, when it
 | ||||||
| 	metricsClient colmetricpb.MetricsServiceClient | 	// is canceled with the stopFunc all exports are canceled.
 | ||||||
|  | 	stopCtx context.Context | ||||||
|  | 	// stopFunc cancels stopCtx, stopping any active exports.
 | ||||||
|  | 	stopFunc context.CancelFunc | ||||||
|  | 
 | ||||||
|  | 	// ourConn keeps track of where conn was created: true if created here on
 | ||||||
|  | 	// Start, or false if passed with an option. This is important on Shutdown
 | ||||||
|  | 	// as the conn should only be closed if created here on start. Otherwise,
 | ||||||
|  | 	// it is up to the processes that passed the conn to close it.
 | ||||||
|  | 	ourConn bool | ||||||
|  | 	conn    *grpc.ClientConn | ||||||
|  | 	mscMu   sync.RWMutex | ||||||
|  | 	msc     colmetricpb.MetricsServiceClient | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| var ( | // Compile time check *client implements otlpmetric.Client.
 | ||||||
| 	errNoClient = errors.New("no client") | var _ otlpmetric.Client = (*client)(nil) | ||||||
| ) |  | ||||||
| 
 | 
 | ||||||
| // NewClient creates a new gRPC metric client.
 | // NewClient creates a new gRPC metric client.
 | ||||||
| func NewClient(opts ...Option) otlpmetric.Client { | func NewClient(opts ...Option) otlpmetric.Client { | ||||||
| 	cfg := otlpconfig.NewDefaultConfig() | 	return newClient(opts...) | ||||||
| 	otlpconfig.ApplyGRPCEnvConfigs(&cfg) | } | ||||||
| 	for _, opt := range opts { | 
 | ||||||
| 		opt.applyGRPCOption(&cfg) | func newClient(opts ...Option) *client { | ||||||
|  | 	cfg := otlpconfig.NewGRPCConfig(asGRPCOptions(opts)...) | ||||||
|  | 
 | ||||||
|  | 	ctx, cancel := context.WithCancel(context.Background()) | ||||||
|  | 
 | ||||||
|  | 	c := &client{ | ||||||
|  | 		endpoint:      cfg.Metrics.Endpoint, | ||||||
|  | 		exportTimeout: cfg.Metrics.Timeout, | ||||||
|  | 		requestFunc:   cfg.RetryConfig.RequestFunc(retryable), | ||||||
|  | 		dialOpts:      cfg.DialOptions, | ||||||
|  | 		stopCtx:       ctx, | ||||||
|  | 		stopFunc:      cancel, | ||||||
|  | 		conn:          cfg.GRPCConn, | ||||||
| 	} | 	} | ||||||
| 
 | 
 | ||||||
| 	c := &client{} | 	if len(cfg.Metrics.Headers) > 0 { | ||||||
| 	c.connection = connection.NewConnection(cfg, cfg.Metrics, c.handleNewConnection) | 		c.metadata = metadata.New(cfg.Metrics.Headers) | ||||||
|  | 	} | ||||||
| 
 | 
 | ||||||
| 	return c | 	return c | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func (c *client) handleNewConnection(cc *grpc.ClientConn) { | // Start establishes a gRPC connection to the collector.
 | ||||||
| 	c.lock.Lock() |  | ||||||
| 	defer c.lock.Unlock() |  | ||||||
| 	if cc != nil { |  | ||||||
| 		c.metricsClient = colmetricpb.NewMetricsServiceClient(cc) |  | ||||||
| 	} else { |  | ||||||
| 		c.metricsClient = nil |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| // Start establishes a connection to the collector.
 |  | ||||||
| func (c *client) Start(ctx context.Context) error { | func (c *client) Start(ctx context.Context) error { | ||||||
| 	return c.connection.StartConnection(ctx) | 	if c.conn == nil { | ||||||
| } | 		// If the caller did not provide a ClientConn when the client was
 | ||||||
| 
 | 		// created, create one using the configuration they did provide.
 | ||||||
| // Stop shuts down the connection to the collector.
 | 		conn, err := grpc.DialContext(ctx, c.endpoint, c.dialOpts...) | ||||||
| func (c *client) Stop(ctx context.Context) error { | 		if err != nil { | ||||||
| 	return c.connection.Shutdown(ctx) | 			return err | ||||||
| } | 		} | ||||||
| 
 | 		// Keep track that we own the lifecycle of this conn and need to close
 | ||||||
| // UploadMetrics sends a batch of metrics to the collector.
 | 		// it on Shutdown.
 | ||||||
| func (c *client) UploadMetrics(ctx context.Context, protoMetrics []*metricpb.ResourceMetrics) error { | 		c.ourConn = true | ||||||
| 	if !c.connection.Connected() { | 		c.conn = conn | ||||||
| 		return fmt.Errorf("metrics exporter is disconnected from the server %s: %w", c.connection.SCfg.Endpoint, c.connection.LastConnectError()) |  | ||||||
| 	} | 	} | ||||||
| 
 | 
 | ||||||
| 	ctx, cancel := c.connection.ContextWithStop(ctx) | 	// The otlpmetric.Client interface states this method is called just once,
 | ||||||
| 	defer cancel() | 	// so no need to check if already started.
 | ||||||
| 	ctx, tCancel := context.WithTimeout(ctx, c.connection.SCfg.Timeout) | 	c.mscMu.Lock() | ||||||
| 	defer tCancel() | 	c.msc = colmetricpb.NewMetricsServiceClient(c.conn) | ||||||
|  | 	c.mscMu.Unlock() | ||||||
| 
 | 
 | ||||||
| 	ctx = c.connection.ContextWithMetadata(ctx) | 	return nil | ||||||
| 	err := func() error { | } | ||||||
| 		c.lock.Lock() |  | ||||||
| 		defer c.lock.Unlock() |  | ||||||
| 		if c.metricsClient == nil { |  | ||||||
| 			return errNoClient |  | ||||||
| 		} |  | ||||||
| 
 | 
 | ||||||
| 		return c.connection.DoRequest(ctx, func(ctx context.Context) error { | var errAlreadyStopped = errors.New("the client is already stopped") | ||||||
| 			_, err := c.metricsClient.Export(ctx, &colmetricpb.ExportMetricsServiceRequest{ | 
 | ||||||
| 				ResourceMetrics: protoMetrics, | // Stop shuts down the client.
 | ||||||
| 			}) | //
 | ||||||
| 			return err | // Any active connections to a remote endpoint are closed if they were created
 | ||||||
| 		}) | // by the client. Any gRPC connection passed during creation using
 | ||||||
|  | // WithGRPCConn will not be closed. It is the caller's responsibility to
 | ||||||
|  | // handle cleanup of that resource.
 | ||||||
|  | //
 | ||||||
|  | // This method synchronizes with the UploadMetrics method of the client. It
 | ||||||
|  | // will wait for any active calls to that method to complete unimpeded, or it
 | ||||||
|  | // will cancel any active calls if ctx expires. If ctx expires, the context
 | ||||||
|  | // error will be forwarded as the returned error. All client held resources
 | ||||||
|  | // will still be released in this situation.
 | ||||||
|  | //
 | ||||||
|  | // If the client has already stopped, an error will be returned describing
 | ||||||
|  | // this.
 | ||||||
|  | func (c *client) Stop(ctx context.Context) error { | ||||||
|  | 	// Acquire the c.mscMu lock within the ctx lifetime.
 | ||||||
|  | 	acquired := make(chan struct{}) | ||||||
|  | 	go func() { | ||||||
|  | 		c.mscMu.Lock() | ||||||
|  | 		close(acquired) | ||||||
| 	}() | 	}() | ||||||
| 	if err != nil { | 	var err error | ||||||
| 		c.connection.SetStateDisconnected(err) | 	select { | ||||||
|  | 	case <-ctx.Done(): | ||||||
|  | 		// The Stop timeout is reached. Kill any remaining exports to force
 | ||||||
|  | 		// the clear of the lock and save the timeout error to return and
 | ||||||
|  | 		// signal the shutdown timed out before cleanly stopping.
 | ||||||
|  | 		c.stopFunc() | ||||||
|  | 		err = ctx.Err() | ||||||
|  | 
 | ||||||
|  | 		// To ensure the client is not left in a dirty state c.msc needs to be
 | ||||||
|  | 		// set to nil. To avoid the race condition when doing this, ensure
 | ||||||
|  | 		// that all the exports are killed (initiated by c.stopFunc).
 | ||||||
|  | 		<-acquired | ||||||
|  | 	case <-acquired: | ||||||
|  | 	} | ||||||
|  | 	// Hold the mscMu lock for the rest of the function to ensure no new
 | ||||||
|  | 	// exports are started.
 | ||||||
|  | 	defer c.mscMu.Unlock() | ||||||
|  | 
 | ||||||
|  | 	// The otlpmetric.Client interface states this method is called only
 | ||||||
|  | 	// once, but there is no guarantee it is called after Start. Ensure the
 | ||||||
|  | 	// client is started before doing anything and let the called know if they
 | ||||||
|  | 	// made a mistake.
 | ||||||
|  | 	if c.msc == nil { | ||||||
|  | 		return errAlreadyStopped | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// Clear c.msc to signal the client is stopped.
 | ||||||
|  | 	c.msc = nil | ||||||
|  | 
 | ||||||
|  | 	if c.ourConn { | ||||||
|  | 		closeErr := c.conn.Close() | ||||||
|  | 		// A context timeout error takes precedence over this error.
 | ||||||
|  | 		if err == nil && closeErr != nil { | ||||||
|  | 			err = closeErr | ||||||
|  | 		} | ||||||
| 	} | 	} | ||||||
| 	return err | 	return err | ||||||
| } | } | ||||||
|  | 
 | ||||||
|  | var errShutdown = errors.New("the client is shutdown") | ||||||
|  | 
 | ||||||
|  | // UploadMetrics sends a batch of spans.
 | ||||||
|  | //
 | ||||||
|  | // Retryable errors from the server will be handled according to any
 | ||||||
|  | // RetryConfig the client was created with.
 | ||||||
|  | func (c *client) UploadMetrics(ctx context.Context, protoMetrics []*metricpb.ResourceMetrics) error { | ||||||
|  | 	// Hold a read lock to ensure a shut down initiated after this starts does
 | ||||||
|  | 	// not abandon the export. This read lock acquire has less priority than a
 | ||||||
|  | 	// write lock acquire (i.e. Stop), meaning if the client is shutting down
 | ||||||
|  | 	// this will come after the shut down.
 | ||||||
|  | 	c.mscMu.RLock() | ||||||
|  | 	defer c.mscMu.RUnlock() | ||||||
|  | 
 | ||||||
|  | 	if c.msc == nil { | ||||||
|  | 		return errShutdown | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	ctx, cancel := c.exportContext(ctx) | ||||||
|  | 	defer cancel() | ||||||
|  | 
 | ||||||
|  | 	return c.requestFunc(ctx, func(iCtx context.Context) error { | ||||||
|  | 		_, err := c.msc.Export(iCtx, &colmetricpb.ExportMetricsServiceRequest{ | ||||||
|  | 			ResourceMetrics: protoMetrics, | ||||||
|  | 		}) | ||||||
|  | 		// nil is converted to OK.
 | ||||||
|  | 		if status.Code(err) == codes.OK { | ||||||
|  | 			// Success.
 | ||||||
|  | 			return nil | ||||||
|  | 		} | ||||||
|  | 		return err | ||||||
|  | 	}) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // exportContext returns a copy of parent with an appropriate deadline and
 | ||||||
|  | // cancellation function.
 | ||||||
|  | //
 | ||||||
|  | // It is the callers responsibility to cancel the returned context once its
 | ||||||
|  | // use is complete, via the parent or directly with the returned CancelFunc, to
 | ||||||
|  | // ensure all resources are correctly released.
 | ||||||
|  | func (c *client) exportContext(parent context.Context) (context.Context, context.CancelFunc) { | ||||||
|  | 	var ( | ||||||
|  | 		ctx    context.Context | ||||||
|  | 		cancel context.CancelFunc | ||||||
|  | 	) | ||||||
|  | 
 | ||||||
|  | 	if c.exportTimeout > 0 { | ||||||
|  | 		ctx, cancel = context.WithTimeout(parent, c.exportTimeout) | ||||||
|  | 	} else { | ||||||
|  | 		ctx, cancel = context.WithCancel(parent) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	if c.metadata.Len() > 0 { | ||||||
|  | 		ctx = metadata.NewOutgoingContext(ctx, c.metadata) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// Unify the client stopCtx with the parent.
 | ||||||
|  | 	go func() { | ||||||
|  | 		select { | ||||||
|  | 		case <-ctx.Done(): | ||||||
|  | 		case <-c.stopCtx.Done(): | ||||||
|  | 			// Cancel the export as the shutdown has timed out.
 | ||||||
|  | 			cancel() | ||||||
|  | 		} | ||||||
|  | 	}() | ||||||
|  | 
 | ||||||
|  | 	return ctx, cancel | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // retryable returns if err identifies a request that can be retried and a
 | ||||||
|  | // duration to wait for if an explicit throttle time is included in err.
 | ||||||
|  | func retryable(err error) (bool, time.Duration) { | ||||||
|  | 	//func retryable(err error) (bool, time.Duration) {
 | ||||||
|  | 	s := status.Convert(err) | ||||||
|  | 	switch s.Code() { | ||||||
|  | 	case codes.Canceled, | ||||||
|  | 		codes.DeadlineExceeded, | ||||||
|  | 		codes.ResourceExhausted, | ||||||
|  | 		codes.Aborted, | ||||||
|  | 		codes.OutOfRange, | ||||||
|  | 		codes.Unavailable, | ||||||
|  | 		codes.DataLoss: | ||||||
|  | 		return true, throttleDelay(s) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	// Not a retry-able error.
 | ||||||
|  | 	return false, 0 | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // throttleDelay returns a duration to wait for if an explicit throttle time
 | ||||||
|  | // is included in the response status.
 | ||||||
|  | func throttleDelay(status *status.Status) time.Duration { | ||||||
|  | 	for _, detail := range status.Details() { | ||||||
|  | 		if t, ok := detail.(*errdetails.RetryInfo); ok { | ||||||
|  | 			return t.RetryDelay.AsDuration() | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return 0 | ||||||
|  | } | ||||||
|  |  | ||||||
|  | @ -24,12 +24,10 @@ import ( | ||||||
| 
 | 
 | ||||||
| 	"github.com/stretchr/testify/assert" | 	"github.com/stretchr/testify/assert" | ||||||
| 	"github.com/stretchr/testify/require" | 	"github.com/stretchr/testify/require" | ||||||
| 	"google.golang.org/genproto/googleapis/rpc/errdetails" |  | ||||||
| 	"google.golang.org/grpc" | 	"google.golang.org/grpc" | ||||||
| 	"google.golang.org/grpc/codes" | 	"google.golang.org/grpc/codes" | ||||||
| 	"google.golang.org/grpc/encoding/gzip" | 	"google.golang.org/grpc/encoding/gzip" | ||||||
| 	"google.golang.org/grpc/status" | 	"google.golang.org/grpc/status" | ||||||
| 	"google.golang.org/protobuf/types/known/durationpb" |  | ||||||
| 
 | 
 | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric" | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric" | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpmetrictest" | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpmetrictest" | ||||||
|  | @ -165,386 +163,6 @@ func TestNewExporter_invokeStartThenStopManyTimes(t *testing.T) { | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func TestNewExporter_collectorConnectionDiesThenReconnectsWhenInRestMode(t *testing.T) { |  | ||||||
| 	// TODO: Fix this test #1527
 |  | ||||||
| 	t.Skip("This test is flaky and needs to be rewritten") |  | ||||||
| 	mc := runMockCollector(t) |  | ||||||
| 
 |  | ||||||
| 	reconnectionPeriod := 20 * time.Millisecond |  | ||||||
| 	ctx := context.Background() |  | ||||||
| 	exp := newGRPCExporter(t, ctx, mc.endpoint, |  | ||||||
| 		otlpmetricgrpc.WithRetry(otlpmetricgrpc.RetrySettings{Enabled: false}), |  | ||||||
| 		otlpmetricgrpc.WithReconnectionPeriod(reconnectionPeriod)) |  | ||||||
| 	defer func() { require.NoError(t, exp.Shutdown(ctx)) }() |  | ||||||
| 
 |  | ||||||
| 	// Wait for a connection.
 |  | ||||||
| 	mc.ln.WaitForConn() |  | ||||||
| 
 |  | ||||||
| 	// We'll now stop the collector right away to simulate a connection
 |  | ||||||
| 	// dying in the midst of communication or even not existing before.
 |  | ||||||
| 	require.NoError(t, mc.stop()) |  | ||||||
| 
 |  | ||||||
| 	// first export, it will send disconnected message to the channel on export failure,
 |  | ||||||
| 	// trigger almost immediate reconnection
 |  | ||||||
| 	require.Error(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 
 |  | ||||||
| 	// second export, it will detect connection issue, change state of exporter to disconnected and
 |  | ||||||
| 	// send message to disconnected channel but this time reconnection gouroutine will be in (rest mode, not listening to the disconnected channel)
 |  | ||||||
| 	require.Error(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 
 |  | ||||||
| 	// as a result we have exporter in disconnected state waiting for disconnection message to reconnect
 |  | ||||||
| 
 |  | ||||||
| 	// resurrect collector
 |  | ||||||
| 	nmc := runMockCollectorAtEndpoint(t, mc.endpoint) |  | ||||||
| 
 |  | ||||||
| 	// make sure reconnection loop hits beginning and goes back to waiting mode
 |  | ||||||
| 	// after hitting beginning of the loop it should reconnect
 |  | ||||||
| 	nmc.ln.WaitForConn() |  | ||||||
| 
 |  | ||||||
| 	n := 10 |  | ||||||
| 	for i := 0; i < n; i++ { |  | ||||||
| 		// when disconnected exp.Export doesnt send disconnected messages again
 |  | ||||||
| 		// it just quits and return last connection error
 |  | ||||||
| 		require.NoError(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	nmaMetrics := nmc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 	if g, w := len(nmaMetrics), n; g != w { |  | ||||||
| 		t.Fatalf("Connected collector: metrics: got %d want %d", g, w) |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	dMetrics := mc.getMetrics() |  | ||||||
| 	// Expecting 0 metrics to have been received by the original but now dead collector
 |  | ||||||
| 	if g, w := len(dMetrics), 0; g != w { |  | ||||||
| 		t.Fatalf("Disconnected collector: spans: got %d want %d", g, w) |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	require.NoError(t, nmc.Stop()) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func TestExporterExportFailureAndRecoveryModes(t *testing.T) { |  | ||||||
| 	tts := []struct { |  | ||||||
| 		name   string |  | ||||||
| 		errors []error |  | ||||||
| 		rs     otlpmetricgrpc.RetrySettings |  | ||||||
| 		fn     func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) |  | ||||||
| 		opts   []otlpmetricgrpc.Option |  | ||||||
| 	}{ |  | ||||||
| 		{ |  | ||||||
| 			name: "Do not retry if succeeded", |  | ||||||
| 			fn: func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) { |  | ||||||
| 				require.NoError(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 
 |  | ||||||
| 				metrics := mc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 				require.Len(t, metrics, 1) |  | ||||||
| 				require.Equal(t, 1, mc.metricSvc.requests, "metric service must receive 1 success request.") |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			name: "Do not retry if 'error' is ok", |  | ||||||
| 			errors: []error{ |  | ||||||
| 				status.Error(codes.OK, ""), |  | ||||||
| 			}, |  | ||||||
| 			fn: func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) { |  | ||||||
| 				require.NoError(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 
 |  | ||||||
| 				metrics := mc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 				require.Len(t, metrics, 0) |  | ||||||
| 				require.Equal(t, 1, mc.metricSvc.requests, "metric service must receive 1 error OK request.") |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			name: "Fail three times and succeed", |  | ||||||
| 			rs: otlpmetricgrpc.RetrySettings{ |  | ||||||
| 				Enabled:         true, |  | ||||||
| 				MaxElapsedTime:  300 * time.Millisecond, |  | ||||||
| 				InitialInterval: 2 * time.Millisecond, |  | ||||||
| 				MaxInterval:     10 * time.Millisecond, |  | ||||||
| 			}, |  | ||||||
| 			errors: []error{ |  | ||||||
| 				status.Error(codes.Unavailable, "backend under pressure"), |  | ||||||
| 				status.Error(codes.Unavailable, "backend under pressure"), |  | ||||||
| 				status.Error(codes.Unavailable, "backend under pressure"), |  | ||||||
| 			}, |  | ||||||
| 			fn: func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) { |  | ||||||
| 				require.NoError(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 
 |  | ||||||
| 				metrics := mc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 				require.Len(t, metrics, 1) |  | ||||||
| 				require.Equal(t, 4, mc.metricSvc.requests, "metric service must receive 3 failure requests and 1 success request.") |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			name: "Permanent error should not be retried", |  | ||||||
| 			rs: otlpmetricgrpc.RetrySettings{ |  | ||||||
| 				Enabled:         true, |  | ||||||
| 				MaxElapsedTime:  300 * time.Millisecond, |  | ||||||
| 				InitialInterval: 2 * time.Millisecond, |  | ||||||
| 				MaxInterval:     10 * time.Millisecond, |  | ||||||
| 			}, |  | ||||||
| 			errors: []error{ |  | ||||||
| 				status.Error(codes.InvalidArgument, "invalid arguments"), |  | ||||||
| 			}, |  | ||||||
| 			fn: func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) { |  | ||||||
| 				require.Error(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 
 |  | ||||||
| 				metric := mc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 				require.Len(t, metric, 0) |  | ||||||
| 				require.Equal(t, 1, mc.metricSvc.requests, "metric service must receive 1 error requests.") |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			name: "Test all transient errors and succeed", |  | ||||||
| 			rs: otlpmetricgrpc.RetrySettings{ |  | ||||||
| 				Enabled:         true, |  | ||||||
| 				MaxElapsedTime:  500 * time.Millisecond, |  | ||||||
| 				InitialInterval: 1 * time.Millisecond, |  | ||||||
| 				MaxInterval:     2 * time.Millisecond, |  | ||||||
| 			}, |  | ||||||
| 			errors: []error{ |  | ||||||
| 				status.Error(codes.Canceled, ""), |  | ||||||
| 				status.Error(codes.DeadlineExceeded, ""), |  | ||||||
| 				status.Error(codes.ResourceExhausted, ""), |  | ||||||
| 				status.Error(codes.Aborted, ""), |  | ||||||
| 				status.Error(codes.OutOfRange, ""), |  | ||||||
| 				status.Error(codes.Unavailable, ""), |  | ||||||
| 				status.Error(codes.DataLoss, ""), |  | ||||||
| 			}, |  | ||||||
| 			fn: func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) { |  | ||||||
| 				require.NoError(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 
 |  | ||||||
| 				metrics := mc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 				require.Len(t, metrics, 1) |  | ||||||
| 				require.Equal(t, 8, mc.metricSvc.requests, "metric service must receive 9 failure requests and 1 success request.") |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			name: "Retry should honor server throttling", |  | ||||||
| 			rs: otlpmetricgrpc.RetrySettings{ |  | ||||||
| 				Enabled:         true, |  | ||||||
| 				MaxElapsedTime:  time.Minute, |  | ||||||
| 				InitialInterval: time.Nanosecond, |  | ||||||
| 				MaxInterval:     time.Nanosecond, |  | ||||||
| 			}, |  | ||||||
| 			opts: []otlpmetricgrpc.Option{ |  | ||||||
| 				otlpmetricgrpc.WithTimeout(time.Millisecond * 100), |  | ||||||
| 			}, |  | ||||||
| 			errors: []error{ |  | ||||||
| 				newThrottlingError(codes.ResourceExhausted, time.Second*30), |  | ||||||
| 			}, |  | ||||||
| 			fn: func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) { |  | ||||||
| 				err := exp.Export(ctx, testResource, oneRecord) |  | ||||||
| 				require.Error(t, err) |  | ||||||
| 				require.Equal(t, "context deadline exceeded", err.Error()) |  | ||||||
| 
 |  | ||||||
| 				metrics := mc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 				require.Len(t, metrics, 0) |  | ||||||
| 				require.Equal(t, 1, mc.metricSvc.requests, "metric service must receive 1 failure requests and 1 success request.") |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			name: "Retry should fail if server throttling is higher than the MaxElapsedTime", |  | ||||||
| 			rs: otlpmetricgrpc.RetrySettings{ |  | ||||||
| 				Enabled:         true, |  | ||||||
| 				MaxElapsedTime:  time.Millisecond * 100, |  | ||||||
| 				InitialInterval: time.Nanosecond, |  | ||||||
| 				MaxInterval:     time.Nanosecond, |  | ||||||
| 			}, |  | ||||||
| 			errors: []error{ |  | ||||||
| 				newThrottlingError(codes.ResourceExhausted, time.Minute), |  | ||||||
| 			}, |  | ||||||
| 			fn: func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) { |  | ||||||
| 				err := exp.Export(ctx, testResource, oneRecord) |  | ||||||
| 				require.Error(t, err) |  | ||||||
| 				require.Equal(t, "max elapsed time expired when respecting server throttle: rpc error: code = ResourceExhausted desc = ", err.Error()) |  | ||||||
| 
 |  | ||||||
| 				metrics := mc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 				require.Len(t, metrics, 0) |  | ||||||
| 				require.Equal(t, 1, mc.metricSvc.requests, "metric service must receive 1 failure requests and 1 success request.") |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			name: "Retry stops if takes too long", |  | ||||||
| 			rs: otlpmetricgrpc.RetrySettings{ |  | ||||||
| 				Enabled:         true, |  | ||||||
| 				MaxElapsedTime:  time.Millisecond * 100, |  | ||||||
| 				InitialInterval: time.Millisecond * 50, |  | ||||||
| 				MaxInterval:     time.Millisecond * 50, |  | ||||||
| 			}, |  | ||||||
| 			errors: []error{ |  | ||||||
| 				status.Error(codes.Unavailable, "unavailable"), |  | ||||||
| 				status.Error(codes.Unavailable, "unavailable"), |  | ||||||
| 				status.Error(codes.Unavailable, "unavailable"), |  | ||||||
| 				status.Error(codes.Unavailable, "unavailable"), |  | ||||||
| 				status.Error(codes.Unavailable, "unavailable"), |  | ||||||
| 				status.Error(codes.Unavailable, "unavailable"), |  | ||||||
| 			}, |  | ||||||
| 			fn: func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) { |  | ||||||
| 				err := exp.Export(ctx, testResource, oneRecord) |  | ||||||
| 				require.Error(t, err) |  | ||||||
| 
 |  | ||||||
| 				require.Equal(t, "max elapsed time expired: rpc error: code = Unavailable desc = unavailable", err.Error()) |  | ||||||
| 
 |  | ||||||
| 				metrics := mc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 				require.Len(t, metrics, 0) |  | ||||||
| 				require.LessOrEqual(t, 1, mc.metricSvc.requests, "metric service must receive at least 1 failure requests.") |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			name: "Disabled retry", |  | ||||||
| 			rs: otlpmetricgrpc.RetrySettings{ |  | ||||||
| 				Enabled: false, |  | ||||||
| 			}, |  | ||||||
| 			errors: []error{ |  | ||||||
| 				status.Error(codes.Unavailable, "unavailable"), |  | ||||||
| 			}, |  | ||||||
| 			fn: func(t *testing.T, ctx context.Context, exp *otlpmetric.Exporter, mc *mockCollector) { |  | ||||||
| 				err := exp.Export(ctx, testResource, oneRecord) |  | ||||||
| 				require.Error(t, err) |  | ||||||
| 
 |  | ||||||
| 				require.Equal(t, "rpc error: code = Unavailable desc = unavailable", err.Error()) |  | ||||||
| 
 |  | ||||||
| 				metrics := mc.getMetrics() |  | ||||||
| 
 |  | ||||||
| 				require.Len(t, metrics, 0) |  | ||||||
| 				require.Equal(t, 1, mc.metricSvc.requests, "metric service must receive 1 failure requests.") |  | ||||||
| 			}, |  | ||||||
| 		}, |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	for _, tt := range tts { |  | ||||||
| 		t.Run(tt.name, func(t *testing.T) { |  | ||||||
| 			ctx := context.Background() |  | ||||||
| 
 |  | ||||||
| 			mc := runMockCollectorWithConfig(t, &mockConfig{ |  | ||||||
| 				errors: tt.errors, |  | ||||||
| 			}) |  | ||||||
| 
 |  | ||||||
| 			opts := []otlpmetricgrpc.Option{ |  | ||||||
| 				otlpmetricgrpc.WithRetry(tt.rs), |  | ||||||
| 			} |  | ||||||
| 
 |  | ||||||
| 			if len(tt.opts) != 0 { |  | ||||||
| 				opts = append(opts, tt.opts...) |  | ||||||
| 			} |  | ||||||
| 
 |  | ||||||
| 			exp := newGRPCExporter(t, ctx, mc.endpoint, opts...) |  | ||||||
| 
 |  | ||||||
| 			tt.fn(t, ctx, exp, mc) |  | ||||||
| 
 |  | ||||||
| 			require.NoError(t, mc.Stop()) |  | ||||||
| 			require.NoError(t, exp.Shutdown(ctx)) |  | ||||||
| 		}) |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func TestPermanentErrorsShouldNotBeRetried(t *testing.T) { |  | ||||||
| 	permanentErrors := []*status.Status{ |  | ||||||
| 		status.New(codes.Unknown, "Unknown"), |  | ||||||
| 		status.New(codes.InvalidArgument, "InvalidArgument"), |  | ||||||
| 		status.New(codes.NotFound, "NotFound"), |  | ||||||
| 		status.New(codes.AlreadyExists, "AlreadyExists"), |  | ||||||
| 		status.New(codes.FailedPrecondition, "FailedPrecondition"), |  | ||||||
| 		status.New(codes.Unimplemented, "Unimplemented"), |  | ||||||
| 		status.New(codes.Internal, "Internal"), |  | ||||||
| 		status.New(codes.PermissionDenied, ""), |  | ||||||
| 		status.New(codes.Unauthenticated, ""), |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	for _, sts := range permanentErrors { |  | ||||||
| 		t.Run(sts.Code().String(), func(t *testing.T) { |  | ||||||
| 			ctx := context.Background() |  | ||||||
| 
 |  | ||||||
| 			mc := runMockCollectorWithConfig(t, &mockConfig{ |  | ||||||
| 				errors: []error{sts.Err()}, |  | ||||||
| 			}) |  | ||||||
| 
 |  | ||||||
| 			exp := newGRPCExporter(t, ctx, mc.endpoint) |  | ||||||
| 
 |  | ||||||
| 			err := exp.Export(ctx, testResource, oneRecord) |  | ||||||
| 			require.Error(t, err) |  | ||||||
| 			require.Len(t, mc.getMetrics(), 0) |  | ||||||
| 			require.Equal(t, 1, mc.metricSvc.requests, "metric service must receive 1 permanent error requests.") |  | ||||||
| 
 |  | ||||||
| 			require.NoError(t, mc.Stop()) |  | ||||||
| 			require.NoError(t, exp.Shutdown(ctx)) |  | ||||||
| 		}) |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func newThrottlingError(code codes.Code, duration time.Duration) error { |  | ||||||
| 	s := status.New(code, "") |  | ||||||
| 
 |  | ||||||
| 	s, _ = s.WithDetails(&errdetails.RetryInfo{RetryDelay: durationpb.New(duration)}) |  | ||||||
| 
 |  | ||||||
| 	return s.Err() |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func TestNewExporter_collectorConnectionDiesThenReconnects(t *testing.T) { |  | ||||||
| 	// TODO: Fix this test #1527
 |  | ||||||
| 	t.Skip("This test is flaky and needs to be rewritten") |  | ||||||
| 	mc := runMockCollector(t) |  | ||||||
| 
 |  | ||||||
| 	reconnectionPeriod := 50 * time.Millisecond |  | ||||||
| 	ctx := context.Background() |  | ||||||
| 	exp := newGRPCExporter(t, ctx, mc.endpoint, |  | ||||||
| 		otlpmetricgrpc.WithRetry(otlpmetricgrpc.RetrySettings{Enabled: false}), |  | ||||||
| 		otlpmetricgrpc.WithReconnectionPeriod(reconnectionPeriod)) |  | ||||||
| 	defer func() { require.NoError(t, exp.Shutdown(ctx)) }() |  | ||||||
| 
 |  | ||||||
| 	mc.ln.WaitForConn() |  | ||||||
| 
 |  | ||||||
| 	// We'll now stop the collector right away to simulate a connection
 |  | ||||||
| 	// dying in the midst of communication or even not existing before.
 |  | ||||||
| 	require.NoError(t, mc.stop()) |  | ||||||
| 
 |  | ||||||
| 	// In the test below, we'll stop the collector many times,
 |  | ||||||
| 	// while exporting metrics and test to ensure that we can
 |  | ||||||
| 	// reconnect.
 |  | ||||||
| 	for j := 0; j < 3; j++ { |  | ||||||
| 
 |  | ||||||
| 		// No endpoint up.
 |  | ||||||
| 		require.Error(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 
 |  | ||||||
| 		// Now resurrect the collector by making a new one but reusing the
 |  | ||||||
| 		// old endpoint, and the collector should reconnect automatically.
 |  | ||||||
| 		nmc := runMockCollectorAtEndpoint(t, mc.endpoint) |  | ||||||
| 
 |  | ||||||
| 		// Give the exporter sometime to reconnect
 |  | ||||||
| 		nmc.ln.WaitForConn() |  | ||||||
| 
 |  | ||||||
| 		n := 10 |  | ||||||
| 		for i := 0; i < n; i++ { |  | ||||||
| 			require.NoError(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		nmaMetrics := nmc.getMetrics() |  | ||||||
| 		// Expecting 10 metrics that were sampled, given that
 |  | ||||||
| 		if g, w := len(nmaMetrics), n; g != w { |  | ||||||
| 			t.Fatalf("Round #%d: Connected collector: spans: got %d want %d", j, g, w) |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		dMetrics := mc.getMetrics() |  | ||||||
| 		// Expecting 0 metrics to have been received by the original but now dead collector
 |  | ||||||
| 		if g, w := len(dMetrics), 0; g != w { |  | ||||||
| 			t.Fatalf("Round #%d: Disconnected collector: spans: got %d want %d", j, g, w) |  | ||||||
| 		} |  | ||||||
| 
 |  | ||||||
| 		// Disconnect for the next try.
 |  | ||||||
| 		require.NoError(t, nmc.stop()) |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| // This test takes a long time to run: to skip it, run tests using: -short
 | // This test takes a long time to run: to skip it, run tests using: -short
 | ||||||
| func TestNewExporter_collectorOnBadConnection(t *testing.T) { | func TestNewExporter_collectorOnBadConnection(t *testing.T) { | ||||||
| 	if testing.Short() { | 	if testing.Short() { | ||||||
|  | @ -641,7 +259,7 @@ func TestNewExporter_WithTimeout(t *testing.T) { | ||||||
| 			}() | 			}() | ||||||
| 
 | 
 | ||||||
| 			ctx := context.Background() | 			ctx := context.Background() | ||||||
| 			exp := newGRPCExporter(t, ctx, mc.endpoint, otlpmetricgrpc.WithTimeout(tt.timeout), otlpmetricgrpc.WithRetry(otlpmetricgrpc.RetrySettings{Enabled: false})) | 			exp := newGRPCExporter(t, ctx, mc.endpoint, otlpmetricgrpc.WithTimeout(tt.timeout), otlpmetricgrpc.WithRetry(otlpmetricgrpc.RetryConfig{Enabled: false})) | ||||||
| 			defer func() { | 			defer func() { | ||||||
| 				_ = exp.Shutdown(ctx) | 				_ = exp.Shutdown(ctx) | ||||||
| 			}() | 			}() | ||||||
|  | @ -662,48 +280,31 @@ func TestNewExporter_WithTimeout(t *testing.T) { | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func TestNewExporter_withInvalidSecurityConfiguration(t *testing.T) { | func TestStartErrorInvalidSecurityConfiguration(t *testing.T) { | ||||||
| 	mc := runMockCollector(t) | 	mc := runMockCollector(t) | ||||||
| 	defer func() { | 	defer func() { | ||||||
| 		_ = mc.stop() | 		_ = mc.stop() | ||||||
| 	}() | 	}() | ||||||
| 
 | 
 | ||||||
| 	ctx := context.Background() |  | ||||||
| 	client := otlpmetricgrpc.NewClient(otlpmetricgrpc.WithEndpoint(mc.endpoint)) | 	client := otlpmetricgrpc.NewClient(otlpmetricgrpc.WithEndpoint(mc.endpoint)) | ||||||
| 	exp, err := otlpmetric.New(ctx, client) | 	err := client.Start(context.Background()) | ||||||
| 	if err != nil { | 	// https://github.com/grpc/grpc-go/blob/a671967dfbaab779d37fd7e597d9248f13806087/clientconn.go#L82
 | ||||||
| 		t.Fatalf("failed to create a new collector exporter: %v", err) | 	assert.EqualError(t, err, "grpc: no transport security set (use grpc.WithInsecure() explicitly or set credentials)") | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	err = exp.Export(ctx, testResource, oneRecord) |  | ||||||
| 
 |  | ||||||
| 	expectedErr := fmt.Sprintf("metrics exporter is disconnected from the server %s: grpc: no transport security set (use grpc.WithInsecure() explicitly or set credentials)", mc.endpoint) |  | ||||||
| 
 |  | ||||||
| 	require.Error(t, err) |  | ||||||
| 	require.Equal(t, expectedErr, err.Error()) |  | ||||||
| 
 |  | ||||||
| 	defer func() { |  | ||||||
| 		_ = exp.Shutdown(ctx) |  | ||||||
| 	}() |  | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func TestDisconnected(t *testing.T) { | func TestStartErrorInvalidAddress(t *testing.T) { | ||||||
| 	ctx := context.Background() | 	client := otlpmetricgrpc.NewClient( | ||||||
| 	// The endpoint is whatever, we want to be disconnected. But we
 | 		otlpmetricgrpc.WithInsecure(), | ||||||
| 	// setting a blocking connection, so dialing to the invalid
 | 		// Validate the connection in Start (which should return the error).
 | ||||||
| 	// endpoint actually fails.
 |  | ||||||
| 	exp := newGRPCExporter(t, ctx, "invalid", |  | ||||||
| 		otlpmetricgrpc.WithReconnectionPeriod(time.Hour), |  | ||||||
| 		otlpmetricgrpc.WithDialOption( | 		otlpmetricgrpc.WithDialOption( | ||||||
| 			grpc.WithBlock(), | 			grpc.WithBlock(), | ||||||
| 			grpc.FailOnNonTempDialError(true), | 			grpc.FailOnNonTempDialError(true), | ||||||
| 		), | 		), | ||||||
|  | 		otlpmetricgrpc.WithEndpoint("invalid"), | ||||||
|  | 		otlpmetricgrpc.WithReconnectionPeriod(time.Hour), | ||||||
| 	) | 	) | ||||||
| 	defer func() { | 	err := client.Start(context.Background()) | ||||||
| 		assert.NoError(t, exp.Shutdown(ctx)) | 	assert.EqualError(t, err, `connection error: desc = "transport: error while dialing: dial tcp: address invalid: missing port in address"`) | ||||||
| 	}() |  | ||||||
| 
 |  | ||||||
| 	assert.Error(t, exp.Export(ctx, testResource, oneRecord)) |  | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func TestEmptyData(t *testing.T) { | func TestEmptyData(t *testing.T) { | ||||||
|  |  | ||||||
|  | @ -0,0 +1,193 @@ | ||||||
|  | // Copyright The OpenTelemetry Authors
 | ||||||
|  | //
 | ||||||
|  | // Licensed under the Apache License, Version 2.0 (the "License");
 | ||||||
|  | // you may not use this file except in compliance with the License.
 | ||||||
|  | // You may obtain a copy of the License at
 | ||||||
|  | //
 | ||||||
|  | //     http://www.apache.org/licenses/LICENSE-2.0
 | ||||||
|  | //
 | ||||||
|  | // Unless required by applicable law or agreed to in writing, software
 | ||||||
|  | // distributed under the License is distributed on an "AS IS" BASIS,
 | ||||||
|  | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 | ||||||
|  | // See the License for the specific language governing permissions and
 | ||||||
|  | // limitations under the License.
 | ||||||
|  | 
 | ||||||
|  | package otlpmetricgrpc | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"context" | ||||||
|  | 	"testing" | ||||||
|  | 	"time" | ||||||
|  | 
 | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | 	"github.com/stretchr/testify/require" | ||||||
|  | 	"google.golang.org/genproto/googleapis/rpc/errdetails" | ||||||
|  | 	"google.golang.org/grpc/codes" | ||||||
|  | 	"google.golang.org/grpc/status" | ||||||
|  | 	"google.golang.org/protobuf/types/known/durationpb" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | func TestThrottleDuration(t *testing.T) { | ||||||
|  | 	c := codes.ResourceExhausted | ||||||
|  | 	testcases := []struct { | ||||||
|  | 		status   *status.Status | ||||||
|  | 		expected time.Duration | ||||||
|  | 	}{ | ||||||
|  | 		{ | ||||||
|  | 			status:   status.New(c, "no retry info"), | ||||||
|  | 			expected: 0, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			status: func() *status.Status { | ||||||
|  | 				s, err := status.New(c, "single retry info").WithDetails( | ||||||
|  | 					&errdetails.RetryInfo{ | ||||||
|  | 						RetryDelay: durationpb.New(15 * time.Millisecond), | ||||||
|  | 					}, | ||||||
|  | 				) | ||||||
|  | 				require.NoError(t, err) | ||||||
|  | 				return s | ||||||
|  | 			}(), | ||||||
|  | 			expected: 15 * time.Millisecond, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			status: func() *status.Status { | ||||||
|  | 				s, err := status.New(c, "error info").WithDetails( | ||||||
|  | 					&errdetails.ErrorInfo{Reason: "no throttle detail"}, | ||||||
|  | 				) | ||||||
|  | 				require.NoError(t, err) | ||||||
|  | 				return s | ||||||
|  | 			}(), | ||||||
|  | 			expected: 0, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			status: func() *status.Status { | ||||||
|  | 				s, err := status.New(c, "error and retry info").WithDetails( | ||||||
|  | 					&errdetails.ErrorInfo{Reason: "with throttle detail"}, | ||||||
|  | 					&errdetails.RetryInfo{ | ||||||
|  | 						RetryDelay: durationpb.New(13 * time.Minute), | ||||||
|  | 					}, | ||||||
|  | 				) | ||||||
|  | 				require.NoError(t, err) | ||||||
|  | 				return s | ||||||
|  | 			}(), | ||||||
|  | 			expected: 13 * time.Minute, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			status: func() *status.Status { | ||||||
|  | 				s, err := status.New(c, "double retry info").WithDetails( | ||||||
|  | 					&errdetails.RetryInfo{ | ||||||
|  | 						RetryDelay: durationpb.New(13 * time.Minute), | ||||||
|  | 					}, | ||||||
|  | 					&errdetails.RetryInfo{ | ||||||
|  | 						RetryDelay: durationpb.New(15 * time.Minute), | ||||||
|  | 					}, | ||||||
|  | 				) | ||||||
|  | 				require.NoError(t, err) | ||||||
|  | 				return s | ||||||
|  | 			}(), | ||||||
|  | 			expected: 13 * time.Minute, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for _, tc := range testcases { | ||||||
|  | 		t.Run(tc.status.Message(), func(t *testing.T) { | ||||||
|  | 			require.Equal(t, tc.expected, throttleDelay(tc.status)) | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestRetryable(t *testing.T) { | ||||||
|  | 	retryableCodes := map[codes.Code]bool{ | ||||||
|  | 		codes.OK:                 false, | ||||||
|  | 		codes.Canceled:           true, | ||||||
|  | 		codes.Unknown:            false, | ||||||
|  | 		codes.InvalidArgument:    false, | ||||||
|  | 		codes.DeadlineExceeded:   true, | ||||||
|  | 		codes.NotFound:           false, | ||||||
|  | 		codes.AlreadyExists:      false, | ||||||
|  | 		codes.PermissionDenied:   false, | ||||||
|  | 		codes.ResourceExhausted:  true, | ||||||
|  | 		codes.FailedPrecondition: false, | ||||||
|  | 		codes.Aborted:            true, | ||||||
|  | 		codes.OutOfRange:         true, | ||||||
|  | 		codes.Unimplemented:      false, | ||||||
|  | 		codes.Internal:           false, | ||||||
|  | 		codes.Unavailable:        true, | ||||||
|  | 		codes.DataLoss:           true, | ||||||
|  | 		codes.Unauthenticated:    false, | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for c, want := range retryableCodes { | ||||||
|  | 		got, _ := retryable(status.Error(c, "")) | ||||||
|  | 		assert.Equalf(t, want, got, "evaluate(%s)", c) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestUnstartedStop(t *testing.T) { | ||||||
|  | 	client := NewClient() | ||||||
|  | 	assert.ErrorIs(t, client.Stop(context.Background()), errAlreadyStopped) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestUnstartedUploadMetric(t *testing.T) { | ||||||
|  | 	client := NewClient() | ||||||
|  | 	assert.ErrorIs(t, client.UploadMetrics(context.Background(), nil), errShutdown) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestExportContextHonorsParentDeadline(t *testing.T) { | ||||||
|  | 	now := time.Now() | ||||||
|  | 	ctx, cancel := context.WithDeadline(context.Background(), now) | ||||||
|  | 	t.Cleanup(cancel) | ||||||
|  | 
 | ||||||
|  | 	// Without a client timeout, the parent deadline should be used.
 | ||||||
|  | 	client := newClient(WithTimeout(0)) | ||||||
|  | 	eCtx, eCancel := client.exportContext(ctx) | ||||||
|  | 	t.Cleanup(eCancel) | ||||||
|  | 
 | ||||||
|  | 	deadline, ok := eCtx.Deadline() | ||||||
|  | 	assert.True(t, ok, "deadline not propagated to child context") | ||||||
|  | 	assert.Equal(t, now, deadline) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestExportContextHonorsClientTimeout(t *testing.T) { | ||||||
|  | 	// Setting a timeout should ensure a deadline is set on the context.
 | ||||||
|  | 	client := newClient(WithTimeout(1 * time.Second)) | ||||||
|  | 	ctx, cancel := client.exportContext(context.Background()) | ||||||
|  | 	t.Cleanup(cancel) | ||||||
|  | 
 | ||||||
|  | 	_, ok := ctx.Deadline() | ||||||
|  | 	assert.True(t, ok, "timeout not set as deadline for child context") | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestExportContextLinksStopSignal(t *testing.T) { | ||||||
|  | 	rootCtx := context.Background() | ||||||
|  | 
 | ||||||
|  | 	client := newClient(WithInsecure()) | ||||||
|  | 	t.Cleanup(func() { require.NoError(t, client.Stop(rootCtx)) }) | ||||||
|  | 	require.NoError(t, client.Start(rootCtx)) | ||||||
|  | 
 | ||||||
|  | 	ctx, cancel := client.exportContext(rootCtx) | ||||||
|  | 	t.Cleanup(cancel) | ||||||
|  | 
 | ||||||
|  | 	require.False(t, func() bool { | ||||||
|  | 		select { | ||||||
|  | 		case <-ctx.Done(): | ||||||
|  | 			return true | ||||||
|  | 		default: | ||||||
|  | 		} | ||||||
|  | 		return false | ||||||
|  | 	}(), "context should not be done prior to canceling it") | ||||||
|  | 
 | ||||||
|  | 	// The client.stopFunc cancels the client.stopCtx. This should have been
 | ||||||
|  | 	// setup as a parent of ctx. Therefore, it should cancel ctx as well.
 | ||||||
|  | 	client.stopFunc() | ||||||
|  | 
 | ||||||
|  | 	// Assert this with Eventually to account for goroutine scheduler timing.
 | ||||||
|  | 	assert.Eventually(t, func() bool { | ||||||
|  | 		select { | ||||||
|  | 		case <-ctx.Done(): | ||||||
|  | 			return true | ||||||
|  | 		default: | ||||||
|  | 		} | ||||||
|  | 		return false | ||||||
|  | 	}, 10*time.Second, time.Microsecond) | ||||||
|  | } | ||||||
|  | @ -18,8 +18,6 @@ import ( | ||||||
| 	"context" | 	"context" | ||||||
| 	"fmt" | 	"fmt" | ||||||
| 	"net" | 	"net" | ||||||
| 	"runtime" |  | ||||||
| 	"strings" |  | ||||||
| 	"sync" | 	"sync" | ||||||
| 	"testing" | 	"testing" | ||||||
| 	"time" | 	"time" | ||||||
|  | @ -94,7 +92,6 @@ type mockCollector struct { | ||||||
| 	metricSvc *mockMetricService | 	metricSvc *mockMetricService | ||||||
| 
 | 
 | ||||||
| 	endpoint string | 	endpoint string | ||||||
| 	ln       *listener |  | ||||||
| 	stopFunc func() | 	stopFunc func() | ||||||
| 	stopOnce sync.Once | 	stopOnce sync.Once | ||||||
| } | } | ||||||
|  | @ -160,9 +157,8 @@ func runMockCollectorWithConfig(t *testing.T, mockConfig *mockConfig) *mockColle | ||||||
| 	srv := grpc.NewServer() | 	srv := grpc.NewServer() | ||||||
| 	mc := makeMockCollector(t, mockConfig) | 	mc := makeMockCollector(t, mockConfig) | ||||||
| 	collectormetricpb.RegisterMetricsServiceServer(srv, mc.metricSvc) | 	collectormetricpb.RegisterMetricsServiceServer(srv, mc.metricSvc) | ||||||
| 	mc.ln = newListener(ln) |  | ||||||
| 	go func() { | 	go func() { | ||||||
| 		_ = srv.Serve((net.Listener)(mc.ln)) | 		_ = srv.Serve(ln) | ||||||
| 	}() | 	}() | ||||||
| 
 | 
 | ||||||
| 	mc.endpoint = ln.Addr().String() | 	mc.endpoint = ln.Addr().String() | ||||||
|  | @ -171,59 +167,3 @@ func runMockCollectorWithConfig(t *testing.T, mockConfig *mockConfig) *mockColle | ||||||
| 
 | 
 | ||||||
| 	return mc | 	return mc | ||||||
| } | } | ||||||
| 
 |  | ||||||
| type listener struct { |  | ||||||
| 	closeOnce sync.Once |  | ||||||
| 	wrapped   net.Listener |  | ||||||
| 	C         chan struct{} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func newListener(wrapped net.Listener) *listener { |  | ||||||
| 	return &listener{ |  | ||||||
| 		wrapped: wrapped, |  | ||||||
| 		C:       make(chan struct{}, 1), |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (l *listener) Close() error { return l.wrapped.Close() } |  | ||||||
| 
 |  | ||||||
| func (l *listener) Addr() net.Addr { return l.wrapped.Addr() } |  | ||||||
| 
 |  | ||||||
| // Accept waits for and returns the next connection to the listener. It will
 |  | ||||||
| // send a signal on l.C that a connection has been made before returning.
 |  | ||||||
| func (l *listener) Accept() (net.Conn, error) { |  | ||||||
| 	conn, err := l.wrapped.Accept() |  | ||||||
| 	if err != nil { |  | ||||||
| 		// Go 1.16 exported net.ErrClosed that could clean up this check, but to
 |  | ||||||
| 		// remain backwards compatible with previous versions of Go that we
 |  | ||||||
| 		// support the following string evaluation is used instead to keep in line
 |  | ||||||
| 		// with the previously recommended way to check this:
 |  | ||||||
| 		// https://github.com/golang/go/issues/4373#issuecomment-353076799
 |  | ||||||
| 		if strings.Contains(err.Error(), "use of closed network connection") { |  | ||||||
| 			// If the listener has been closed, do not allow callers of
 |  | ||||||
| 			// WaitForConn to wait for a connection that will never come.
 |  | ||||||
| 			l.closeOnce.Do(func() { close(l.C) }) |  | ||||||
| 		} |  | ||||||
| 		return conn, err |  | ||||||
| 	} |  | ||||||
| 
 |  | ||||||
| 	select { |  | ||||||
| 	case l.C <- struct{}{}: |  | ||||||
| 	default: |  | ||||||
| 		// If C is full, assume nobody is listening and move on.
 |  | ||||||
| 	} |  | ||||||
| 	return conn, nil |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| // WaitForConn will wait indefintely for a connection to be estabilished with
 |  | ||||||
| // the listener before returning.
 |  | ||||||
| func (l *listener) WaitForConn() { |  | ||||||
| 	for { |  | ||||||
| 		select { |  | ||||||
| 		case <-l.C: |  | ||||||
| 			return |  | ||||||
| 		default: |  | ||||||
| 			runtime.Gosched() |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  |  | ||||||
|  | @ -23,16 +23,28 @@ import ( | ||||||
| 
 | 
 | ||||||
| 	"go.opentelemetry.io/otel" | 	"go.opentelemetry.io/otel" | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpconfig" | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpconfig" | ||||||
|  | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/retry" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| // Option applies an option to the gRPC client.
 | // Option applies an option to the gRPC driver.
 | ||||||
| type Option interface { | type Option interface { | ||||||
| 	applyGRPCOption(*otlpconfig.Config) | 	applyGRPCOption(*otlpconfig.Config) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // RetrySettings defines configuration for retrying batches in case of export failure
 | func asGRPCOptions(opts []Option) []otlpconfig.GRPCOption { | ||||||
| // using an exponential backoff.
 | 	converted := make([]otlpconfig.GRPCOption, len(opts)) | ||||||
| type RetrySettings otlpconfig.RetrySettings | 	for i, o := range opts { | ||||||
|  | 		converted[i] = otlpconfig.NewGRPCOption(o.applyGRPCOption) | ||||||
|  | 	} | ||||||
|  | 	return converted | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // RetryConfig defines configuration for retrying export of span batches that
 | ||||||
|  | // failed to be received by the target endpoint.
 | ||||||
|  | //
 | ||||||
|  | // This configuration does not define any network retry strategy. That is
 | ||||||
|  | // entirely handled by the gRPC ClientConn.
 | ||||||
|  | type RetryConfig retry.Config | ||||||
| 
 | 
 | ||||||
| type wrappedOption struct { | type wrappedOption struct { | ||||||
| 	otlpconfig.GRPCOption | 	otlpconfig.GRPCOption | ||||||
|  | @ -42,22 +54,28 @@ func (w wrappedOption) applyGRPCOption(cfg *otlpconfig.Config) { | ||||||
| 	w.ApplyGRPCOption(cfg) | 	w.ApplyGRPCOption(cfg) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithInsecure disables client transport security for the exporter's gRPC connection
 | // WithInsecure disables client transport security for the exporter's gRPC
 | ||||||
| // just like grpc.WithInsecure() https://pkg.go.dev/google.golang.org/grpc#WithInsecure
 | // connection just like grpc.WithInsecure()
 | ||||||
| // does. Note, by default, client security is required unless WithInsecure is used.
 | // (https://pkg.go.dev/google.golang.org/grpc#WithInsecure) does. Note, by
 | ||||||
|  | // default, client security is required unless WithInsecure is used.
 | ||||||
|  | //
 | ||||||
|  | // This option has no effect if WithGRPCConn is used.
 | ||||||
| func WithInsecure() Option { | func WithInsecure() Option { | ||||||
| 	return wrappedOption{otlpconfig.WithInsecure()} | 	return wrappedOption{otlpconfig.WithInsecure()} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithEndpoint allows one to set the endpoint that the exporter will
 | // WithEndpoint sets the target endpoint the exporter will connect to. If
 | ||||||
| // connect to the collector on. If unset, it will instead try to use
 | // unset, localhost:4317 will be used as a default.
 | ||||||
| // connect to DefaultCollectorHost:DefaultCollectorPort.
 | //
 | ||||||
|  | // This option has no effect if WithGRPCConn is used.
 | ||||||
| func WithEndpoint(endpoint string) Option { | func WithEndpoint(endpoint string) Option { | ||||||
| 	return wrappedOption{otlpconfig.WithEndpoint(endpoint)} | 	return wrappedOption{otlpconfig.WithEndpoint(endpoint)} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithReconnectionPeriod allows one to set the delay between next connection attempt
 | // WithReconnectionPeriod set the minimum amount of time between connection
 | ||||||
| // after failing to connect with the collector.
 | // attempts to the target endpoint.
 | ||||||
|  | //
 | ||||||
|  | // This option has no effect if WithGRPCConn is used.
 | ||||||
| func WithReconnectionPeriod(rp time.Duration) Option { | func WithReconnectionPeriod(rp time.Duration) Option { | ||||||
| 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | ||||||
| 		cfg.ReconnectionPeriod = rp | 		cfg.ReconnectionPeriod = rp | ||||||
|  | @ -74,25 +92,30 @@ func compressorToCompression(compressor string) otlpconfig.Compression { | ||||||
| 	return otlpconfig.NoCompression | 	return otlpconfig.NoCompression | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithCompressor will set the compressor for the gRPC client to use when sending requests.
 | // WithCompressor sets the compressor for the gRPC client to use when sending
 | ||||||
| // It is the responsibility of the caller to ensure that the compressor set has been registered
 | // requests. It is the responsibility of the caller to ensure that the
 | ||||||
| // with google.golang.org/grpc/encoding. This can be done by encoding.RegisterCompressor. Some
 | // compressor set has been registered with google.golang.org/grpc/encoding.
 | ||||||
| // compressors auto-register on import, such as gzip, which can be registered by calling
 | // This can be done by encoding.RegisterCompressor. Some compressors
 | ||||||
|  | // auto-register on import, such as gzip, which can be registered by calling
 | ||||||
| // `import _ "google.golang.org/grpc/encoding/gzip"`.
 | // `import _ "google.golang.org/grpc/encoding/gzip"`.
 | ||||||
|  | //
 | ||||||
|  | // This option has no effect if WithGRPCConn is used.
 | ||||||
| func WithCompressor(compressor string) Option { | func WithCompressor(compressor string) Option { | ||||||
| 	return wrappedOption{otlpconfig.WithCompression(compressorToCompression(compressor))} | 	return wrappedOption{otlpconfig.WithCompression(compressorToCompression(compressor))} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithHeaders will send the provided headers with gRPC requests.
 | // WithHeaders will send the provided headers with each gRPC requests.
 | ||||||
| func WithHeaders(headers map[string]string) Option { | func WithHeaders(headers map[string]string) Option { | ||||||
| 	return wrappedOption{otlpconfig.WithHeaders(headers)} | 	return wrappedOption{otlpconfig.WithHeaders(headers)} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithTLSCredentials allows the connection to use TLS credentials
 | // WithTLSCredentials allows the connection to use TLS credentials when
 | ||||||
| // when talking to the server. It takes in grpc.TransportCredentials instead
 | // talking to the server. It takes in grpc.TransportCredentials instead of say
 | ||||||
| // of say a Certificate file or a tls.Certificate, because the retrieving of
 | // a Certificate file or a tls.Certificate, because the retrieving of these
 | ||||||
| // these credentials can be done in many ways e.g. plain file, in code tls.Config
 | // credentials can be done in many ways e.g. plain file, in code tls.Config or
 | ||||||
| // or by certificate rotation, so it is up to the caller to decide what to use.
 | // by certificate rotation, so it is up to the caller to decide what to use.
 | ||||||
|  | //
 | ||||||
|  | // This option has no effect if WithGRPCConn is used.
 | ||||||
| func WithTLSCredentials(creds credentials.TransportCredentials) Option { | func WithTLSCredentials(creds credentials.TransportCredentials) Option { | ||||||
| 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | ||||||
| 		cfg.Metrics.GRPCCredentials = creds | 		cfg.Metrics.GRPCCredentials = creds | ||||||
|  | @ -100,40 +123,63 @@ func WithTLSCredentials(creds credentials.TransportCredentials) Option { | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithServiceConfig defines the default gRPC service config used.
 | // WithServiceConfig defines the default gRPC service config used.
 | ||||||
|  | //
 | ||||||
|  | // This option has no effect if WithGRPCConn is used.
 | ||||||
| func WithServiceConfig(serviceConfig string) Option { | func WithServiceConfig(serviceConfig string) Option { | ||||||
| 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | ||||||
| 		cfg.ServiceConfig = serviceConfig | 		cfg.ServiceConfig = serviceConfig | ||||||
| 	})} | 	})} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithDialOption opens support to any grpc.DialOption to be used. If it conflicts
 | // WithDialOption sets explicit grpc.DialOptions to use when making a
 | ||||||
| // with some other configuration the GRPC specified via the collector the ones here will
 | // connection. The options here are appended to the internal grpc.DialOptions
 | ||||||
| // take preference since they are set last.
 | // used so they will take precedence over any other internal grpc.DialOptions
 | ||||||
|  | // they might conflict with.
 | ||||||
|  | //
 | ||||||
|  | // This option has no effect if WithGRPCConn is used.
 | ||||||
| func WithDialOption(opts ...grpc.DialOption) Option { | func WithDialOption(opts ...grpc.DialOption) Option { | ||||||
| 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | ||||||
| 		cfg.DialOptions = opts | 		cfg.DialOptions = opts | ||||||
| 	})} | 	})} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithGRPCConn allows reusing existing gRPC connection when it has already been
 | // WithGRPCConn sets conn as the gRPC ClientConn used for all communication.
 | ||||||
| // established for other services. When set, other dial options will be ignored.
 | //
 | ||||||
|  | // This option takes precedence over any other option that relates to
 | ||||||
|  | // establishing or persisting a gRPC connection to a target endpoint. Any
 | ||||||
|  | // other option of those types passed will be ignored.
 | ||||||
|  | //
 | ||||||
|  | // It is the callers responsibility to close the passed conn. The client
 | ||||||
|  | // Shutdown method will not close this connection.
 | ||||||
| func WithGRPCConn(conn *grpc.ClientConn) Option { | func WithGRPCConn(conn *grpc.ClientConn) Option { | ||||||
| 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | 	return wrappedOption{otlpconfig.NewGRPCOption(func(cfg *otlpconfig.Config) { | ||||||
| 		cfg.GRPCConn = conn | 		cfg.GRPCConn = conn | ||||||
| 	})} | 	})} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithTimeout tells the client the max waiting time for the backend to process
 | // WithTimeout sets the max amount of time a client will attempt to export a
 | ||||||
| // each metrics batch. If unset, the default will be 10 seconds.
 | // batch of spans. This takes precedence over any retry settings defined with
 | ||||||
|  | // WithRetry, once this time limit has been reached the export is abandoned
 | ||||||
|  | // and the batch of spans is dropped.
 | ||||||
|  | //
 | ||||||
|  | // If unset, the default timeout will be set to 10 seconds.
 | ||||||
| func WithTimeout(duration time.Duration) Option { | func WithTimeout(duration time.Duration) Option { | ||||||
| 	return wrappedOption{otlpconfig.WithTimeout(duration)} | 	return wrappedOption{otlpconfig.WithTimeout(duration)} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithRetry configures the retry policy for transient errors that may occurs when
 | // WithRetry sets the retry policy for transient retryable errors that may be
 | ||||||
| // exporting metrics. An exponential back-off algorithm is used to
 | // returned by the target endpoint when exporting a batch of spans.
 | ||||||
| // ensure endpoints are not overwhelmed with retries. If unset, the default
 | //
 | ||||||
| // retry policy will retry after 5 seconds and increase exponentially after each
 | // If the target endpoint responds with not only a retryable error, but
 | ||||||
| // error for a total of 1 minute.
 | // explicitly returns a backoff time in the response. That time will take
 | ||||||
| func WithRetry(settings RetrySettings) Option { | // precedence over these settings.
 | ||||||
| 	return wrappedOption{otlpconfig.WithRetry(otlpconfig.RetrySettings(settings))} | //
 | ||||||
|  | // These settings do not define any network retry strategy. That is entirely
 | ||||||
|  | // handled by the gRPC ClientConn.
 | ||||||
|  | //
 | ||||||
|  | // If unset, the default retry policy will be used. It will retry the export
 | ||||||
|  | // 5 seconds after receiving a retryable error and increase exponentially
 | ||||||
|  | // after each error for no more than a total time of 1 minute.
 | ||||||
|  | func WithRetry(settings RetryConfig) Option { | ||||||
|  | 	return wrappedOption{otlpconfig.WithRetry(retry.Config(settings))} | ||||||
| } | } | ||||||
|  |  | ||||||
|  | @ -21,24 +21,32 @@ import ( | ||||||
| 	"fmt" | 	"fmt" | ||||||
| 	"io" | 	"io" | ||||||
| 	"io/ioutil" | 	"io/ioutil" | ||||||
| 	"math/rand" |  | ||||||
| 	"net" | 	"net" | ||||||
| 	"net/http" | 	"net/http" | ||||||
| 	"path" | 	"path" | ||||||
|  | 	"strconv" | ||||||
| 	"strings" | 	"strings" | ||||||
|  | 	"sync" | ||||||
| 	"time" | 	"time" | ||||||
| 
 | 
 | ||||||
| 	"google.golang.org/protobuf/proto" | 	"google.golang.org/protobuf/proto" | ||||||
| 
 | 
 | ||||||
| 	"go.opentelemetry.io/otel" |  | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric" | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric" | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpconfig" | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpconfig" | ||||||
|  | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/retry" | ||||||
| 	colmetricpb "go.opentelemetry.io/proto/otlp/collector/metrics/v1" | 	colmetricpb "go.opentelemetry.io/proto/otlp/collector/metrics/v1" | ||||||
| 	metricpb "go.opentelemetry.io/proto/otlp/metrics/v1" | 	metricpb "go.opentelemetry.io/proto/otlp/metrics/v1" | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| const contentTypeProto = "application/x-protobuf" | const contentTypeProto = "application/x-protobuf" | ||||||
| 
 | 
 | ||||||
|  | var gzPool = sync.Pool{ | ||||||
|  | 	New: func() interface{} { | ||||||
|  | 		w := gzip.NewWriter(ioutil.Discard) | ||||||
|  | 		return w | ||||||
|  | 	}, | ||||||
|  | } | ||||||
|  | 
 | ||||||
| // Keep it in sync with golang's DefaultTransport from net/http! We
 | // Keep it in sync with golang's DefaultTransport from net/http! We
 | ||||||
| // have our own copy to avoid handling a situation where the
 | // have our own copy to avoid handling a situation where the
 | ||||||
| // DefaultTransport is overwritten with some different implementation
 | // DefaultTransport is overwritten with some different implementation
 | ||||||
|  | @ -57,11 +65,13 @@ var ourTransport = &http.Transport{ | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| type client struct { | type client struct { | ||||||
| 	name       string | 	name        string | ||||||
| 	cfg        otlpconfig.SignalConfig | 	cfg         otlpconfig.SignalConfig | ||||||
| 	generalCfg otlpconfig.Config | 	generalCfg  otlpconfig.Config | ||||||
| 	client     *http.Client | 	requestFunc retry.RequestFunc | ||||||
| 	stopCh     chan struct{} | 	client      *http.Client | ||||||
|  | 	stopCh      chan struct{} | ||||||
|  | 	stopOnce    sync.Once | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // NewClient creates a new HTTP metric client.
 | // NewClient creates a new HTTP metric client.
 | ||||||
|  | @ -73,7 +83,7 @@ func NewClient(opts ...Option) otlpmetric.Client { | ||||||
| 	} | 	} | ||||||
| 
 | 
 | ||||||
| 	for pathPtr, defaultPath := range map[*string]string{ | 	for pathPtr, defaultPath := range map[*string]string{ | ||||||
| 		&cfg.Metrics.URLPath: defaultMetricsPath, | 		&cfg.Metrics.URLPath: otlpconfig.DefaultMetricsPath, | ||||||
| 	} { | 	} { | ||||||
| 		tmp := strings.TrimSpace(*pathPtr) | 		tmp := strings.TrimSpace(*pathPtr) | ||||||
| 		if tmp == "" { | 		if tmp == "" { | ||||||
|  | @ -86,15 +96,6 @@ func NewClient(opts ...Option) otlpmetric.Client { | ||||||
| 		} | 		} | ||||||
| 		*pathPtr = tmp | 		*pathPtr = tmp | ||||||
| 	} | 	} | ||||||
| 	if cfg.MaxAttempts <= 0 { |  | ||||||
| 		cfg.MaxAttempts = defaultMaxAttempts |  | ||||||
| 	} |  | ||||||
| 	if cfg.MaxAttempts > defaultMaxAttempts { |  | ||||||
| 		cfg.MaxAttempts = defaultMaxAttempts |  | ||||||
| 	} |  | ||||||
| 	if cfg.Backoff <= 0 { |  | ||||||
| 		cfg.Backoff = defaultBackoff |  | ||||||
| 	} |  | ||||||
| 
 | 
 | ||||||
| 	httpClient := &http.Client{ | 	httpClient := &http.Client{ | ||||||
| 		Transport: ourTransport, | 		Transport: ourTransport, | ||||||
|  | @ -108,11 +109,12 @@ func NewClient(opts ...Option) otlpmetric.Client { | ||||||
| 
 | 
 | ||||||
| 	stopCh := make(chan struct{}) | 	stopCh := make(chan struct{}) | ||||||
| 	return &client{ | 	return &client{ | ||||||
| 		name:       "metrics", | 		name:        "metrics", | ||||||
| 		cfg:        cfg.Metrics, | 		cfg:         cfg.Metrics, | ||||||
| 		generalCfg: cfg, | 		generalCfg:  cfg, | ||||||
| 		stopCh:     stopCh, | 		requestFunc: cfg.RetryConfig.RequestFunc(evaluate), | ||||||
| 		client:     httpClient, | 		stopCh:      stopCh, | ||||||
|  | 		client:      httpClient, | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | @ -129,7 +131,9 @@ func (d *client) Start(ctx context.Context) error { | ||||||
| 
 | 
 | ||||||
| // Stop shuts down the client and interrupt any in-flight request.
 | // Stop shuts down the client and interrupt any in-flight request.
 | ||||||
| func (d *client) Stop(ctx context.Context) error { | func (d *client) Stop(ctx context.Context) error { | ||||||
| 	close(d.stopCh) | 	d.stopOnce.Do(func() { | ||||||
|  | 		close(d.stopCh) | ||||||
|  | 	}) | ||||||
| 	select { | 	select { | ||||||
| 	case <-ctx.Done(): | 	case <-ctx.Done(): | ||||||
| 		return ctx.Err() | 		return ctx.Err() | ||||||
|  | @ -147,41 +151,150 @@ func (d *client) UploadMetrics(ctx context.Context, protoMetrics []*metricpb.Res | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		return err | 		return err | ||||||
| 	} | 	} | ||||||
| 	return d.send(ctx, rawRequest) |  | ||||||
| } |  | ||||||
| 
 | 
 | ||||||
| func (d *client) send(ctx context.Context, rawRequest []byte) error { | 	ctx, cancel := d.contextWithStop(ctx) | ||||||
| 	address := fmt.Sprintf("%s://%s%s", d.getScheme(), d.cfg.Endpoint, d.cfg.URLPath) |  | ||||||
| 	var cancel context.CancelFunc |  | ||||||
| 	ctx, cancel = d.contextWithStop(ctx) |  | ||||||
| 	defer cancel() | 	defer cancel() | ||||||
| 	for i := 0; i < d.generalCfg.MaxAttempts; i++ { | 
 | ||||||
| 		response, err := d.singleSend(ctx, rawRequest, address) | 	request, err := d.newRequest(rawRequest) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return err | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return d.requestFunc(ctx, func(ctx context.Context) error { | ||||||
|  | 		select { | ||||||
|  | 		case <-ctx.Done(): | ||||||
|  | 			return ctx.Err() | ||||||
|  | 		default: | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		request.reset(ctx) | ||||||
|  | 		resp, err := d.client.Do(request.Request) | ||||||
| 		if err != nil { | 		if err != nil { | ||||||
| 			return err | 			return err | ||||||
| 		} | 		} | ||||||
| 		// We don't care about the body, so try to read it
 | 
 | ||||||
| 		// into /dev/null and close it immediately. The
 | 		var rErr error | ||||||
| 		// reading part is to facilitate connection reuse.
 | 		switch resp.StatusCode { | ||||||
| 		_, _ = io.Copy(ioutil.Discard, response.Body) |  | ||||||
| 		_ = response.Body.Close() |  | ||||||
| 		switch response.StatusCode { |  | ||||||
| 		case http.StatusOK: | 		case http.StatusOK: | ||||||
| 			return nil | 			// Success, do not retry.
 | ||||||
| 		case http.StatusTooManyRequests: | 		case http.StatusTooManyRequests, | ||||||
| 			fallthrough | 			http.StatusServiceUnavailable: | ||||||
| 		case http.StatusServiceUnavailable: | 			// Retry-able failure.
 | ||||||
| 			select { | 			rErr = newResponseError(resp.Header) | ||||||
| 			case <-time.After(getWaitDuration(d.generalCfg.Backoff, i)): | 
 | ||||||
| 				continue | 			// Going to retry, drain the body to reuse the connection.
 | ||||||
| 			case <-ctx.Done(): | 			if _, err := io.Copy(ioutil.Discard, resp.Body); err != nil { | ||||||
| 				return ctx.Err() | 				_ = resp.Body.Close() | ||||||
|  | 				return err | ||||||
| 			} | 			} | ||||||
| 		default: | 		default: | ||||||
| 			return fmt.Errorf("failed to send %s to %s with HTTP status %s", d.name, address, response.Status) | 			rErr = fmt.Errorf("failed to send %s to %s: %s", d.name, request.URL, resp.Status) | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		if err := resp.Body.Close(); err != nil { | ||||||
|  | 			return err | ||||||
|  | 		} | ||||||
|  | 		return rErr | ||||||
|  | 	}) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (d *client) newRequest(body []byte) (request, error) { | ||||||
|  | 	address := fmt.Sprintf("%s://%s%s", d.getScheme(), d.cfg.Endpoint, d.cfg.URLPath) | ||||||
|  | 	r, err := http.NewRequest(http.MethodPost, address, nil) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return request{Request: r}, err | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	for k, v := range d.cfg.Headers { | ||||||
|  | 		r.Header.Set(k, v) | ||||||
|  | 	} | ||||||
|  | 	r.Header.Set("Content-Type", contentTypeProto) | ||||||
|  | 
 | ||||||
|  | 	req := request{Request: r} | ||||||
|  | 	switch Compression(d.cfg.Compression) { | ||||||
|  | 	case NoCompression: | ||||||
|  | 		r.ContentLength = (int64)(len(body)) | ||||||
|  | 		req.bodyReader = bodyReader(body) | ||||||
|  | 	case GzipCompression: | ||||||
|  | 		// Ensure the content length is not used.
 | ||||||
|  | 		r.ContentLength = -1 | ||||||
|  | 		r.Header.Set("Content-Encoding", "gzip") | ||||||
|  | 
 | ||||||
|  | 		gz := gzPool.Get().(*gzip.Writer) | ||||||
|  | 		defer gzPool.Put(gz) | ||||||
|  | 
 | ||||||
|  | 		var b bytes.Buffer | ||||||
|  | 		gz.Reset(&b) | ||||||
|  | 
 | ||||||
|  | 		if _, err := gz.Write(body); err != nil { | ||||||
|  | 			return req, err | ||||||
|  | 		} | ||||||
|  | 		// Close needs to be called to ensure body if fully written.
 | ||||||
|  | 		if err := gz.Close(); err != nil { | ||||||
|  | 			return req, err | ||||||
|  | 		} | ||||||
|  | 
 | ||||||
|  | 		req.bodyReader = bodyReader(b.Bytes()) | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return req, nil | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // bodyReader returns a closure returning a new reader for buf.
 | ||||||
|  | func bodyReader(buf []byte) func() io.ReadCloser { | ||||||
|  | 	return func() io.ReadCloser { | ||||||
|  | 		return ioutil.NopCloser(bytes.NewReader(buf)) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // request wraps an http.Request with a resettable body reader.
 | ||||||
|  | type request struct { | ||||||
|  | 	*http.Request | ||||||
|  | 
 | ||||||
|  | 	// bodyReader allows the same body to be used for multiple requests.
 | ||||||
|  | 	bodyReader func() io.ReadCloser | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // reset reinitializes the request Body and uses ctx for the request.
 | ||||||
|  | func (r *request) reset(ctx context.Context) { | ||||||
|  | 	r.Body = r.bodyReader() | ||||||
|  | 	r.Request = r.Request.WithContext(ctx) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // retryableError represents a request failure that can be retried.
 | ||||||
|  | type retryableError struct { | ||||||
|  | 	throttle int64 | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // newResponseError returns a retryableError and will extract any explicit
 | ||||||
|  | // throttle delay contained in headers.
 | ||||||
|  | func newResponseError(header http.Header) error { | ||||||
|  | 	var rErr retryableError | ||||||
|  | 	if s, ok := header["Retry-After"]; ok { | ||||||
|  | 		if t, err := strconv.ParseInt(s[0], 10, 64); err == nil { | ||||||
|  | 			rErr.throttle = t | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
| 	return fmt.Errorf("failed to send data to %s after %d tries", address, d.generalCfg.MaxAttempts) | 	return rErr | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func (e retryableError) Error() string { | ||||||
|  | 	return "retry-able request failure" | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | // evaluate returns if err is retry-able. If it is and it includes an explicit
 | ||||||
|  | // throttling delay, that delay is also returned.
 | ||||||
|  | func evaluate(err error) (bool, time.Duration) { | ||||||
|  | 	if err == nil { | ||||||
|  | 		return false, 0 | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	rErr, ok := err.(retryableError) | ||||||
|  | 	if !ok { | ||||||
|  | 		return false, 0 | ||||||
|  | 	} | ||||||
|  | 
 | ||||||
|  | 	return true, time.Duration(rErr.throttle) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func (d *client) getScheme() string { | func (d *client) getScheme() string { | ||||||
|  | @ -191,26 +304,6 @@ func (d *client) getScheme() string { | ||||||
| 	return "https" | 	return "https" | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func getWaitDuration(backoff time.Duration, i int) time.Duration { |  | ||||||
| 	// Strategy: after nth failed attempt, attempt resending after
 |  | ||||||
| 	// k * initialBackoff + jitter, where k is a random number in
 |  | ||||||
| 	// range [0, 2^n-1), and jitter is a random percentage of
 |  | ||||||
| 	// initialBackoff from [-5%, 5%).
 |  | ||||||
| 	//
 |  | ||||||
| 	// Based on
 |  | ||||||
| 	// https://en.wikipedia.org/wiki/Exponential_backoff#Example_exponential_backoff_algorithm
 |  | ||||||
| 	//
 |  | ||||||
| 	// Jitter is our addition.
 |  | ||||||
| 
 |  | ||||||
| 	// There won't be an overflow, since i is capped to
 |  | ||||||
| 	// defaultMaxAttempts (5).
 |  | ||||||
| 	upperK := (int64)(1) << (i + 1) |  | ||||||
| 	jitterPercent := (rand.Float64() - 0.5) / 10. |  | ||||||
| 	jitter := jitterPercent * (float64)(backoff) |  | ||||||
| 	k := rand.Int63n(upperK) |  | ||||||
| 	return (time.Duration)(k)*backoff + (time.Duration)(jitter) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (d *client) contextWithStop(ctx context.Context) (context.Context, context.CancelFunc) { | func (d *client) contextWithStop(ctx context.Context) (context.Context, context.CancelFunc) { | ||||||
| 	// Unify the parent context Done signal with the client's stop
 | 	// Unify the parent context Done signal with the client's stop
 | ||||||
| 	// channel.
 | 	// channel.
 | ||||||
|  | @ -226,51 +319,3 @@ func (d *client) contextWithStop(ctx context.Context) (context.Context, context. | ||||||
| 	}(ctx, cancel) | 	}(ctx, cancel) | ||||||
| 	return ctx, cancel | 	return ctx, cancel | ||||||
| } | } | ||||||
| 
 |  | ||||||
| func (d *client) singleSend(ctx context.Context, rawRequest []byte, address string) (*http.Response, error) { |  | ||||||
| 	request, err := http.NewRequestWithContext(ctx, http.MethodPost, address, nil) |  | ||||||
| 	if err != nil { |  | ||||||
| 		return nil, err |  | ||||||
| 	} |  | ||||||
| 	bodyReader, contentLength, headers := d.prepareBody(rawRequest) |  | ||||||
| 	// Not closing bodyReader through defer, the HTTP Client's
 |  | ||||||
| 	// Transport will do it for us
 |  | ||||||
| 	request.Body = bodyReader |  | ||||||
| 	request.ContentLength = contentLength |  | ||||||
| 	for key, values := range headers { |  | ||||||
| 		for _, value := range values { |  | ||||||
| 			request.Header.Add(key, value) |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
| 	return d.client.Do(request) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func (d *client) prepareBody(rawRequest []byte) (io.ReadCloser, int64, http.Header) { |  | ||||||
| 	var bodyReader io.ReadCloser |  | ||||||
| 	headers := http.Header{} |  | ||||||
| 	for k, v := range d.cfg.Headers { |  | ||||||
| 		headers.Set(k, v) |  | ||||||
| 	} |  | ||||||
| 	contentLength := (int64)(len(rawRequest)) |  | ||||||
| 	headers.Set("Content-Type", contentTypeProto) |  | ||||||
| 	requestReader := bytes.NewBuffer(rawRequest) |  | ||||||
| 	switch Compression(d.cfg.Compression) { |  | ||||||
| 	case NoCompression: |  | ||||||
| 		bodyReader = ioutil.NopCloser(requestReader) |  | ||||||
| 	case GzipCompression: |  | ||||||
| 		preader, pwriter := io.Pipe() |  | ||||||
| 		go func() { |  | ||||||
| 			defer pwriter.Close() |  | ||||||
| 			gzipper := gzip.NewWriter(pwriter) |  | ||||||
| 			defer gzipper.Close() |  | ||||||
| 			_, err := io.Copy(gzipper, requestReader) |  | ||||||
| 			if err != nil { |  | ||||||
| 				otel.Handle(fmt.Errorf("otlphttp: failed to gzip request: %v", err)) |  | ||||||
| 			} |  | ||||||
| 		}() |  | ||||||
| 		headers.Set("Content-Encoding", "gzip") |  | ||||||
| 		bodyReader = preader |  | ||||||
| 		contentLength = -1 |  | ||||||
| 	} |  | ||||||
| 	return bodyReader, contentLength, headers |  | ||||||
| } |  | ||||||
|  |  | ||||||
|  | @ -16,7 +16,6 @@ package otlpmetrichttp_test | ||||||
| 
 | 
 | ||||||
| import ( | import ( | ||||||
| 	"context" | 	"context" | ||||||
| 	"fmt" |  | ||||||
| 	"net/http" | 	"net/http" | ||||||
| 	"os" | 	"os" | ||||||
| 	"testing" | 	"testing" | ||||||
|  | @ -144,32 +143,6 @@ func TestExporterShutdown(t *testing.T) { | ||||||
| 	}) | 	}) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func TestRetry(t *testing.T) { |  | ||||||
| 	statuses := []int{ |  | ||||||
| 		http.StatusTooManyRequests, |  | ||||||
| 		http.StatusServiceUnavailable, |  | ||||||
| 	} |  | ||||||
| 	mcCfg := mockCollectorConfig{ |  | ||||||
| 		InjectHTTPStatus: statuses, |  | ||||||
| 	} |  | ||||||
| 	mc := runMockCollector(t, mcCfg) |  | ||||||
| 	defer mc.MustStop(t) |  | ||||||
| 	client := otlpmetrichttp.NewClient( |  | ||||||
| 		otlpmetrichttp.WithEndpoint(mc.Endpoint()), |  | ||||||
| 		otlpmetrichttp.WithInsecure(), |  | ||||||
| 		otlpmetrichttp.WithMaxAttempts(len(statuses)+1), |  | ||||||
| 	) |  | ||||||
| 	ctx := context.Background() |  | ||||||
| 	exporter, err := otlpmetric.New(ctx, client) |  | ||||||
| 	require.NoError(t, err) |  | ||||||
| 	defer func() { |  | ||||||
| 		assert.NoError(t, exporter.Shutdown(ctx)) |  | ||||||
| 	}() |  | ||||||
| 	err = exporter.Export(ctx, testResource, oneRecord) |  | ||||||
| 	assert.NoError(t, err) |  | ||||||
| 	assert.Len(t, mc.GetMetrics(), 1) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func TestTimeout(t *testing.T) { | func TestTimeout(t *testing.T) { | ||||||
| 	mcCfg := mockCollectorConfig{ | 	mcCfg := mockCollectorConfig{ | ||||||
| 		InjectDelay: 100 * time.Millisecond, | 		InjectDelay: 100 * time.Millisecond, | ||||||
|  | @ -191,58 +164,6 @@ func TestTimeout(t *testing.T) { | ||||||
| 	assert.Equal(t, true, os.IsTimeout(err)) | 	assert.Equal(t, true, os.IsTimeout(err)) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func TestRetryFailed(t *testing.T) { |  | ||||||
| 	statuses := []int{ |  | ||||||
| 		http.StatusTooManyRequests, |  | ||||||
| 		http.StatusServiceUnavailable, |  | ||||||
| 	} |  | ||||||
| 	mcCfg := mockCollectorConfig{ |  | ||||||
| 		InjectHTTPStatus: statuses, |  | ||||||
| 	} |  | ||||||
| 	mc := runMockCollector(t, mcCfg) |  | ||||||
| 	defer mc.MustStop(t) |  | ||||||
| 	driver := otlpmetrichttp.NewClient( |  | ||||||
| 		otlpmetrichttp.WithEndpoint(mc.Endpoint()), |  | ||||||
| 		otlpmetrichttp.WithInsecure(), |  | ||||||
| 		otlpmetrichttp.WithMaxAttempts(1), |  | ||||||
| 	) |  | ||||||
| 	ctx := context.Background() |  | ||||||
| 	exporter, err := otlpmetric.New(ctx, driver) |  | ||||||
| 	require.NoError(t, err) |  | ||||||
| 	defer func() { |  | ||||||
| 		assert.NoError(t, exporter.Shutdown(ctx)) |  | ||||||
| 	}() |  | ||||||
| 	err = exporter.Export(ctx, testResource, oneRecord) |  | ||||||
| 	assert.Error(t, err) |  | ||||||
| 	assert.Empty(t, mc.GetMetrics()) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func TestNoRetry(t *testing.T) { |  | ||||||
| 	statuses := []int{ |  | ||||||
| 		http.StatusBadRequest, |  | ||||||
| 	} |  | ||||||
| 	mcCfg := mockCollectorConfig{ |  | ||||||
| 		InjectHTTPStatus: statuses, |  | ||||||
| 	} |  | ||||||
| 	mc := runMockCollector(t, mcCfg) |  | ||||||
| 	defer mc.MustStop(t) |  | ||||||
| 	driver := otlpmetrichttp.NewClient( |  | ||||||
| 		otlpmetrichttp.WithEndpoint(mc.Endpoint()), |  | ||||||
| 		otlpmetrichttp.WithInsecure(), |  | ||||||
| 		otlpmetrichttp.WithMaxAttempts(len(statuses)+1), |  | ||||||
| 	) |  | ||||||
| 	ctx := context.Background() |  | ||||||
| 	exporter, err := otlpmetric.New(ctx, driver) |  | ||||||
| 	require.NoError(t, err) |  | ||||||
| 	defer func() { |  | ||||||
| 		assert.NoError(t, exporter.Shutdown(ctx)) |  | ||||||
| 	}() |  | ||||||
| 	err = exporter.Export(ctx, testResource, oneRecord) |  | ||||||
| 	assert.Error(t, err) |  | ||||||
| 	assert.Equal(t, fmt.Sprintf("failed to send metrics to http://%s/v1/metrics with HTTP status 400 Bad Request", mc.endpoint), err.Error()) |  | ||||||
| 	assert.Empty(t, mc.GetMetrics()) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func TestEmptyData(t *testing.T) { | func TestEmptyData(t *testing.T) { | ||||||
| 	mcCfg := mockCollectorConfig{} | 	mcCfg := mockCollectorConfig{} | ||||||
| 	mc := runMockCollector(t, mcCfg) | 	mc := runMockCollector(t, mcCfg) | ||||||
|  | @ -263,88 +184,6 @@ func TestEmptyData(t *testing.T) { | ||||||
| 	assert.NotEmpty(t, mc.GetMetrics()) | 	assert.NotEmpty(t, mc.GetMetrics()) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| func TestUnreasonableMaxAttempts(t *testing.T) { |  | ||||||
| 	// Max attempts is 5, we set collector to fail 7 times and try
 |  | ||||||
| 	// to configure max attempts to be either negative or too
 |  | ||||||
| 	// large. Since we set max attempts to 5 in such cases,
 |  | ||||||
| 	// exporting to the collector should fail.
 |  | ||||||
| 	type testcase struct { |  | ||||||
| 		name        string |  | ||||||
| 		maxAttempts int |  | ||||||
| 	} |  | ||||||
| 	for _, tc := range []testcase{ |  | ||||||
| 		{ |  | ||||||
| 			name:        "negative max attempts", |  | ||||||
| 			maxAttempts: -3, |  | ||||||
| 		}, |  | ||||||
| 		{ |  | ||||||
| 			name:        "too large max attempts", |  | ||||||
| 			maxAttempts: 10, |  | ||||||
| 		}, |  | ||||||
| 	} { |  | ||||||
| 		t.Run(tc.name, func(t *testing.T) { |  | ||||||
| 			statuses := make([]int, 0, 7) |  | ||||||
| 			for i := 0; i < cap(statuses); i++ { |  | ||||||
| 				statuses = append(statuses, http.StatusTooManyRequests) |  | ||||||
| 			} |  | ||||||
| 			mcCfg := mockCollectorConfig{ |  | ||||||
| 				InjectHTTPStatus: statuses, |  | ||||||
| 			} |  | ||||||
| 			mc := runMockCollector(t, mcCfg) |  | ||||||
| 			defer mc.MustStop(t) |  | ||||||
| 			driver := otlpmetrichttp.NewClient( |  | ||||||
| 				otlpmetrichttp.WithEndpoint(mc.Endpoint()), |  | ||||||
| 				otlpmetrichttp.WithInsecure(), |  | ||||||
| 				otlpmetrichttp.WithMaxAttempts(tc.maxAttempts), |  | ||||||
| 				otlpmetrichttp.WithBackoff(time.Millisecond), |  | ||||||
| 			) |  | ||||||
| 			ctx := context.Background() |  | ||||||
| 			exporter, err := otlpmetric.New(ctx, driver) |  | ||||||
| 			require.NoError(t, err) |  | ||||||
| 			defer func() { |  | ||||||
| 				assert.NoError(t, exporter.Shutdown(ctx)) |  | ||||||
| 			}() |  | ||||||
| 			err = exporter.Export(ctx, testResource, oneRecord) |  | ||||||
| 			assert.Error(t, err) |  | ||||||
| 			assert.Empty(t, mc.GetMetrics()) |  | ||||||
| 		}) |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func TestUnreasonableBackoff(t *testing.T) { |  | ||||||
| 	// This sets backoff to negative value, which gets corrected
 |  | ||||||
| 	// to default backoff instead of being used. Default max
 |  | ||||||
| 	// attempts is 5, so we set the collector to fail 4 times, but
 |  | ||||||
| 	// we set the deadline to 3 times of the default backoff, so
 |  | ||||||
| 	// this should show that deadline is not met, meaning that the
 |  | ||||||
| 	// retries weren't immediate (as negative backoff could
 |  | ||||||
| 	// imply).
 |  | ||||||
| 	statuses := make([]int, 0, 4) |  | ||||||
| 	for i := 0; i < cap(statuses); i++ { |  | ||||||
| 		statuses = append(statuses, http.StatusTooManyRequests) |  | ||||||
| 	} |  | ||||||
| 	mcCfg := mockCollectorConfig{ |  | ||||||
| 		InjectHTTPStatus: statuses, |  | ||||||
| 	} |  | ||||||
| 	mc := runMockCollector(t, mcCfg) |  | ||||||
| 	defer mc.MustStop(t) |  | ||||||
| 	driver := otlpmetrichttp.NewClient( |  | ||||||
| 		otlpmetrichttp.WithEndpoint(mc.Endpoint()), |  | ||||||
| 		otlpmetrichttp.WithInsecure(), |  | ||||||
| 		otlpmetrichttp.WithBackoff(-time.Millisecond), |  | ||||||
| 	) |  | ||||||
| 	ctx, cancel := context.WithTimeout(context.Background(), 3*(300*time.Millisecond)) |  | ||||||
| 	defer cancel() |  | ||||||
| 	exporter, err := otlpmetric.New(ctx, driver) |  | ||||||
| 	require.NoError(t, err) |  | ||||||
| 	defer func() { |  | ||||||
| 		assert.NoError(t, exporter.Shutdown(context.Background())) |  | ||||||
| 	}() |  | ||||||
| 	err = exporter.Export(ctx, testResource, oneRecord) |  | ||||||
| 	assert.Error(t, err) |  | ||||||
| 	assert.Empty(t, mc.GetMetrics()) |  | ||||||
| } |  | ||||||
| 
 |  | ||||||
| func TestCancelledContext(t *testing.T) { | func TestCancelledContext(t *testing.T) { | ||||||
| 	statuses := []int{ | 	statuses := []int{ | ||||||
| 		http.StatusBadRequest, | 		http.StatusBadRequest, | ||||||
|  |  | ||||||
|  | @ -0,0 +1,68 @@ | ||||||
|  | // Copyright The OpenTelemetry Authors
 | ||||||
|  | //
 | ||||||
|  | // Licensed under the Apache License, Version 2.0 (the "License");
 | ||||||
|  | // you may not use this file except in compliance with the License.
 | ||||||
|  | // You may obtain a copy of the License at
 | ||||||
|  | //
 | ||||||
|  | //     http://www.apache.org/licenses/LICENSE-2.0
 | ||||||
|  | //
 | ||||||
|  | // Unless required by applicable law or agreed to in writing, software
 | ||||||
|  | // distributed under the License is distributed on an "AS IS" BASIS,
 | ||||||
|  | // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 | ||||||
|  | // See the License for the specific language governing permissions and
 | ||||||
|  | // limitations under the License.
 | ||||||
|  | 
 | ||||||
|  | package otlpmetrichttp | ||||||
|  | 
 | ||||||
|  | import ( | ||||||
|  | 	"testing" | ||||||
|  | 	"time" | ||||||
|  | 
 | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | 	"github.com/stretchr/testify/require" | ||||||
|  | ) | ||||||
|  | 
 | ||||||
|  | func TestUnreasonableBackoff(t *testing.T) { | ||||||
|  | 	cIface := NewClient( | ||||||
|  | 		WithEndpoint("http://localhost"), | ||||||
|  | 		WithInsecure(), | ||||||
|  | 		WithBackoff(-time.Microsecond), | ||||||
|  | 	) | ||||||
|  | 	require.IsType(t, &client{}, cIface) | ||||||
|  | 	c := cIface.(*client) | ||||||
|  | 	assert.True(t, c.generalCfg.RetryConfig.Enabled) | ||||||
|  | 	assert.Equal(t, 5*time.Second, c.generalCfg.RetryConfig.InitialInterval) | ||||||
|  | 	assert.Equal(t, 300*time.Millisecond, c.generalCfg.RetryConfig.MaxInterval) | ||||||
|  | 	assert.Equal(t, time.Minute, c.generalCfg.RetryConfig.MaxElapsedTime) | ||||||
|  | } | ||||||
|  | 
 | ||||||
|  | func TestUnreasonableMaxAttempts(t *testing.T) { | ||||||
|  | 	type testcase struct { | ||||||
|  | 		name        string | ||||||
|  | 		maxAttempts int | ||||||
|  | 	} | ||||||
|  | 	for _, tc := range []testcase{ | ||||||
|  | 		{ | ||||||
|  | 			name:        "negative max attempts", | ||||||
|  | 			maxAttempts: -3, | ||||||
|  | 		}, | ||||||
|  | 		{ | ||||||
|  | 			name:        "too large max attempts", | ||||||
|  | 			maxAttempts: 10, | ||||||
|  | 		}, | ||||||
|  | 	} { | ||||||
|  | 		t.Run(tc.name, func(t *testing.T) { | ||||||
|  | 			cIface := NewClient( | ||||||
|  | 				WithEndpoint("http://localhost"), | ||||||
|  | 				WithInsecure(), | ||||||
|  | 				WithMaxAttempts(tc.maxAttempts), | ||||||
|  | 			) | ||||||
|  | 			require.IsType(t, &client{}, cIface) | ||||||
|  | 			c := cIface.(*client) | ||||||
|  | 			assert.True(t, c.generalCfg.RetryConfig.Enabled) | ||||||
|  | 			assert.Equal(t, 5*time.Second, c.generalCfg.RetryConfig.InitialInterval) | ||||||
|  | 			assert.Equal(t, 30*time.Second, c.generalCfg.RetryConfig.MaxInterval) | ||||||
|  | 			assert.Equal(t, 145*time.Second, c.generalCfg.RetryConfig.MaxElapsedTime) | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  | @ -4,7 +4,6 @@ go 1.16 | ||||||
| 
 | 
 | ||||||
| require ( | require ( | ||||||
| 	github.com/stretchr/testify v1.7.0 | 	github.com/stretchr/testify v1.7.0 | ||||||
| 	go.opentelemetry.io/otel v1.2.0 |  | ||||||
| 	go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.25.0 | 	go.opentelemetry.io/otel/exporters/otlp/otlpmetric v0.25.0 | ||||||
| 	go.opentelemetry.io/otel/sdk v1.2.0 | 	go.opentelemetry.io/otel/sdk v1.2.0 | ||||||
| 	go.opentelemetry.io/proto/otlp v0.11.0 | 	go.opentelemetry.io/proto/otlp v0.11.0 | ||||||
|  |  | ||||||
|  | @ -4,6 +4,7 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03 | ||||||
| github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= | github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY= | ||||||
| github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= | github.com/benbjohnson/clock v1.3.0 h1:ip6w0uFQkncKQ979AypyG0ER7mqUSBdKLOgAle/AT8A= | ||||||
| github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= | github.com/benbjohnson/clock v1.3.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= | ||||||
|  | github.com/cenkalti/backoff/v4 v4.1.2 h1:6Yo7N8UP2K6LWZnW94DLVSSrbobcWdVzAYOisuDPIFo= | ||||||
| github.com/cenkalti/backoff/v4 v4.1.2/go.mod h1:scbssz8iZGpm3xbr14ovlUdkxfGXNInqkPWOWmG2CLw= | github.com/cenkalti/backoff/v4 v4.1.2/go.mod h1:scbssz8iZGpm3xbr14ovlUdkxfGXNInqkPWOWmG2CLw= | ||||||
| github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= | github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= | ||||||
| github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= | github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= | ||||||
|  |  | ||||||
|  | @ -19,19 +19,7 @@ import ( | ||||||
| 	"time" | 	"time" | ||||||
| 
 | 
 | ||||||
| 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpconfig" | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/otlpconfig" | ||||||
| ) | 	"go.opentelemetry.io/otel/exporters/otlp/otlpmetric/internal/retry" | ||||||
| 
 |  | ||||||
| const ( |  | ||||||
| 	// defaultMaxAttempts describes how many times the driver
 |  | ||||||
| 	// should retry the sending of the payload in case of a
 |  | ||||||
| 	// retryable error.
 |  | ||||||
| 	defaultMaxAttempts int = 5 |  | ||||||
| 	// defaultMetricsPath is a default URL path for endpoint that
 |  | ||||||
| 	// receives metrics.
 |  | ||||||
| 	defaultMetricsPath string = "/v1/metrics" |  | ||||||
| 	// defaultBackoff is a default base backoff time used in the
 |  | ||||||
| 	// exponential backoff strategy.
 |  | ||||||
| 	defaultBackoff time.Duration = 300 * time.Millisecond |  | ||||||
| ) | ) | ||||||
| 
 | 
 | ||||||
| // Compression describes the compression used for payloads sent to the
 | // Compression describes the compression used for payloads sent to the
 | ||||||
|  | @ -52,6 +40,10 @@ type Option interface { | ||||||
| 	applyHTTPOption(*otlpconfig.Config) | 	applyHTTPOption(*otlpconfig.Config) | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
|  | // RetryConfig defines configuration for retrying batches in case of export
 | ||||||
|  | // failure using an exponential backoff.
 | ||||||
|  | type RetryConfig retry.Config | ||||||
|  | 
 | ||||||
| type wrappedOption struct { | type wrappedOption struct { | ||||||
| 	otlpconfig.HTTPOption | 	otlpconfig.HTTPOption | ||||||
| } | } | ||||||
|  | @ -84,15 +76,67 @@ func WithURLPath(urlPath string) Option { | ||||||
| // will try to send the payload in case of retryable errors.
 | // will try to send the payload in case of retryable errors.
 | ||||||
| // The max attempts is limited to at most 5 retries. If unset,
 | // The max attempts is limited to at most 5 retries. If unset,
 | ||||||
| // default (5) will be used.
 | // default (5) will be used.
 | ||||||
|  | //
 | ||||||
|  | // Deprecated: Use WithRetry instead.
 | ||||||
| func WithMaxAttempts(maxAttempts int) Option { | func WithMaxAttempts(maxAttempts int) Option { | ||||||
| 	return wrappedOption{otlpconfig.WithMaxAttempts(maxAttempts)} | 	if maxAttempts > 5 || maxAttempts < 0 { | ||||||
|  | 		maxAttempts = 5 | ||||||
|  | 	} | ||||||
|  | 	return wrappedOption{ | ||||||
|  | 		otlpconfig.NewHTTPOption(func(cfg *otlpconfig.Config) { | ||||||
|  | 			cfg.RetryConfig.Enabled = true | ||||||
|  | 
 | ||||||
|  | 			var ( | ||||||
|  | 				init = cfg.RetryConfig.InitialInterval | ||||||
|  | 				maxI = cfg.RetryConfig.MaxInterval | ||||||
|  | 				maxE = cfg.RetryConfig.MaxElapsedTime | ||||||
|  | 			) | ||||||
|  | 
 | ||||||
|  | 			if init == 0 { | ||||||
|  | 				init = retry.DefaultConfig.InitialInterval | ||||||
|  | 			} | ||||||
|  | 			if maxI == 0 { | ||||||
|  | 				maxI = retry.DefaultConfig.MaxInterval | ||||||
|  | 			} | ||||||
|  | 			if maxE == 0 { | ||||||
|  | 				maxE = retry.DefaultConfig.MaxElapsedTime | ||||||
|  | 			} | ||||||
|  | 			attempts := int64(maxE+init) / int64(maxI) | ||||||
|  | 
 | ||||||
|  | 			if int64(maxAttempts) == attempts { | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  | 
 | ||||||
|  | 			maxE = time.Duration(int64(maxAttempts)*int64(maxI)) - init | ||||||
|  | 
 | ||||||
|  | 			cfg.RetryConfig.InitialInterval = init | ||||||
|  | 			cfg.RetryConfig.MaxInterval = maxI | ||||||
|  | 			cfg.RetryConfig.MaxElapsedTime = maxE | ||||||
|  | 		}), | ||||||
|  | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithBackoff tells the driver to use the duration as a base of the
 | // WithBackoff tells the driver to use the duration as a base of the
 | ||||||
| // exponential backoff strategy. If unset, default (300ms) will be
 | // exponential backoff strategy. If unset, default (300ms) will be
 | ||||||
| // used.
 | // used.
 | ||||||
|  | //
 | ||||||
|  | // Deprecated: Use WithRetry instead.
 | ||||||
| func WithBackoff(duration time.Duration) Option { | func WithBackoff(duration time.Duration) Option { | ||||||
| 	return wrappedOption{otlpconfig.WithBackoff(duration)} | 	if duration < 0 { | ||||||
|  | 		duration = 300 * time.Millisecond | ||||||
|  | 	} | ||||||
|  | 	return wrappedOption{ | ||||||
|  | 		otlpconfig.NewHTTPOption(func(cfg *otlpconfig.Config) { | ||||||
|  | 			cfg.RetryConfig.Enabled = true | ||||||
|  | 			cfg.RetryConfig.MaxInterval = duration | ||||||
|  | 			if cfg.RetryConfig.InitialInterval == 0 { | ||||||
|  | 				cfg.RetryConfig.InitialInterval = retry.DefaultConfig.InitialInterval | ||||||
|  | 			} | ||||||
|  | 			if cfg.RetryConfig.MaxElapsedTime == 0 { | ||||||
|  | 				cfg.RetryConfig.MaxElapsedTime = retry.DefaultConfig.MaxElapsedTime | ||||||
|  | 			} | ||||||
|  | 		}), | ||||||
|  | 	} | ||||||
| } | } | ||||||
| 
 | 
 | ||||||
| // WithTLSClientConfig can be used to set up a custom TLS
 | // WithTLSClientConfig can be used to set up a custom TLS
 | ||||||
|  | @ -120,3 +164,12 @@ func WithHeaders(headers map[string]string) Option { | ||||||
| func WithTimeout(duration time.Duration) Option { | func WithTimeout(duration time.Duration) Option { | ||||||
| 	return wrappedOption{otlpconfig.WithTimeout(duration)} | 	return wrappedOption{otlpconfig.WithTimeout(duration)} | ||||||
| } | } | ||||||
|  | 
 | ||||||
|  | // WithRetry configures the retry policy for transient errors that may occurs
 | ||||||
|  | // when exporting traces. An exponential back-off algorithm is used to ensure
 | ||||||
|  | // endpoints are not overwhelmed with retries. If unset, the default retry
 | ||||||
|  | // policy will retry after 5 seconds and increase exponentially after each
 | ||||||
|  | // error for a total of 1 minute.
 | ||||||
|  | func WithRetry(rc RetryConfig) Option { | ||||||
|  | 	return wrappedOption{otlpconfig.WithRetry(retry.Config(rc))} | ||||||
|  | } | ||||||
|  |  | ||||||
		Loading…
	
		Reference in New Issue