From c63aeef12620f4731ab025f0ac1f67c367759826 Mon Sep 17 00:00:00 2001 From: Doug Fawley Date: Wed, 20 Nov 2024 15:40:17 -0800 Subject: [PATCH 01/57] transport: add send operations to ClientStream and ServerStream (#7808) --- internal/transport/client_stream.go | 41 +++++++++-- internal/transport/handler_server.go | 10 ++- internal/transport/handler_server_test.go | 14 ++-- internal/transport/http2_client.go | 43 +++++------- internal/transport/http2_server.go | 25 ++++--- internal/transport/keepalive_test.go | 6 +- internal/transport/server_stream.go | 42 ++++++++--- internal/transport/transport.go | 85 ++++++++-------------- internal/transport/transport_test.go | 86 +++++++++++------------ rpc_util.go | 7 +- rpc_util_test.go | 6 +- server.go | 45 +++++------- stream.go | 39 +++------- 13 files changed, 220 insertions(+), 229 deletions(-) diff --git a/internal/transport/client_stream.go b/internal/transport/client_stream.go index 112b34f6d7cc..8ed347c54195 100644 --- a/internal/transport/client_stream.go +++ b/internal/transport/client_stream.go @@ -21,6 +21,8 @@ package transport import ( "sync/atomic" + "golang.org/x/net/http2" + "google.golang.org/grpc/mem" "google.golang.org/grpc/metadata" "google.golang.org/grpc/status" ) @@ -29,7 +31,7 @@ import ( type ClientStream struct { *Stream // Embed for common stream functionality. - ct ClientTransport + ct *http2Client done chan struct{} // closed at the end of stream to unblock writers. doneFunc func() // invoked at the end of stream. @@ -42,21 +44,48 @@ type ClientStream struct { header metadata.MD // the received header metadata noHeaders bool // set if the client never received headers (set only after the stream is done). - bytesReceived uint32 // indicates whether any bytes have been received on this stream - unprocessed uint32 // set if the server sends a refused stream or GOAWAY including this stream + bytesReceived atomic.Bool // indicates whether any bytes have been received on this stream + unprocessed atomic.Bool // set if the server sends a refused stream or GOAWAY including this stream status *status.Status // the status error received from the server } +// Read reads an n byte message from the input stream. +func (s *ClientStream) Read(n int) (mem.BufferSlice, error) { + b, err := s.Stream.read(n) + if err == nil { + s.ct.incrMsgRecv() + } + return b, err +} + +// Close closes the stream and popagates err to any readers. +func (s *ClientStream) Close(err error) { + var ( + rst bool + rstCode http2.ErrCode + ) + if err != nil { + rst = true + rstCode = http2.ErrCodeCancel + } + s.ct.closeStream(s, err, rst, rstCode, status.Convert(err), nil, false) +} + +// Write writes the hdr and data bytes to the output stream. +func (s *ClientStream) Write(hdr []byte, data mem.BufferSlice, opts *WriteOptions) error { + return s.ct.write(s, hdr, data, opts) +} + // BytesReceived indicates whether any bytes have been received on this stream. func (s *ClientStream) BytesReceived() bool { - return atomic.LoadUint32(&s.bytesReceived) == 1 + return s.bytesReceived.Load() } // Unprocessed indicates whether the server did not process this stream -- // i.e. it sent a refused stream or GOAWAY including this stream ID. func (s *ClientStream) Unprocessed() bool { - return atomic.LoadUint32(&s.unprocessed) == 1 + return s.unprocessed.Load() } func (s *ClientStream) waitOnHeader() { @@ -64,7 +93,7 @@ func (s *ClientStream) waitOnHeader() { case <-s.ctx.Done(): // Close the stream to prevent headers/trailers from changing after // this function returns. - s.ct.CloseStream(s, ContextErr(s.ctx.Err())) + s.Close(ContextErr(s.ctx.Err())) // headerChan could possibly not be closed yet if closeStream raced // with operateHeaders; wait until it is closed explicitly here. <-s.headerChan diff --git a/internal/transport/handler_server.go b/internal/transport/handler_server.go index 0ebe4a71cb9b..d9305a65d88f 100644 --- a/internal/transport/handler_server.go +++ b/internal/transport/handler_server.go @@ -225,7 +225,7 @@ func (ht *serverHandlerTransport) do(fn func()) error { } } -func (ht *serverHandlerTransport) WriteStatus(s *ServerStream, st *status.Status) error { +func (ht *serverHandlerTransport) writeStatus(s *ServerStream, st *status.Status) error { ht.writeStatusMu.Lock() defer ht.writeStatusMu.Unlock() @@ -333,7 +333,7 @@ func (ht *serverHandlerTransport) writeCustomHeaders(s *ServerStream) { s.hdrMu.Unlock() } -func (ht *serverHandlerTransport) Write(s *ServerStream, hdr []byte, data mem.BufferSlice, _ *Options) error { +func (ht *serverHandlerTransport) write(s *ServerStream, hdr []byte, data mem.BufferSlice, _ *WriteOptions) error { // Always take a reference because otherwise there is no guarantee the data will // be available after this function returns. This is what callers to Write // expect. @@ -357,7 +357,7 @@ func (ht *serverHandlerTransport) Write(s *ServerStream, hdr []byte, data mem.Bu return nil } -func (ht *serverHandlerTransport) WriteHeader(s *ServerStream, md metadata.MD) error { +func (ht *serverHandlerTransport) writeHeader(s *ServerStream, md metadata.MD) error { if err := s.SetHeader(md); err != nil { return err } @@ -473,9 +473,7 @@ func (ht *serverHandlerTransport) runStream() { } } -func (ht *serverHandlerTransport) IncrMsgSent() {} - -func (ht *serverHandlerTransport) IncrMsgRecv() {} +func (ht *serverHandlerTransport) incrMsgRecv() {} func (ht *serverHandlerTransport) Drain(string) { panic("Drain() is not implemented") diff --git a/internal/transport/handler_server_test.go b/internal/transport/handler_server_test.go index 32f6325bd0ee..ca1a38de0f86 100644 --- a/internal/transport/handler_server_test.go +++ b/internal/transport/handler_server_test.go @@ -310,7 +310,7 @@ func (s) TestHandlerTransport_HandleStreams(t *testing.T) { } st.bodyw.Close() // no body - st.ht.WriteStatus(s, status.New(codes.OK, "")) + s.WriteStatus(status.New(codes.OK, "")) } st.ht.HandleStreams( context.Background(), func(s *ServerStream) { go handleStream(s) }, @@ -343,7 +343,7 @@ func handleStreamCloseBodyTest(t *testing.T, statusCode codes.Code, msg string) st := newHandleStreamTest(t) handleStream := func(s *ServerStream) { - st.ht.WriteStatus(s, status.New(statusCode, msg)) + s.WriteStatus(status.New(statusCode, msg)) } st.ht.HandleStreams( context.Background(), func(s *ServerStream) { go handleStream(s) }, @@ -392,7 +392,7 @@ func (s) TestHandlerTransport_HandleStreams_Timeout(t *testing.T) { t.Errorf("ctx.Err = %v; want %v", err, context.DeadlineExceeded) return } - ht.WriteStatus(s, status.New(codes.DeadlineExceeded, "too slow")) + s.WriteStatus(status.New(codes.DeadlineExceeded, "too slow")) } ht.HandleStreams( context.Background(), func(s *ServerStream) { go runStream(s) }, @@ -423,7 +423,7 @@ func (s) TestHandlerTransport_HandleStreams_MultiWriteStatus(t *testing.T) { for i := 0; i < 5; i++ { go func() { defer wg.Done() - st.ht.WriteStatus(s, status.New(codes.OK, "")) + s.WriteStatus(status.New(codes.OK, "")) }() } wg.Wait() @@ -439,8 +439,8 @@ func (s) TestHandlerTransport_HandleStreams_WriteStatusWrite(t *testing.T) { } st.bodyw.Close() // no body - st.ht.WriteStatus(s, status.New(codes.OK, "")) - st.ht.Write(s, []byte("hdr"), newBufferSlice([]byte("data")), &Options{}) + s.WriteStatus(status.New(codes.OK, "")) + s.Write([]byte("hdr"), newBufferSlice([]byte("data")), &WriteOptions{}) }) } @@ -477,7 +477,7 @@ func (s) TestHandlerTransport_HandleStreams_ErrDetails(t *testing.T) { hst := newHandleStreamTest(t) handleStream := func(s *ServerStream) { - hst.ht.WriteStatus(s, st) + s.WriteStatus(st) } hst.ht.HandleStreams( context.Background(), func(s *ServerStream) { go handleStream(s) }, diff --git a/internal/transport/http2_client.go b/internal/transport/http2_client.go index fb5b0c130d58..f323ab7f45a6 100644 --- a/internal/transport/http2_client.go +++ b/internal/transport/http2_client.go @@ -508,7 +508,7 @@ func (t *http2Client) newStream(ctx context.Context, callHdr *CallHdr) *ClientSt ctxDone: s.ctx.Done(), recv: s.buf, closeStream: func(err error) { - t.CloseStream(s, err) + s.Close(err) }, }, windowHandler: func(n int) { @@ -759,7 +759,7 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (*ClientS return } // The stream was unprocessed by the server. - atomic.StoreUint32(&s.unprocessed, 1) + s.unprocessed.Store(true) s.write(recvMsg{err: err}) close(s.done) // If headerChan isn't closed, then close it. @@ -904,20 +904,6 @@ func (t *http2Client) NewStream(ctx context.Context, callHdr *CallHdr) (*ClientS return s, nil } -// CloseStream clears the footprint of a stream when the stream is not needed any more. -// This must not be executed in reader's goroutine. -func (t *http2Client) CloseStream(s *ClientStream, err error) { - var ( - rst bool - rstCode http2.ErrCode - ) - if err != nil { - rst = true - rstCode = http2.ErrCodeCancel - } - t.closeStream(s, err, rst, rstCode, status.Convert(err), nil, false) -} - func (t *http2Client) closeStream(s *ClientStream, err error, rst bool, rstCode http2.ErrCode, st *status.Status, mdata map[string][]string, eosReceived bool) { // Set stream status to done. if s.swapState(streamDone) == streamDone { @@ -1081,7 +1067,7 @@ func (t *http2Client) GracefulClose() { // Write formats the data into HTTP2 data frame(s) and sends it out. The caller // should proceed only if Write returns nil. -func (t *http2Client) Write(s *ClientStream, hdr []byte, data mem.BufferSlice, opts *Options) error { +func (t *http2Client) write(s *ClientStream, hdr []byte, data mem.BufferSlice, opts *WriteOptions) error { reader := data.Reader() if opts.Last { @@ -1110,6 +1096,7 @@ func (t *http2Client) Write(s *ClientStream, hdr []byte, data mem.BufferSlice, o _ = reader.Close() return err } + t.incrMsgSent() return nil } @@ -1238,7 +1225,7 @@ func (t *http2Client) handleRSTStream(f *http2.RSTStreamFrame) { } if f.ErrCode == http2.ErrCodeRefusedStream { // The stream was unprocessed by the server. - atomic.StoreUint32(&s.unprocessed, 1) + s.unprocessed.Store(true) } statusCode, ok := http2ErrConvTab[f.ErrCode] if !ok { @@ -1383,7 +1370,7 @@ func (t *http2Client) handleGoAway(f *http2.GoAwayFrame) error { for streamID, stream := range t.activeStreams { if streamID > id && streamID <= upperLimit { // The stream was unprocessed by the server. - atomic.StoreUint32(&stream.unprocessed, 1) + stream.unprocessed.Store(true) streamsToClose = append(streamsToClose, stream) } } @@ -1435,7 +1422,7 @@ func (t *http2Client) operateHeaders(frame *http2.MetaHeadersFrame) { return } endStream := frame.StreamEnded() - atomic.StoreUint32(&s.bytesReceived, 1) + s.bytesReceived.Store(true) initialHeader := atomic.LoadUint32(&s.headerChanClosed) == 0 if !initialHeader && !endStream { @@ -1805,14 +1792,18 @@ func (t *http2Client) socketMetrics() *channelz.EphemeralSocketMetrics { func (t *http2Client) RemoteAddr() net.Addr { return t.remoteAddr } -func (t *http2Client) IncrMsgSent() { - t.channelz.SocketMetrics.MessagesSent.Add(1) - t.channelz.SocketMetrics.LastMessageSentTimestamp.Store(time.Now().UnixNano()) +func (t *http2Client) incrMsgSent() { + if channelz.IsOn() { + t.channelz.SocketMetrics.MessagesSent.Add(1) + t.channelz.SocketMetrics.LastMessageSentTimestamp.Store(time.Now().UnixNano()) + } } -func (t *http2Client) IncrMsgRecv() { - t.channelz.SocketMetrics.MessagesReceived.Add(1) - t.channelz.SocketMetrics.LastMessageReceivedTimestamp.Store(time.Now().UnixNano()) +func (t *http2Client) incrMsgRecv() { + if channelz.IsOn() { + t.channelz.SocketMetrics.MessagesReceived.Add(1) + t.channelz.SocketMetrics.LastMessageReceivedTimestamp.Store(time.Now().UnixNano()) + } } func (t *http2Client) getOutFlowWindow() int64 { diff --git a/internal/transport/http2_server.go b/internal/transport/http2_server.go index fefd9020ca07..0055fddd7ecf 100644 --- a/internal/transport/http2_server.go +++ b/internal/transport/http2_server.go @@ -969,7 +969,7 @@ func (t *http2Server) streamContextErr(s *ServerStream) error { } // WriteHeader sends the header metadata md back to the client. -func (t *http2Server) WriteHeader(s *ServerStream, md metadata.MD) error { +func (t *http2Server) writeHeader(s *ServerStream, md metadata.MD) error { s.hdrMu.Lock() defer s.hdrMu.Unlock() if s.getState() == streamDone { @@ -1042,7 +1042,7 @@ func (t *http2Server) writeHeaderLocked(s *ServerStream) error { // There is no further I/O operations being able to perform on this stream. // TODO(zhaoq): Now it indicates the end of entire stream. Revisit if early // OK is adopted. -func (t *http2Server) WriteStatus(s *ServerStream, st *status.Status) error { +func (t *http2Server) writeStatus(s *ServerStream, st *status.Status) error { s.hdrMu.Lock() defer s.hdrMu.Unlock() @@ -1113,11 +1113,11 @@ func (t *http2Server) WriteStatus(s *ServerStream, st *status.Status) error { // Write converts the data into HTTP2 data frame and sends it out. Non-nil error // is returns if it fails (e.g., framing error, transport error). -func (t *http2Server) Write(s *ServerStream, hdr []byte, data mem.BufferSlice, _ *Options) error { +func (t *http2Server) write(s *ServerStream, hdr []byte, data mem.BufferSlice, _ *WriteOptions) error { reader := data.Reader() if !s.isHeaderSent() { // Headers haven't been written yet. - if err := t.WriteHeader(s, nil); err != nil { + if err := t.writeHeader(s, nil); err != nil { _ = reader.Close() return err } @@ -1143,6 +1143,7 @@ func (t *http2Server) Write(s *ServerStream, hdr []byte, data mem.BufferSlice, _ _ = reader.Close() return err } + t.incrMsgSent() return nil } @@ -1411,14 +1412,18 @@ func (t *http2Server) socketMetrics() *channelz.EphemeralSocketMetrics { } } -func (t *http2Server) IncrMsgSent() { - t.channelz.SocketMetrics.MessagesSent.Add(1) - t.channelz.SocketMetrics.LastMessageSentTimestamp.Add(1) +func (t *http2Server) incrMsgSent() { + if channelz.IsOn() { + t.channelz.SocketMetrics.MessagesSent.Add(1) + t.channelz.SocketMetrics.LastMessageSentTimestamp.Add(1) + } } -func (t *http2Server) IncrMsgRecv() { - t.channelz.SocketMetrics.MessagesReceived.Add(1) - t.channelz.SocketMetrics.LastMessageReceivedTimestamp.Add(1) +func (t *http2Server) incrMsgRecv() { + if channelz.IsOn() { + t.channelz.SocketMetrics.MessagesReceived.Add(1) + t.channelz.SocketMetrics.LastMessageReceivedTimestamp.Add(1) + } } func (t *http2Server) getOutFlowWindow() int64 { diff --git a/internal/transport/keepalive_test.go b/internal/transport/keepalive_test.go index ad377e6b241b..037b0b1c1b77 100644 --- a/internal/transport/keepalive_test.go +++ b/internal/transport/keepalive_test.go @@ -68,7 +68,7 @@ func (s) TestMaxConnectionIdle(t *testing.T) { if err != nil { t.Fatalf("client.NewStream() failed: %v", err) } - client.CloseStream(stream, io.EOF) + stream.Close(io.EOF) // Verify the server sends a GoAway to client after MaxConnectionIdle timeout // kicks in. @@ -708,7 +708,7 @@ func (s) TestTCPUserTimeout(t *testing.T) { if err != nil { t.Fatalf("client.NewStream() failed: %v", err) } - client.CloseStream(stream, io.EOF) + stream.Close(io.EOF) // check client TCP user timeout only when non TLS // TODO : find a way to get the underlying conn for client when TLS @@ -772,7 +772,7 @@ func checkForHealthyStream(client *http2Client) error { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() stream, err := client.NewStream(ctx, &CallHdr{}) - client.CloseStream(stream, err) + stream.Close(err) return err } diff --git a/internal/transport/server_stream.go b/internal/transport/server_stream.go index acbf014900bc..a22a90151494 100644 --- a/internal/transport/server_stream.go +++ b/internal/transport/server_stream.go @@ -25,14 +25,16 @@ import ( "sync" "sync/atomic" + "google.golang.org/grpc/mem" "google.golang.org/grpc/metadata" + "google.golang.org/grpc/status" ) // ServerStream implements streaming functionality for a gRPC server. type ServerStream struct { *Stream // Embed for common stream functionality. - st ServerTransport + st internalServerTransport ctxDone <-chan struct{} // closed at the end of stream. Cache of ctx.Done() (for performance) cancel context.CancelFunc // invoked at the end of stream to cancel ctx. @@ -44,18 +46,43 @@ type ServerStream struct { // hdrMu protects outgoing header and trailer metadata. hdrMu sync.Mutex header metadata.MD // the outgoing header metadata. Updated by WriteHeader. - headerSent uint32 // atomically set to 1 when the headers are sent out. + headerSent atomic.Bool // atomically set when the headers are sent out. +} + +// Read reads an n byte message from the input stream. +func (s *ServerStream) Read(n int) (mem.BufferSlice, error) { + b, err := s.Stream.read(n) + if err == nil { + s.st.incrMsgRecv() + } + return b, err +} + +// SendHeader sends the header metadata for the given stream. +func (s *ServerStream) SendHeader(md metadata.MD) error { + return s.st.writeHeader(s, md) +} + +// Write writes the hdr and data bytes to the output stream. +func (s *ServerStream) Write(hdr []byte, data mem.BufferSlice, opts *WriteOptions) error { + return s.st.write(s, hdr, data, opts) +} + +// WriteStatus sends the status of a stream to the client. WriteStatus is +// the final call made on a stream and always occurs. +func (s *ServerStream) WriteStatus(st *status.Status) error { + return s.st.writeStatus(s, st) } // isHeaderSent indicates whether headers have been sent. func (s *ServerStream) isHeaderSent() bool { - return atomic.LoadUint32(&s.headerSent) == 1 + return s.headerSent.Load() } // updateHeaderSent updates headerSent and returns true // if it was already set. func (s *ServerStream) updateHeaderSent() bool { - return atomic.SwapUint32(&s.headerSent, 1) == 1 + return s.headerSent.Swap(true) } // RecvCompress returns the compression algorithm applied to the inbound @@ -134,13 +161,6 @@ func (s *ServerStream) SetHeader(md metadata.MD) error { return nil } -// SendHeader sends the given header metadata. The given metadata is -// combined with any metadata set by previous calls to SetHeader and -// then written to the transport stream. -func (s *ServerStream) SendHeader(md metadata.MD) error { - return s.st.WriteHeader(s, md) -} - // SetTrailer sets the trailer metadata which will be sent with the RPC status // by the server. This can be called multiple times. // This should not be called parallel to other data writes. diff --git a/internal/transport/transport.go b/internal/transport/transport.go index f3148e31c5dd..2859b87755f0 100644 --- a/internal/transport/transport.go +++ b/internal/transport/transport.go @@ -131,7 +131,7 @@ type recvBufferReader struct { err error } -func (r *recvBufferReader) ReadHeader(header []byte) (n int, err error) { +func (r *recvBufferReader) ReadMessageHeader(header []byte) (n int, err error) { if r.err != nil { return 0, r.err } @@ -140,9 +140,9 @@ func (r *recvBufferReader) ReadHeader(header []byte) (n int, err error) { return n, nil } if r.closeStream != nil { - n, r.err = r.readHeaderClient(header) + n, r.err = r.readMessageHeaderClient(header) } else { - n, r.err = r.readHeader(header) + n, r.err = r.readMessageHeader(header) } return n, r.err } @@ -172,12 +172,12 @@ func (r *recvBufferReader) Read(n int) (buf mem.Buffer, err error) { return buf, r.err } -func (r *recvBufferReader) readHeader(header []byte) (n int, err error) { +func (r *recvBufferReader) readMessageHeader(header []byte) (n int, err error) { select { case <-r.ctxDone: return 0, ContextErr(r.ctx.Err()) case m := <-r.recv.get(): - return r.readHeaderAdditional(m, header) + return r.readMessageHeaderAdditional(m, header) } } @@ -190,7 +190,7 @@ func (r *recvBufferReader) read(n int) (buf mem.Buffer, err error) { } } -func (r *recvBufferReader) readHeaderClient(header []byte) (n int, err error) { +func (r *recvBufferReader) readMessageHeaderClient(header []byte) (n int, err error) { // If the context is canceled, then closes the stream with nil metadata. // closeStream writes its error parameter to r.recv as a recvMsg. // r.readAdditional acts on that message and returns the necessary error. @@ -211,9 +211,9 @@ func (r *recvBufferReader) readHeaderClient(header []byte) (n int, err error) { // faster. r.closeStream(ContextErr(r.ctx.Err())) m := <-r.recv.get() - return r.readHeaderAdditional(m, header) + return r.readMessageHeaderAdditional(m, header) case m := <-r.recv.get(): - return r.readHeaderAdditional(m, header) + return r.readMessageHeaderAdditional(m, header) } } @@ -244,7 +244,7 @@ func (r *recvBufferReader) readClient(n int) (buf mem.Buffer, err error) { } } -func (r *recvBufferReader) readHeaderAdditional(m recvMsg, header []byte) (n int, err error) { +func (r *recvBufferReader) readMessageHeaderAdditional(m recvMsg, header []byte) (n int, err error) { r.recv.load() if m.err != nil { if m.buffer != nil { @@ -342,23 +342,22 @@ func (s *Stream) write(m recvMsg) { s.buf.put(m) } -// ReadHeader reads data into the provided header slice from the stream. It -// first checks if there was an error during a previous read operation and +// ReadMessageHeader reads data into the provided header slice from the stream. +// It first checks if there was an error during a previous read operation and // returns it if present. It then requests a read operation for the length of // the header. It continues to read from the stream until the entire header -// slice is filled or an error occurs. If an `io.EOF` error is encountered -// with partially read data, it is converted to `io.ErrUnexpectedEOF` to -// indicate an unexpected end of the stream. The method returns any error -// encountered during the read process or nil if the header was successfully -// read. -func (s *Stream) ReadHeader(header []byte) (err error) { +// slice is filled or an error occurs. If an `io.EOF` error is encountered with +// partially read data, it is converted to `io.ErrUnexpectedEOF` to indicate an +// unexpected end of the stream. The method returns any error encountered during +// the read process or nil if the header was successfully read. +func (s *Stream) ReadMessageHeader(header []byte) (err error) { // Don't request a read if there was an error earlier if er := s.trReader.er; er != nil { return er } s.requestRead(len(header)) for len(header) != 0 { - n, err := s.trReader.ReadHeader(header) + n, err := s.trReader.ReadMessageHeader(header) header = header[n:] if len(header) == 0 { err = nil @@ -374,7 +373,7 @@ func (s *Stream) ReadHeader(header []byte) (err error) { } // Read reads n bytes from the wire for this stream. -func (s *Stream) Read(n int) (data mem.BufferSlice, err error) { +func (s *Stream) read(n int) (data mem.BufferSlice, err error) { // Don't request a read if there was an error earlier if er := s.trReader.er; er != nil { return nil, er @@ -414,8 +413,8 @@ type transportReader struct { er error } -func (t *transportReader) ReadHeader(header []byte) (int, error) { - n, err := t.reader.ReadHeader(header) +func (t *transportReader) ReadMessageHeader(header []byte) (int, error) { + n, err := t.reader.ReadMessageHeader(header) if err != nil { t.er = err return 0, err @@ -509,9 +508,9 @@ type ConnectOptions struct { BufferPool mem.BufferPool } -// Options provides additional hints and information for message +// WriteOptions provides additional hints and information for message // transmission. -type Options struct { +type WriteOptions struct { // Last indicates whether this write is the last piece for // this stream. Last bool @@ -560,19 +559,9 @@ type ClientTransport interface { // It does not block. GracefulClose() - // Write sends the data for the given stream. A nil stream indicates - // the write is to be performed on the transport as a whole. - Write(s *ClientStream, hdr []byte, data mem.BufferSlice, opts *Options) error - // NewStream creates a Stream for an RPC. NewStream(ctx context.Context, callHdr *CallHdr) (*ClientStream, error) - // CloseStream clears the footprint of a stream when the stream is - // not needed any more. The err indicates the error incurred when - // CloseStream is called. Must be called when a stream is finished - // unless the associated transport is closing. - CloseStream(stream *ClientStream, err error) - // Error returns a channel that is closed when some I/O error // happens. Typically the caller should have a goroutine to monitor // this in order to take action (e.g., close the current transport @@ -591,12 +580,6 @@ type ClientTransport interface { // RemoteAddr returns the remote network address. RemoteAddr() net.Addr - - // IncrMsgSent increments the number of message sent through this transport. - IncrMsgSent() - - // IncrMsgRecv increments the number of message received through this transport. - IncrMsgRecv() } // ServerTransport is the common interface for all gRPC server-side transport @@ -608,18 +591,6 @@ type ServerTransport interface { // HandleStreams receives incoming streams using the given handler. HandleStreams(context.Context, func(*ServerStream)) - // WriteHeader sends the header metadata for the given stream. - // WriteHeader may not be called on all streams. - WriteHeader(s *ServerStream, md metadata.MD) error - - // Write sends the data for the given stream. - // Write may not be called on all streams. - Write(s *ServerStream, hdr []byte, data mem.BufferSlice, opts *Options) error - - // WriteStatus sends the status of a stream to the client. WriteStatus is - // the final call made on a stream and always occurs. - WriteStatus(s *ServerStream, st *status.Status) error - // Close tears down the transport. Once it is called, the transport // should not be accessed any more. All the pending streams and their // handlers will be terminated asynchronously. @@ -630,12 +601,14 @@ type ServerTransport interface { // Drain notifies the client this ServerTransport stops accepting new RPCs. Drain(debugData string) +} - // IncrMsgSent increments the number of message sent through this transport. - IncrMsgSent() - - // IncrMsgRecv increments the number of message received through this transport. - IncrMsgRecv() +type internalServerTransport interface { + ServerTransport + writeHeader(s *ServerStream, md metadata.MD) error + write(s *ServerStream, hdr []byte, data mem.BufferSlice, opts *WriteOptions) error + writeStatus(s *ServerStream, st *status.Status) error + incrMsgRecv() } // connectionErrorf creates an ConnectionError with the specified error description. diff --git a/internal/transport/transport_test.go b/internal/transport/transport_test.go index d315718d9572..c91757b3f96e 100644 --- a/internal/transport/transport_test.go +++ b/internal/transport/transport_test.go @@ -80,7 +80,7 @@ func newBufferSlice(b []byte) mem.BufferSlice { } func (s *Stream) readTo(p []byte) (int, error) { - data, err := s.Read(len(p)) + data, err := s.read(len(p)) defer data.Free() if err != nil { @@ -144,13 +144,13 @@ func (h *testStreamHandler) handleStream(t *testing.T, s *ServerStream) { } if !bytes.Equal(p, req) { t.Errorf("handleStream got %v, want %v", p, req) - h.t.WriteStatus(s, status.New(codes.Internal, "panic")) + s.WriteStatus(status.New(codes.Internal, "panic")) return } // send a response back to the client. - h.t.Write(s, nil, newBufferSlice(resp), &Options{}) + s.Write(nil, newBufferSlice(resp), &WriteOptions{}) // send the trailer to end the stream. - h.t.WriteStatus(s, status.New(codes.OK, "")) + s.WriteStatus(status.New(codes.OK, "")) } func (h *testStreamHandler) handleStreamPingPong(t *testing.T, s *ServerStream) { @@ -158,25 +158,25 @@ func (h *testStreamHandler) handleStreamPingPong(t *testing.T, s *ServerStream) for { if _, err := s.readTo(header); err != nil { if err == io.EOF { - h.t.WriteStatus(s, status.New(codes.OK, "")) + s.WriteStatus(status.New(codes.OK, "")) return } t.Errorf("Error on server while reading data header: %v", err) - h.t.WriteStatus(s, status.New(codes.Internal, "panic")) + s.WriteStatus(status.New(codes.Internal, "panic")) return } sz := binary.BigEndian.Uint32(header[1:]) msg := make([]byte, int(sz)) if _, err := s.readTo(msg); err != nil { t.Errorf("Error on server while reading message: %v", err) - h.t.WriteStatus(s, status.New(codes.Internal, "panic")) + s.WriteStatus(status.New(codes.Internal, "panic")) return } buf := make([]byte, sz+5) buf[0] = byte(0) binary.BigEndian.PutUint32(buf[1:], uint32(sz)) copy(buf[5:], msg) - h.t.Write(s, nil, newBufferSlice(buf), &Options{}) + s.Write(nil, newBufferSlice(buf), &WriteOptions{}) } } @@ -184,7 +184,7 @@ func (h *testStreamHandler) handleStreamMisbehave(t *testing.T, s *ServerStream) conn, ok := s.st.(*http2Server) if !ok { t.Errorf("Failed to convert %v to *http2Server", s.st) - h.t.WriteStatus(s, status.New(codes.Internal, "")) + s.WriteStatus(status.New(codes.Internal, "")) return } var sent int @@ -215,7 +215,7 @@ func (h *testStreamHandler) handleStreamMisbehave(t *testing.T, s *ServerStream) func (h *testStreamHandler) handleStreamEncodingRequiredStatus(s *ServerStream) { // raw newline is not accepted by http2 framer so it must be encoded. - h.t.WriteStatus(s, encodingTestStatus) + s.WriteStatus(encodingTestStatus) // Drain any remaining buffers from the stream since it was closed early. s.Read(math.MaxInt) } @@ -300,7 +300,7 @@ func (h *testStreamHandler) handleStreamDelayRead(t *testing.T, s *ServerStream) // This write will cause server to run out of stream level, // flow control and the other side won't send a window update // until that happens. - if err := h.t.Write(s, nil, newBufferSlice(resp), &Options{}); err != nil { + if err := s.Write(nil, newBufferSlice(resp), &WriteOptions{}); err != nil { t.Errorf("server Write got %v, want ", err) return } @@ -313,7 +313,7 @@ func (h *testStreamHandler) handleStreamDelayRead(t *testing.T, s *ServerStream) return } // send the trailer to end the stream. - if err := h.t.WriteStatus(s, status.New(codes.OK, "")); err != nil { + if err := s.WriteStatus(status.New(codes.OK, "")); err != nil { t.Errorf("server WriteStatus got %v, want ", err) return } @@ -535,7 +535,7 @@ func (s) TestInflightStreamClosing(t *testing.T) { }() // should unblock concurrent stream.Read - client.CloseStream(stream, serr) + stream.Close(serr) // wait for stream.Read error timeout := time.NewTimer(5 * time.Second) @@ -618,8 +618,8 @@ func (s) TestClientSendAndReceive(t *testing.T) { if s2.id != 3 { t.Fatalf("wrong stream id: %d", s2.id) } - opts := Options{Last: true} - if err := ct.Write(s1, nil, newBufferSlice(expectedRequest), &opts); err != nil && err != io.EOF { + opts := WriteOptions{Last: true} + if err := s1.Write(nil, newBufferSlice(expectedRequest), &opts); err != nil && err != io.EOF { t.Fatalf("failed to send data: %v", err) } p := make([]byte, len(expectedResponse)) @@ -655,8 +655,8 @@ func performOneRPC(ct ClientTransport) { if err != nil { return } - opts := Options{Last: true} - if err := ct.Write(s, []byte{}, newBufferSlice(expectedRequest), &opts); err == nil || err == io.EOF { + opts := WriteOptions{Last: true} + if err := s.Write([]byte{}, newBufferSlice(expectedRequest), &opts); err == nil || err == io.EOF { time.Sleep(5 * time.Millisecond) // The following s.Recv()'s could error out because the // underlying transport is gone. @@ -701,7 +701,7 @@ func (s) TestLargeMessage(t *testing.T) { if err != nil { t.Errorf("%v.NewStream(_, _) = _, %v, want _, ", ct, err) } - if err := ct.Write(s, []byte{}, newBufferSlice(expectedRequestLarge), &Options{Last: true}); err != nil && err != io.EOF { + if err := s.Write([]byte{}, newBufferSlice(expectedRequestLarge), &WriteOptions{Last: true}); err != nil && err != io.EOF { t.Errorf("%v.Write(_, _, _) = %v, want ", ct, err) } p := make([]byte, len(expectedResponseLarge)) @@ -792,7 +792,7 @@ func (s) TestLargeMessageWithDelayRead(t *testing.T) { // This write will cause client to run out of stream level, // flow control and the other side won't send a window update // until that happens. - if err := ct.Write(s, []byte{}, newBufferSlice(expectedRequestLarge), &Options{}); err != nil { + if err := s.Write([]byte{}, newBufferSlice(expectedRequestLarge), &WriteOptions{}); err != nil { t.Fatalf("write(_, _, _) = %v, want ", err) } p := make([]byte, len(expectedResponseLarge)) @@ -807,7 +807,7 @@ func (s) TestLargeMessageWithDelayRead(t *testing.T) { if _, err := s.readTo(p); err != nil || !bytes.Equal(p, expectedResponseLarge) { t.Fatalf("s.Read(_) = _, %v, want _, ", err) } - if err := ct.Write(s, []byte{}, newBufferSlice(expectedRequestLarge), &Options{Last: true}); err != nil { + if err := s.Write([]byte{}, newBufferSlice(expectedRequestLarge), &WriteOptions{Last: true}); err != nil { t.Fatalf("Write(_, _, _) = %v, want ", err) } if _, err = s.readTo(p); err != io.EOF { @@ -847,7 +847,7 @@ func (s) TestGracefulClose(t *testing.T) { outgoingHeader[0] = byte(0) binary.BigEndian.PutUint32(outgoingHeader[1:], uint32(len(msg))) incomingHeader := make([]byte, 5) - if err := ct.Write(s, outgoingHeader, newBufferSlice(msg), &Options{}); err != nil { + if err := s.Write(outgoingHeader, newBufferSlice(msg), &WriteOptions{}); err != nil { t.Fatalf("Error while writing: %v", err) } if _, err := s.readTo(incomingHeader); err != nil { @@ -879,7 +879,7 @@ func (s) TestGracefulClose(t *testing.T) { } // Confirm the existing stream still functions as expected. - ct.Write(s, nil, nil, &Options{Last: true}) + s.Write(nil, nil, &WriteOptions{Last: true}) if _, err := s.readTo(incomingHeader); err != io.EOF { t.Fatalf("Client expected EOF from the server. Got: %v", err) } @@ -904,12 +904,12 @@ func (s) TestLargeMessageSuspension(t *testing.T) { // stream.go to keep track of context timeout and call CloseStream. go func() { <-ctx.Done() - ct.CloseStream(s, ContextErr(ctx.Err())) + s.Close(ContextErr(ctx.Err())) }() // Write should not be done successfully due to flow control. msg := make([]byte, initialWindowSize*8) - ct.Write(s, nil, newBufferSlice(msg), &Options{}) - err = ct.Write(s, nil, newBufferSlice(msg), &Options{Last: true}) + s.Write(nil, newBufferSlice(msg), &WriteOptions{}) + err = s.Write(nil, newBufferSlice(msg), &WriteOptions{Last: true}) if err != errStreamDone { t.Fatalf("Write got %v, want io.EOF", err) } @@ -977,7 +977,7 @@ func (s) TestMaxStreams(t *testing.T) { }() // Close all the extra streams created and make sure the new stream is not created. for _, str := range slist { - ct.CloseStream(str, nil) + str.Close(nil) } select { case <-done: @@ -985,7 +985,7 @@ func (s) TestMaxStreams(t *testing.T) { default: } // Close the first stream created so that the new stream can finally be created. - ct.CloseStream(s, nil) + s.Close(nil) <-done ct.Close(fmt.Errorf("closed manually by test")) <-ct.writerDone @@ -1108,7 +1108,7 @@ func (s) TestClientConnDecoupledFromApplicationRead(t *testing.T) { t.Fatalf("Didn't find stream corresponding to client cstream.id: %v on the server", cstream1.id) } // Exhaust client's connection window. - if err := st.Write(sstream1, []byte{}, newBufferSlice(make([]byte, defaultWindowSize)), &Options{}); err != nil { + if err := sstream1.Write([]byte{}, newBufferSlice(make([]byte, defaultWindowSize)), &WriteOptions{}); err != nil { t.Fatalf("Server failed to write data. Err: %v", err) } notifyChan = make(chan struct{}) @@ -1133,7 +1133,7 @@ func (s) TestClientConnDecoupledFromApplicationRead(t *testing.T) { t.Fatalf("Didn't find stream corresponding to client cstream.id: %v on the server", cstream2.id) } // Server should be able to send data on the new stream, even though the client hasn't read anything on the first stream. - if err := st.Write(sstream2, []byte{}, newBufferSlice(make([]byte, defaultWindowSize)), &Options{}); err != nil { + if err := sstream2.Write([]byte{}, newBufferSlice(make([]byte, defaultWindowSize)), &WriteOptions{}); err != nil { t.Fatalf("Server failed to write data. Err: %v", err) } @@ -1179,7 +1179,7 @@ func (s) TestServerConnDecoupledFromApplicationRead(t *testing.T) { t.Fatalf("Failed to create 1st stream. Err: %v", err) } // Exhaust server's connection window. - if err := client.Write(cstream1, nil, newBufferSlice(make([]byte, defaultWindowSize)), &Options{Last: true}); err != nil { + if err := cstream1.Write(nil, newBufferSlice(make([]byte, defaultWindowSize)), &WriteOptions{Last: true}); err != nil { t.Fatalf("Client failed to write data. Err: %v", err) } //Client should be able to create another stream and send data on it. @@ -1187,7 +1187,7 @@ func (s) TestServerConnDecoupledFromApplicationRead(t *testing.T) { if err != nil { t.Fatalf("Failed to create 2nd stream. Err: %v", err) } - if err := client.Write(cstream2, nil, newBufferSlice(make([]byte, defaultWindowSize)), &Options{}); err != nil { + if err := cstream2.Write(nil, newBufferSlice(make([]byte, defaultWindowSize)), &WriteOptions{}); err != nil { t.Fatalf("Client failed to write data. Err: %v", err) } // Get the streams on server. @@ -1458,7 +1458,7 @@ func (s) TestClientWithMisbehavedServer(t *testing.T) { timer := time.NewTimer(time.Second * 5) go func() { // This go routine mimics the one in stream.go to call CloseStream. <-str.Done() - ct.CloseStream(str, nil) + str.Close(nil) }() select { case <-timer.C: @@ -1485,8 +1485,8 @@ func (s) TestEncodingRequiredStatus(t *testing.T) { if err != nil { return } - opts := Options{Last: true} - if err := ct.Write(s, nil, newBufferSlice(expectedRequest), &opts); err != nil && err != errStreamDone { + opts := WriteOptions{Last: true} + if err := s.Write(nil, newBufferSlice(expectedRequest), &opts); err != nil && err != errStreamDone { t.Fatalf("Failed to write the request: %v", err) } p := make([]byte, http2MaxFrameLen) @@ -1671,10 +1671,10 @@ func testFlowControlAccountCheck(t *testing.T, msgSize int, wc windowSizeConfig) buf := make([]byte, msgSize+5) buf[0] = byte(0) binary.BigEndian.PutUint32(buf[1:], uint32(msgSize)) - opts := Options{} + opts := WriteOptions{} header := make([]byte, 5) for i := 1; i <= 5; i++ { - if err := client.Write(stream, nil, newBufferSlice(buf), &opts); err != nil { + if err := stream.Write(nil, newBufferSlice(buf), &opts); err != nil { t.Errorf("Error on client while writing message %v on stream %v: %v", i, stream.id, err) return } @@ -1714,7 +1714,7 @@ func testFlowControlAccountCheck(t *testing.T, msgSize int, wc windowSizeConfig) st.mu.Unlock() // Close all streams for _, stream := range clientStreams { - client.Write(stream, nil, nil, &Options{Last: true}) + stream.Write(nil, nil, &WriteOptions{Last: true}) if _, err := stream.readTo(make([]byte, 5)); err != io.EOF { t.Fatalf("Client expected an EOF from the server. Got: %v", err) } @@ -2230,7 +2230,7 @@ func (s) TestWriteHeaderConnectionError(t *testing.T) { <-serverTransport.done // Write header on a closed server transport. - err = serverTransport.WriteHeader(sstream, metadata.MD{}) + err = sstream.SendHeader(metadata.MD{}) st := status.Convert(err) if st.Code() != codes.Unavailable { t.Fatalf("WriteHeader() failed with status code %s, want %s", st.Code(), codes.Unavailable) @@ -2277,13 +2277,13 @@ func runPingPongTest(t *testing.T, msgSize int) { outgoingHeader := make([]byte, 5) outgoingHeader[0] = byte(0) binary.BigEndian.PutUint32(outgoingHeader[1:], uint32(msgSize)) - opts := &Options{} + opts := &WriteOptions{} incomingHeader := make([]byte, 5) ctx, cancel = context.WithTimeout(ctx, 10*time.Millisecond) defer cancel() for ctx.Err() == nil { - if err := client.Write(stream, outgoingHeader, newBufferSlice(msg), opts); err != nil { + if err := stream.Write(outgoingHeader, newBufferSlice(msg), opts); err != nil { t.Fatalf("Error on client while writing message. Err: %v", err) } if _, err := stream.readTo(incomingHeader); err != nil { @@ -2296,7 +2296,7 @@ func runPingPongTest(t *testing.T, msgSize int) { } } - client.Write(stream, nil, nil, &Options{Last: true}) + stream.Write(nil, nil, &WriteOptions{Last: true}) if _, err := stream.readTo(incomingHeader); err != io.EOF { t.Fatalf("Client expected EOF from the server. Got: %v", err) } @@ -2934,7 +2934,7 @@ func (s) TestClientCloseReturnsEarlyWhenGoAwayWriteHangs(t *testing.T) { // TestReadHeaderMultipleBuffers tests the stream when the gRPC headers are // split across multiple buffers. It verifies that the reporting of the // number of bytes read for flow control is correct. -func (s) TestReadHeaderMultipleBuffers(t *testing.T) { +func (s) TestReadMessageHeaderMultipleBuffers(t *testing.T) { headerLen := 5 recvBuffer := newRecvBuffer() recvBuffer.put(recvMsg{buffer: make(mem.SliceBuffer, 3)}) @@ -2953,7 +2953,7 @@ func (s) TestReadHeaderMultipleBuffers(t *testing.T) { } header := make([]byte, headerLen) - err := s.ReadHeader(header) + err := s.ReadMessageHeader(header) if err != nil { t.Fatalf("ReadHeader(%v) = %v", header, err) } diff --git a/rpc_util.go b/rpc_util.go index 4f7fbca3484e..9fac2b08b48b 100644 --- a/rpc_util.go +++ b/rpc_util.go @@ -622,7 +622,7 @@ func (pf payloadFormat) isCompressed() bool { } type streamReader interface { - ReadHeader(header []byte) error + ReadMessageHeader(header []byte) error Read(n int) (mem.BufferSlice, error) } @@ -656,7 +656,7 @@ type parser struct { // that the underlying streamReader must not return an incompatible // error. func (p *parser) recvMsg(maxReceiveMessageSize int) (payloadFormat, mem.BufferSlice, error) { - err := p.r.ReadHeader(p.header[:]) + err := p.r.ReadMessageHeader(p.header[:]) if err != nil { return 0, nil, err } @@ -664,9 +664,6 @@ func (p *parser) recvMsg(maxReceiveMessageSize int) (payloadFormat, mem.BufferSl pf := payloadFormat(p.header[0]) length := binary.BigEndian.Uint32(p.header[1:]) - if length == 0 { - return pf, nil, nil - } if int64(length) > int64(maxInt) { return 0, nil, status.Errorf(codes.ResourceExhausted, "grpc: received message larger than max length allowed on current machine (%d vs. %d)", length, maxInt) } diff --git a/rpc_util_test.go b/rpc_util_test.go index ca7990a005f1..94f50bc24ade 100644 --- a/rpc_util_test.go +++ b/rpc_util_test.go @@ -40,7 +40,7 @@ type fullReader struct { data []byte } -func (f *fullReader) ReadHeader(header []byte) error { +func (f *fullReader) ReadMessageHeader(header []byte) error { buf, err := f.Read(len(header)) defer buf.Free() if err != nil { @@ -52,6 +52,10 @@ func (f *fullReader) ReadHeader(header []byte) error { } func (f *fullReader) Read(n int) (mem.BufferSlice, error) { + if n == 0 { + return nil, nil + } + if len(f.data) == 0 { return nil, io.EOF } diff --git a/server.go b/server.go index 05f0c6e2ad79..16065a027ae8 100644 --- a/server.go +++ b/server.go @@ -1137,7 +1137,7 @@ func (s *Server) incrCallsFailed() { s.channelz.ServerMetrics.CallsFailed.Add(1) } -func (s *Server) sendResponse(ctx context.Context, t transport.ServerTransport, stream *transport.ServerStream, msg any, cp Compressor, opts *transport.Options, comp encoding.Compressor) error { +func (s *Server) sendResponse(ctx context.Context, stream *transport.ServerStream, msg any, cp Compressor, opts *transport.WriteOptions, comp encoding.Compressor) error { data, err := encode(s.getCodec(stream.ContentSubtype()), msg) if err != nil { channelz.Error(logger, s.channelz, "grpc: server failed to encode response: ", err) @@ -1166,7 +1166,7 @@ func (s *Server) sendResponse(ctx context.Context, t transport.ServerTransport, if payloadLen > s.opts.maxSendMessageSize { return status.Errorf(codes.ResourceExhausted, "grpc: trying to send message larger than max (%d vs. %d)", payloadLen, s.opts.maxSendMessageSize) } - err = t.Write(stream, hdr, payload, opts) + err = stream.Write(hdr, payload, opts) if err == nil { if len(s.opts.statsHandlers) != 0 { for _, sh := range s.opts.statsHandlers { @@ -1213,7 +1213,7 @@ func getChainUnaryHandler(interceptors []UnaryServerInterceptor, curr int, info } } -func (s *Server) processUnaryRPC(ctx context.Context, t transport.ServerTransport, stream *transport.ServerStream, info *serviceInfo, md *MethodDesc, trInfo *traceInfo) (err error) { +func (s *Server) processUnaryRPC(ctx context.Context, stream *transport.ServerStream, info *serviceInfo, md *MethodDesc, trInfo *traceInfo) (err error) { shs := s.opts.statsHandlers if len(shs) != 0 || trInfo != nil || channelz.IsOn() { if channelz.IsOn() { @@ -1321,7 +1321,7 @@ func (s *Server) processUnaryRPC(ctx context.Context, t transport.ServerTranspor decomp = encoding.GetCompressor(rc) if decomp == nil { st := status.Newf(codes.Unimplemented, "grpc: Decompressor is not installed for grpc-encoding %q", rc) - t.WriteStatus(stream, st) + stream.WriteStatus(st) return st.Err() } } @@ -1355,15 +1355,12 @@ func (s *Server) processUnaryRPC(ctx context.Context, t transport.ServerTranspor d, err := recvAndDecompress(&parser{r: stream, bufferPool: s.opts.bufferPool}, stream, dc, s.opts.maxReceiveMessageSize, payInfo, decomp, true) if err != nil { - if e := t.WriteStatus(stream, status.Convert(err)); e != nil { + if e := stream.WriteStatus(status.Convert(err)); e != nil { channelz.Warningf(logger, s.channelz, "grpc: Server.processUnaryRPC failed to write status: %v", e) } return err } defer d.Free() - if channelz.IsOn() { - t.IncrMsgRecv() - } df := func(v any) error { if err := s.getCodec(stream.ContentSubtype()).Unmarshal(d, v); err != nil { return status.Errorf(codes.Internal, "grpc: error unmarshalling request: %v", err) @@ -1405,7 +1402,7 @@ func (s *Server) processUnaryRPC(ctx context.Context, t transport.ServerTranspor trInfo.tr.LazyLog(stringer(appStatus.Message()), true) trInfo.tr.SetError() } - if e := t.WriteStatus(stream, appStatus); e != nil { + if e := stream.WriteStatus(appStatus); e != nil { channelz.Warningf(logger, s.channelz, "grpc: Server.processUnaryRPC failed to write status: %v", e) } if len(binlogs) != 0 { @@ -1432,20 +1429,20 @@ func (s *Server) processUnaryRPC(ctx context.Context, t transport.ServerTranspor if trInfo != nil { trInfo.tr.LazyLog(stringer("OK"), false) } - opts := &transport.Options{Last: true} + opts := &transport.WriteOptions{Last: true} // Server handler could have set new compressor by calling SetSendCompressor. // In case it is set, we need to use it for compressing outbound message. if stream.SendCompress() != sendCompressorName { comp = encoding.GetCompressor(stream.SendCompress()) } - if err := s.sendResponse(ctx, t, stream, reply, cp, opts, comp); err != nil { + if err := s.sendResponse(ctx, stream, reply, cp, opts, comp); err != nil { if err == io.EOF { // The entire stream is done (for unary RPC only). return err } if sts, ok := status.FromError(err); ok { - if e := t.WriteStatus(stream, sts); e != nil { + if e := stream.WriteStatus(sts); e != nil { channelz.Warningf(logger, s.channelz, "grpc: Server.processUnaryRPC failed to write status: %v", e) } } else { @@ -1485,9 +1482,6 @@ func (s *Server) processUnaryRPC(ctx context.Context, t transport.ServerTranspor binlog.Log(ctx, sm) } } - if channelz.IsOn() { - t.IncrMsgSent() - } if trInfo != nil { trInfo.tr.LazyLog(&payload{sent: true, msg: reply}, true) } @@ -1503,7 +1497,7 @@ func (s *Server) processUnaryRPC(ctx context.Context, t transport.ServerTranspor binlog.Log(ctx, st) } } - return t.WriteStatus(stream, statusOK) + return stream.WriteStatus(statusOK) } // chainStreamServerInterceptors chains all stream server interceptors into one. @@ -1542,7 +1536,7 @@ func getChainStreamHandler(interceptors []StreamServerInterceptor, curr int, inf } } -func (s *Server) processStreamingRPC(ctx context.Context, t transport.ServerTransport, stream *transport.ServerStream, info *serviceInfo, sd *StreamDesc, trInfo *traceInfo) (err error) { +func (s *Server) processStreamingRPC(ctx context.Context, stream *transport.ServerStream, info *serviceInfo, sd *StreamDesc, trInfo *traceInfo) (err error) { if channelz.IsOn() { s.incrCallsStarted() } @@ -1562,7 +1556,6 @@ func (s *Server) processStreamingRPC(ctx context.Context, t transport.ServerTran ctx = NewContextWithServerTransportStream(ctx, stream) ss := &serverStream{ ctx: ctx, - t: t, s: stream, p: &parser{r: stream, bufferPool: s.opts.bufferPool}, codec: s.getCodec(stream.ContentSubtype()), @@ -1649,7 +1642,7 @@ func (s *Server) processStreamingRPC(ctx context.Context, t transport.ServerTran ss.decomp = encoding.GetCompressor(rc) if ss.decomp == nil { st := status.Newf(codes.Unimplemented, "grpc: Decompressor is not installed for grpc-encoding %q", rc) - t.WriteStatus(ss.s, st) + ss.s.WriteStatus(st) return st.Err() } } @@ -1718,7 +1711,7 @@ func (s *Server) processStreamingRPC(ctx context.Context, t transport.ServerTran binlog.Log(ctx, st) } } - t.WriteStatus(ss.s, appStatus) + ss.s.WriteStatus(appStatus) // TODO: Should we log an error from WriteStatus here and below? return appErr } @@ -1736,7 +1729,7 @@ func (s *Server) processStreamingRPC(ctx context.Context, t transport.ServerTran binlog.Log(ctx, st) } } - return t.WriteStatus(ss.s, statusOK) + return ss.s.WriteStatus(statusOK) } func (s *Server) handleStream(t transport.ServerTransport, stream *transport.ServerStream) { @@ -1769,7 +1762,7 @@ func (s *Server) handleStream(t transport.ServerTransport, stream *transport.Ser ti.tr.SetError() } errDesc := fmt.Sprintf("malformed method name: %q", stream.Method()) - if err := t.WriteStatus(stream, status.New(codes.Unimplemented, errDesc)); err != nil { + if err := stream.WriteStatus(status.New(codes.Unimplemented, errDesc)); err != nil { if ti != nil { ti.tr.LazyLog(&fmtStringer{"%v", []any{err}}, true) ti.tr.SetError() @@ -1806,17 +1799,17 @@ func (s *Server) handleStream(t transport.ServerTransport, stream *transport.Ser srv, knownService := s.services[service] if knownService { if md, ok := srv.methods[method]; ok { - s.processUnaryRPC(ctx, t, stream, srv, md, ti) + s.processUnaryRPC(ctx, stream, srv, md, ti) return } if sd, ok := srv.streams[method]; ok { - s.processStreamingRPC(ctx, t, stream, srv, sd, ti) + s.processStreamingRPC(ctx, stream, srv, sd, ti) return } } // Unknown service, or known server unknown method. if unknownDesc := s.opts.unknownStreamDesc; unknownDesc != nil { - s.processStreamingRPC(ctx, t, stream, nil, unknownDesc, ti) + s.processStreamingRPC(ctx, stream, nil, unknownDesc, ti) return } var errDesc string @@ -1829,7 +1822,7 @@ func (s *Server) handleStream(t transport.ServerTransport, stream *transport.Ser ti.tr.LazyPrintf("%s", errDesc) ti.tr.SetError() } - if err := t.WriteStatus(stream, status.New(codes.Unimplemented, errDesc)); err != nil { + if err := stream.WriteStatus(status.New(codes.Unimplemented, errDesc)); err != nil { if ti != nil { ti.tr.LazyLog(&fmtStringer{"%v", []any{err}}, true) ti.tr.SetError() diff --git a/stream.go b/stream.go index 722209cc6670..6d10d0ac8713 100644 --- a/stream.go +++ b/stream.go @@ -993,7 +993,7 @@ func (cs *clientStream) CloseSend() error { } cs.sentLast = true op := func(a *csAttempt) error { - a.t.Write(a.s, nil, nil, &transport.Options{Last: true}) + a.s.Write(nil, nil, &transport.WriteOptions{Last: true}) // Always return nil; io.EOF is the only error that might make sense // instead, but there is no need to signal the client to call RecvMsg // as the only use left for the stream after CloseSend is to call @@ -1085,7 +1085,7 @@ func (a *csAttempt) sendMsg(m any, hdr []byte, payld mem.BufferSlice, dataLength } a.mu.Unlock() } - if err := a.t.Write(a.s, hdr, payld, &transport.Options{Last: !cs.desc.ClientStreams}); err != nil { + if err := a.s.Write(hdr, payld, &transport.WriteOptions{Last: !cs.desc.ClientStreams}); err != nil { if !cs.desc.ClientStreams { // For non-client-streaming RPCs, we return nil instead of EOF on error // because the generated code requires it. finish is not called; RecvMsg() @@ -1099,9 +1099,6 @@ func (a *csAttempt) sendMsg(m any, hdr []byte, payld mem.BufferSlice, dataLength sh.HandleRPC(a.ctx, outPayload(true, m, dataLength, payloadLength, time.Now())) } } - if channelz.IsOn() { - a.t.IncrMsgSent() - } return nil } @@ -1155,9 +1152,6 @@ func (a *csAttempt) recvMsg(m any, payInfo *payloadInfo) (err error) { Length: payInfo.uncompressedBytes.Len(), }) } - if channelz.IsOn() { - a.t.IncrMsgRecv() - } if cs.desc.ServerStreams { // Subsequent messages should be received by subsequent RecvMsg calls. return nil @@ -1185,7 +1179,7 @@ func (a *csAttempt) finish(err error) { } var tr metadata.MD if a.s != nil { - a.t.CloseStream(a.s, err) + a.s.Close(err) tr = a.s.Trailer() } @@ -1382,7 +1376,7 @@ func (as *addrConnStream) CloseSend() error { } as.sentLast = true - as.t.Write(as.s, nil, nil, &transport.Options{Last: true}) + as.s.Write(nil, nil, &transport.WriteOptions{Last: true}) // Always return nil; io.EOF is the only error that might make sense // instead, but there is no need to signal the client to call RecvMsg // as the only use left for the stream after CloseSend is to call @@ -1432,7 +1426,7 @@ func (as *addrConnStream) SendMsg(m any) (err error) { return status.Errorf(codes.ResourceExhausted, "trying to send message larger than max (%d vs. %d)", payload.Len(), *as.callInfo.maxSendMessageSize) } - if err := as.t.Write(as.s, hdr, payload, &transport.Options{Last: !as.desc.ClientStreams}); err != nil { + if err := as.s.Write(hdr, payload, &transport.WriteOptions{Last: !as.desc.ClientStreams}); err != nil { if !as.desc.ClientStreams { // For non-client-streaming RPCs, we return nil instead of EOF on error // because the generated code requires it. finish is not called; RecvMsg() @@ -1442,9 +1436,6 @@ func (as *addrConnStream) SendMsg(m any) (err error) { return io.EOF } - if channelz.IsOn() { - as.t.IncrMsgSent() - } return nil } @@ -1482,9 +1473,6 @@ func (as *addrConnStream) RecvMsg(m any) (err error) { return toRPCErr(err) } - if channelz.IsOn() { - as.t.IncrMsgRecv() - } if as.desc.ServerStreams { // Subsequent messages should be received by subsequent RecvMsg calls. return nil @@ -1512,7 +1500,7 @@ func (as *addrConnStream) finish(err error) { err = nil } if as.s != nil { - as.t.CloseStream(as.s, err) + as.s.Close(err) } if err != nil { @@ -1579,7 +1567,6 @@ type ServerStream interface { // serverStream implements a server side Stream. type serverStream struct { ctx context.Context - t transport.ServerTransport s *transport.ServerStream p *parser codec baseCodec @@ -1630,7 +1617,7 @@ func (ss *serverStream) SendHeader(md metadata.MD) error { return status.Error(codes.Internal, err.Error()) } - err = ss.t.WriteHeader(ss.s, md) + err = ss.s.SendHeader(md) if len(ss.binlogs) != 0 && !ss.serverHeaderBinlogged { h, _ := ss.s.Header() sh := &binarylog.ServerHeader{ @@ -1670,7 +1657,7 @@ func (ss *serverStream) SendMsg(m any) (err error) { } if err != nil && err != io.EOF { st, _ := status.FromError(toRPCErr(err)) - ss.t.WriteStatus(ss.s, st) + ss.s.WriteStatus(st) // Non-user specified status was sent out. This should be an error // case (as a server side Cancel maybe). // @@ -1678,9 +1665,6 @@ func (ss *serverStream) SendMsg(m any) (err error) { // status from the service handler, we will log that error instead. // This behavior is similar to an interceptor. } - if channelz.IsOn() && err == nil { - ss.t.IncrMsgSent() - } }() // Server handler could have set new compressor by calling SetSendCompressor. @@ -1712,7 +1696,7 @@ func (ss *serverStream) SendMsg(m any) (err error) { if payloadLen > ss.maxSendMessageSize { return status.Errorf(codes.ResourceExhausted, "trying to send message larger than max (%d vs. %d)", payloadLen, ss.maxSendMessageSize) } - if err := ss.t.Write(ss.s, hdr, payload, &transport.Options{Last: false}); err != nil { + if err := ss.s.Write(hdr, payload, &transport.WriteOptions{Last: false}); err != nil { return toRPCErr(err) } @@ -1758,7 +1742,7 @@ func (ss *serverStream) RecvMsg(m any) (err error) { } if err != nil && err != io.EOF { st, _ := status.FromError(toRPCErr(err)) - ss.t.WriteStatus(ss.s, st) + ss.s.WriteStatus(st) // Non-user specified status was sent out. This should be an error // case (as a server side Cancel maybe). // @@ -1766,9 +1750,6 @@ func (ss *serverStream) RecvMsg(m any) (err error) { // status from the service handler, we will log that error instead. // This behavior is similar to an interceptor. } - if channelz.IsOn() && err == nil { - ss.t.IncrMsgRecv() - } }() var payInfo *payloadInfo if len(ss.statsHandler) != 0 || len(ss.binlogs) != 0 { From 87f0254f112632b1745897553e48aff5884c9217 Mon Sep 17 00:00:00 2001 From: Purnesh Dixit Date: Fri, 22 Nov 2024 00:45:02 +0530 Subject: [PATCH 02/57] xdsclient: fix new watcher hang when registering for removed resource (#7853) --- xds/internal/xdsclient/authority.go | 5 ++ .../xdsclient/tests/lds_watchers_test.go | 81 +++++++++++++++++++ 2 files changed, 86 insertions(+) diff --git a/xds/internal/xdsclient/authority.go b/xds/internal/xdsclient/authority.go index 27abb64ef6d5..04bd278d2c47 100644 --- a/xds/internal/xdsclient/authority.go +++ b/xds/internal/xdsclient/authority.go @@ -641,6 +641,11 @@ func (a *authority) watchResource(rType xdsresource.Type, resourceName string, w resource := state.cache a.watcherCallbackSerializer.TrySchedule(func(context.Context) { watcher.OnUpdate(resource, func() {}) }) } + // If the metadata field is updated to indicate that the management + // server does not have this resource, notify the new watcher. + if state.md.Status == xdsresource.ServiceStatusNotExist { + a.watcherCallbackSerializer.TrySchedule(func(context.Context) { watcher.OnResourceDoesNotExist(func() {}) }) + } cleanup = a.unwatchResource(rType, resourceName, watcher) }, func() { if a.logger.V(2) { diff --git a/xds/internal/xdsclient/tests/lds_watchers_test.go b/xds/internal/xdsclient/tests/lds_watchers_test.go index 2ea2c50ce18b..38e1f1760383 100644 --- a/xds/internal/xdsclient/tests/lds_watchers_test.go +++ b/xds/internal/xdsclient/tests/lds_watchers_test.go @@ -871,6 +871,87 @@ func (s) TestLDSWatch_ResourceRemoved(t *testing.T) { } } +// TestLDSWatch_NewWatcherForRemovedResource covers the case where a new +// watcher registers for a resource that has been removed. The test verifies +// the following scenarios: +// 1. When a resource is deleted by the management server, any active +// watchers of that resource should be notified with a "resource removed" +// error through their watch callback. +// 2. If a new watcher attempts to register for a resource that has already +// been deleted, its watch callback should be immediately invoked with a +// "resource removed" error. +func (s) TestLDSWatch_NewWatcherForRemovedResource(t *testing.T) { + mgmtServer := e2e.StartManagementServer(t, e2e.ManagementServerOptions{}) + + nodeID := uuid.New().String() + bc := e2e.DefaultBootstrapContents(t, nodeID, mgmtServer.Address) + + // Create an xDS client with the above bootstrap contents. + client, close, err := xdsclient.NewForTesting(xdsclient.OptionsForTesting{ + Name: t.Name(), + Contents: bc, + }) + if err != nil { + t.Fatalf("Failed to create xDS client: %v", err) + } + defer close() + + // Register watch for the listener resource and have the + // callbacks push the received updates on to a channel. + lw1 := newListenerWatcher() + ldsCancel1 := xdsresource.WatchListener(client, ldsName, lw1) + defer ldsCancel1() + + // Configure the management server to return listener resource, + // corresponding to the registered watch. + resource := e2e.UpdateOptions{ + NodeID: nodeID, + Listeners: []*v3listenerpb.Listener{e2e.DefaultClientListener(ldsName, rdsName)}, + SkipValidation: true, + } + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + if err := mgmtServer.Update(ctx, resource); err != nil { + t.Fatalf("Failed to update management server with resource: %v, err: %v", resource, err) + } + + // Verify the contents of the received update for existing watch. + wantUpdate := listenerUpdateErrTuple{ + update: xdsresource.ListenerUpdate{ + RouteConfigName: rdsName, + HTTPFilters: []xdsresource.HTTPFilter{{Name: "router"}}, + }, + } + if err := verifyListenerUpdate(ctx, lw1.updateCh, wantUpdate); err != nil { + t.Fatal(err) + } + + // Remove the listener resource on the management server. + resource = e2e.UpdateOptions{ + NodeID: nodeID, + Listeners: []*v3listenerpb.Listener{}, + SkipValidation: true, + } + if err := mgmtServer.Update(ctx, resource); err != nil { + t.Fatalf("Failed to update management server with resource: %v, err: %v", resource, err) + } + + // The existing watcher should receive a resource removed error. + updateError := listenerUpdateErrTuple{err: xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "")} + if err := verifyListenerUpdate(ctx, lw1.updateCh, updateError); err != nil { + t.Fatal(err) + } + + // New watchers attempting to register for a deleted resource should also + // receive a "resource removed" error. + lw2 := newListenerWatcher() + ldsCancel2 := xdsresource.WatchListener(client, ldsName, lw2) + defer ldsCancel2() + if err := verifyListenerUpdate(ctx, lw2.updateCh, updateError); err != nil { + t.Fatal(err) + } +} + // TestLDSWatch_NACKError covers the case where an update from the management // server is NACK'ed by the xdsclient. The test verifies that the error is // propagated to the watcher. From 44a5eb9231ec6e753dab5ded7cda6f81788fc3f7 Mon Sep 17 00:00:00 2001 From: Purnesh Dixit Date: Fri, 22 Nov 2024 01:02:44 +0530 Subject: [PATCH 03/57] xdsclient: fix new watcher to get both old good update and nack error (if exist) from the cache (#7851) --- xds/internal/xdsclient/authority.go | 11 +- .../xdsclient/tests/lds_watchers_test.go | 198 ++++++++++++++++-- 2 files changed, 187 insertions(+), 22 deletions(-) diff --git a/xds/internal/xdsclient/authority.go b/xds/internal/xdsclient/authority.go index 04bd278d2c47..24673a8d9077 100644 --- a/xds/internal/xdsclient/authority.go +++ b/xds/internal/xdsclient/authority.go @@ -633,7 +633,8 @@ func (a *authority) watchResource(rType xdsresource.Type, resourceName string, w // Always add the new watcher to the set of watchers. state.watchers[watcher] = true - // If we have a cached copy of the resource, notify the new watcher. + // If we have a cached copy of the resource, notify the new watcher + // immediately. if state.cache != nil { if a.logger.V(2) { a.logger.Infof("Resource type %q with resource name %q found in cache: %s", rType.TypeName(), resourceName, state.cache.ToJSON()) @@ -641,6 +642,14 @@ func (a *authority) watchResource(rType xdsresource.Type, resourceName string, w resource := state.cache a.watcherCallbackSerializer.TrySchedule(func(context.Context) { watcher.OnUpdate(resource, func() {}) }) } + // If last update was NACK'd, notify the new watcher of error + // immediately as well. + if state.md.Status == xdsresource.ServiceStatusNACKed { + if a.logger.V(2) { + a.logger.Infof("Resource type %q with resource name %q was NACKed: %s", rType.TypeName(), resourceName, state.cache.ToJSON()) + } + a.watcherCallbackSerializer.TrySchedule(func(context.Context) { watcher.OnError(state.md.ErrState.Err, func() {}) }) + } // If the metadata field is updated to indicate that the management // server does not have this resource, notify the new watcher. if state.md.Status == xdsresource.ServiceStatusNotExist { diff --git a/xds/internal/xdsclient/tests/lds_watchers_test.go b/xds/internal/xdsclient/tests/lds_watchers_test.go index 38e1f1760383..7b49b9b17b74 100644 --- a/xds/internal/xdsclient/tests/lds_watchers_test.go +++ b/xds/internal/xdsclient/tests/lds_watchers_test.go @@ -71,22 +71,47 @@ func newListenerWatcher() *listenerWatcher { return &listenerWatcher{updateCh: testutils.NewChannel()} } -func (cw *listenerWatcher) OnUpdate(update *xdsresource.ListenerResourceData, onDone xdsresource.OnDoneFunc) { - cw.updateCh.Send(listenerUpdateErrTuple{update: update.Resource}) +func (lw *listenerWatcher) OnUpdate(update *xdsresource.ListenerResourceData, onDone xdsresource.OnDoneFunc) { + lw.updateCh.Send(listenerUpdateErrTuple{update: update.Resource}) onDone() } -func (cw *listenerWatcher) OnError(err error, onDone xdsresource.OnDoneFunc) { +func (lw *listenerWatcher) OnError(err error, onDone xdsresource.OnDoneFunc) { // When used with a go-control-plane management server that continuously // resends resources which are NACKed by the xDS client, using a `Replace()` // here and in OnResourceDoesNotExist() simplifies tests which will have // access to the most recently received error. - cw.updateCh.Replace(listenerUpdateErrTuple{err: err}) + lw.updateCh.Replace(listenerUpdateErrTuple{err: err}) onDone() } -func (cw *listenerWatcher) OnResourceDoesNotExist(onDone xdsresource.OnDoneFunc) { - cw.updateCh.Replace(listenerUpdateErrTuple{err: xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "Listener not found in received response")}) +func (lw *listenerWatcher) OnResourceDoesNotExist(onDone xdsresource.OnDoneFunc) { + lw.updateCh.Replace(listenerUpdateErrTuple{err: xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "Listener not found in received response")}) + onDone() +} + +type listenerWatcherMultiple struct { + updateCh *testutils.Channel +} + +// TODO: delete this once `newListenerWatcher` is modified to handle multiple +// updates (https://github.com/grpc/grpc-go/issues/7864). +func newListenerWatcherMultiple(size int) *listenerWatcherMultiple { + return &listenerWatcherMultiple{updateCh: testutils.NewChannelWithSize(size)} +} + +func (lw *listenerWatcherMultiple) OnUpdate(update *xdsresource.ListenerResourceData, onDone xdsresource.OnDoneFunc) { + lw.updateCh.Send(listenerUpdateErrTuple{update: update.Resource}) + onDone() +} + +func (lw *listenerWatcherMultiple) OnError(err error, onDone xdsresource.OnDoneFunc) { + lw.updateCh.Send(listenerUpdateErrTuple{err: err}) + onDone() +} + +func (lw *listenerWatcherMultiple) OnResourceDoesNotExist(onDone xdsresource.OnDoneFunc) { + lw.updateCh.Send(listenerUpdateErrTuple{err: xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "Listener not found in received response")}) onDone() } @@ -155,6 +180,18 @@ func verifyListenerUpdate(ctx context.Context, updateCh *testutils.Channel, want return nil } +func verifyUnknownListenerError(ctx context.Context, updateCh *testutils.Channel, wantErr string) error { + u, err := updateCh.Receive(ctx) + if err != nil { + return fmt.Errorf("timeout when waiting for a listener error from the management server: %v", err) + } + gotErr := u.(listenerUpdateErrTuple).err + if gotErr == nil || !strings.Contains(gotErr.Error(), wantErr) { + return fmt.Errorf("update received with error: %v, want %q", gotErr, wantErr) + } + return nil +} + // TestLDSWatch covers the case where a single watcher exists for a single // listener resource. The test verifies the following scenarios: // 1. An update from the management server containing the resource being @@ -953,8 +990,9 @@ func (s) TestLDSWatch_NewWatcherForRemovedResource(t *testing.T) { } // TestLDSWatch_NACKError covers the case where an update from the management -// server is NACK'ed by the xdsclient. The test verifies that the error is -// propagated to the watcher. +// server is NACKed by the xdsclient. The test verifies that the error is +// propagated to the existing watcher. After NACK, if a new watcher registers +// for the resource, error is propagated to the new watcher as well. func (s) TestLDSWatch_NACKError(t *testing.T) { mgmtServer := e2e.StartManagementServer(t, e2e.ManagementServerOptions{}) @@ -992,19 +1030,141 @@ func (s) TestLDSWatch_NACKError(t *testing.T) { } // Verify that the expected error is propagated to the watcher. - u, err := lw.updateCh.Receive(ctx) + // Verify that the expected error is propagated to the existing watcher. + if err := verifyUnknownListenerError(ctx, lw.updateCh, wantListenerNACKErr); err != nil { + t.Fatal(err) + } + + // Verify that the expected error is propagated to the new watcher as well. + lw2 := newListenerWatcher() + ldsCancel2 := xdsresource.WatchListener(client, ldsName, lw2) + defer ldsCancel2() + // Verify that the expected error is propagated to the existing watcher. + if err := verifyUnknownListenerError(ctx, lw2.updateCh, wantListenerNACKErr); err != nil { + t.Fatal(err) + } +} + +// TestLDSWatch_ResourceCaching_WithNACKError covers the case where a watch is +// registered for a resource which is already present in the cache with an old +// good update as well as latest NACK error. The test verifies that new watcher +// receives both good update and error without a new resource request being +// sent to the management server. +func TestLDSWatch_ResourceCaching_NACKError(t *testing.T) { + firstRequestReceived := false + firstAckReceived := grpcsync.NewEvent() + secondAckReceived := grpcsync.NewEvent() + secondRequestReceived := grpcsync.NewEvent() + + mgmtServer := e2e.StartManagementServer(t, e2e.ManagementServerOptions{ + OnStreamRequest: func(id int64, req *v3discoverypb.DiscoveryRequest) error { + // The first request has an empty version string. + if !firstRequestReceived && req.GetVersionInfo() == "" { + firstRequestReceived = true + return nil + } + // The first ack has a non-empty version string. + if !firstAckReceived.HasFired() && req.GetVersionInfo() != "" { + firstAckReceived.Fire() + return nil + } + // The second ack has a non-empty version string. + if !secondAckReceived.HasFired() && req.GetVersionInfo() != "" { + secondAckReceived.Fire() + return nil + } + // Any requests after the first request and two acks, are not expected. + secondRequestReceived.Fire() + return nil + }, + }) + + nodeID := uuid.New().String() + bc := e2e.DefaultBootstrapContents(t, nodeID, mgmtServer.Address) + + // Create an xDS client with the above bootstrap contents. + client, close, err := xdsclient.NewForTesting(xdsclient.OptionsForTesting{ + Name: t.Name(), + Contents: bc, + }) if err != nil { - t.Fatalf("timeout when waiting for a listener resource from the management server: %v", err) + t.Fatalf("Failed to create xDS client: %v", err) } - gotErr := u.(listenerUpdateErrTuple).err - if gotErr == nil || !strings.Contains(gotErr.Error(), wantListenerNACKErr) { - t.Fatalf("update received with error: %v, want %q", gotErr, wantListenerNACKErr) + defer close() + + // Register a watch for a listener resource and have the watch + // callback push the received update on to a channel. + lw1 := newListenerWatcher() + ldsCancel1 := xdsresource.WatchListener(client, ldsName, lw1) + defer ldsCancel1() + + // Configure the management server to return a single listener + // resource, corresponding to the one we registered a watch for. + resources := e2e.UpdateOptions{ + NodeID: nodeID, + Listeners: []*v3listenerpb.Listener{e2e.DefaultClientListener(ldsName, rdsName)}, + SkipValidation: true, + } + ctx, cancel := context.WithTimeout(context.Background(), 1000*defaultTestTimeout) + defer cancel() + if err := mgmtServer.Update(ctx, resources); err != nil { + t.Fatalf("Failed to update management server with resources: %v, err: %v", resources, err) + } + + // Verify the contents of the received update. + wantUpdate := listenerUpdateErrTuple{ + update: xdsresource.ListenerUpdate{ + RouteConfigName: rdsName, + HTTPFilters: []xdsresource.HTTPFilter{{Name: "router"}}, + }, + } + if err := verifyListenerUpdate(ctx, lw1.updateCh, wantUpdate); err != nil { + t.Fatal(err) + } + + // Configure the management server to return a single listener resource + // which is expected to be NACKed by the client. + resources = e2e.UpdateOptions{ + NodeID: nodeID, + Listeners: []*v3listenerpb.Listener{badListenerResource(t, ldsName)}, + SkipValidation: true, + } + if err := mgmtServer.Update(ctx, resources); err != nil { + t.Fatalf("Failed to update management server with resources: %v, err: %v", resources, err) + } + + // Verify that the expected error is propagated to the existing watcher. + if err := verifyUnknownListenerError(ctx, lw1.updateCh, wantListenerNACKErr); err != nil { + t.Fatal(err) + } + + // Register another watch for the same resource. This should get the update + // and error from the cache. + lw2 := newListenerWatcherMultiple(2) + ldsCancel2 := xdsresource.WatchListener(client, ldsName, lw2) + defer ldsCancel2() + if err := verifyListenerUpdate(ctx, lw2.updateCh, wantUpdate); err != nil { + t.Fatal(err) + } + // Verify that the expected error is propagated to the existing watcher. + if err := verifyUnknownListenerError(ctx, lw2.updateCh, wantListenerNACKErr); err != nil { + t.Fatal(err) + } + + // No request should get sent out as part of this watch. + sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout) + defer sCancel() + select { + case <-sCtx.Done(): + case <-secondRequestReceived.Done(): + t.Fatal("xdsClient sent out request instead of using update from cache") + default: } } // TestLDSWatch_PartialValid covers the case where a response from the // management server contains both valid and invalid resources and is expected -// to be NACK'ed by the xdsclient. The test verifies that watchers corresponding +// to be NACKed by the xdsclient. The test verifies that watchers corresponding // to the valid resource receive the update, while watchers corresponding to the // invalid resource receive an error. func (s) TestLDSWatch_PartialValid(t *testing.T) { @@ -1071,13 +1231,9 @@ func (s) TestLDSWatch_PartialValid(t *testing.T) { // Verify that the expected error is propagated to the watcher which // requested for the bad resource. - u, err := lw1.updateCh.Receive(ctx) - if err != nil { - t.Fatalf("timeout when waiting for a listener resource from the management server: %v", err) - } - gotErr := u.(listenerUpdateErrTuple).err - if gotErr == nil || !strings.Contains(gotErr.Error(), wantListenerNACKErr) { - t.Fatalf("update received with error: %v, want %q", gotErr, wantListenerNACKErr) + // Verify that the expected error is propagated to the existing watcher. + if err := verifyUnknownListenerError(ctx, lw1.updateCh, wantListenerNACKErr); err != nil { + t.Fatal(err) } // Verify that the watcher watching the good resource receives a good From 93f1cc163b21b863fff761f0122db733d03aa657 Mon Sep 17 00:00:00 2001 From: Brad Town Date: Fri, 22 Nov 2024 10:46:40 -0800 Subject: [PATCH 04/57] credentials/alts: avoid SRV and TXT lookups for handshaker service (#7861) --- credentials/alts/internal/handshaker/service/service.go | 4 +++- internal/resolver/dns/dns_resolver.go | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/credentials/alts/internal/handshaker/service/service.go b/credentials/alts/internal/handshaker/service/service.go index b3af03590729..fbfde5d047fe 100644 --- a/credentials/alts/internal/handshaker/service/service.go +++ b/credentials/alts/internal/handshaker/service/service.go @@ -47,8 +47,10 @@ func Dial(hsAddress string) (*grpc.ClientConn, error) { if !ok { // Create a new connection to the handshaker service. Note that // this connection stays open until the application is closed. + // Disable the service config to avoid unnecessary TXT record lookups that + // cause timeouts with some versions of systemd-resolved. var err error - hsConn, err = grpc.Dial(hsAddress, grpc.WithTransportCredentials(insecure.NewCredentials())) + hsConn, err = grpc.Dial(hsAddress, grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithDisableServiceConfig()) if err != nil { return nil, err } diff --git a/internal/resolver/dns/dns_resolver.go b/internal/resolver/dns/dns_resolver.go index cc5d5e05c010..b080ae30bc1b 100644 --- a/internal/resolver/dns/dns_resolver.go +++ b/internal/resolver/dns/dns_resolver.go @@ -237,7 +237,9 @@ func (d *dnsResolver) watcher() { } func (d *dnsResolver) lookupSRV(ctx context.Context) ([]resolver.Address, error) { - if !EnableSRVLookups { + // Skip this particular host to avoid timeouts with some versions of + // systemd-resolved. + if !EnableSRVLookups || d.host == "metadata.google.internal." { return nil, nil } var newAddrs []resolver.Address From 13d5a168d98ac33cb1c28083d29b76d92d24085d Mon Sep 17 00:00:00 2001 From: Zach Reyes <39203661+zasweq@users.noreply.github.com> Date: Fri, 22 Nov 2024 19:20:03 -0500 Subject: [PATCH 05/57] balancer/weightedroundrobin: Switch Weighted Round Robin to use pick first instead of SubConns (#7826) --- balancer/endpointsharding/endpointsharding.go | 27 +- balancer/weightedroundrobin/balancer.go | 569 +++++++++--------- balancer/weightedroundrobin/balancer_test.go | 59 ++ balancer/weightedroundrobin/metrics_test.go | 10 +- balancer/weightedroundrobin/scheduler.go | 14 +- 5 files changed, 374 insertions(+), 305 deletions(-) diff --git a/balancer/endpointsharding/endpointsharding.go b/balancer/endpointsharding/endpointsharding.go index b5b92143194b..263c024a84c7 100644 --- a/balancer/endpointsharding/endpointsharding.go +++ b/balancer/endpointsharding/endpointsharding.go @@ -28,19 +28,33 @@ package endpointsharding import ( "encoding/json" "errors" + "fmt" + rand "math/rand/v2" "sync" "sync/atomic" - rand "math/rand/v2" - "google.golang.org/grpc/balancer" "google.golang.org/grpc/balancer/base" + "google.golang.org/grpc/balancer/pickfirst" + "google.golang.org/grpc/balancer/pickfirst/pickfirstleaf" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/internal/balancer/gracefulswitch" + "google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/resolver" "google.golang.org/grpc/serviceconfig" ) +// PickFirstConfig is a pick first config without shuffling enabled. +var PickFirstConfig string + +func init() { + name := pickfirst.Name + if !envconfig.NewPickFirstEnabled { + name = pickfirstleaf.Name + } + PickFirstConfig = fmt.Sprintf("[{%q: {}}]", name) +} + // ChildState is the balancer state of a child along with the endpoint which // identifies the child balancer. type ChildState struct { @@ -100,9 +114,6 @@ func (es *endpointSharding) UpdateClientConnState(state balancer.ClientConnState // Update/Create new children. for _, endpoint := range state.ResolverState.Endpoints { - if len(endpoint.Addresses) == 0 { - continue - } if _, ok := newChildren.Get(endpoint); ok { // Endpoint child was already created, continue to avoid duplicate // update. @@ -143,6 +154,9 @@ func (es *endpointSharding) UpdateClientConnState(state balancer.ClientConnState } } es.children.Store(newChildren) + if newChildren.Len() == 0 { + return balancer.ErrBadResolverState + } return ret } @@ -306,6 +320,3 @@ func (bw *balancerWrapper) UpdateState(state balancer.State) { func ParseConfig(cfg json.RawMessage) (serviceconfig.LoadBalancingConfig, error) { return gracefulswitch.ParseConfig(cfg) } - -// PickFirstConfig is a pick first config without shuffling enabled. -const PickFirstConfig = "[{\"pick_first\": {}}]" diff --git a/balancer/weightedroundrobin/balancer.go b/balancer/weightedroundrobin/balancer.go index a0511772d2fa..c9c5b576bb0c 100644 --- a/balancer/weightedroundrobin/balancer.go +++ b/balancer/weightedroundrobin/balancer.go @@ -19,9 +19,7 @@ package weightedroundrobin import ( - "context" "encoding/json" - "errors" "fmt" rand "math/rand/v2" "sync" @@ -30,12 +28,13 @@ import ( "unsafe" "google.golang.org/grpc/balancer" - "google.golang.org/grpc/balancer/base" + "google.golang.org/grpc/balancer/endpointsharding" "google.golang.org/grpc/balancer/weightedroundrobin/internal" "google.golang.org/grpc/balancer/weightedtarget" "google.golang.org/grpc/connectivity" estats "google.golang.org/grpc/experimental/stats" "google.golang.org/grpc/internal/grpclog" + "google.golang.org/grpc/internal/grpcsync" iserviceconfig "google.golang.org/grpc/internal/serviceconfig" "google.golang.org/grpc/orca" "google.golang.org/grpc/resolver" @@ -84,23 +83,31 @@ var ( }) ) +// endpointSharding which specifies pick first children. +var endpointShardingLBConfig serviceconfig.LoadBalancingConfig + func init() { balancer.Register(bb{}) + var err error + endpointShardingLBConfig, err = endpointsharding.ParseConfig(json.RawMessage(endpointsharding.PickFirstConfig)) + if err != nil { + logger.Fatal(err) + } } type bb struct{} func (bb) Build(cc balancer.ClientConn, bOpts balancer.BuildOptions) balancer.Balancer { b := &wrrBalancer{ - cc: cc, - subConns: resolver.NewAddressMap(), - csEvltr: &balancer.ConnectivityStateEvaluator{}, - scMap: make(map[balancer.SubConn]*weightedSubConn), - connectivityState: connectivity.Connecting, - target: bOpts.Target.String(), - metricsRecorder: bOpts.MetricsRecorder, + ClientConn: cc, + target: bOpts.Target.String(), + metricsRecorder: bOpts.MetricsRecorder, + addressWeights: resolver.NewAddressMap(), + endpointToWeight: resolver.NewEndpointMap(), + scToWeight: make(map[balancer.SubConn]*endpointWeight), } + b.child = endpointsharding.NewBalancer(b, bOpts) b.logger = prefixLogger(b) b.logger.Infof("Created") return b @@ -141,123 +148,189 @@ func (bb) Name() string { return Name } +// updateEndpointsLocked updates endpoint weight state based off new update, by +// starting and clearing any endpoint weights needed. +// +// Caller must hold b.mu. +func (b *wrrBalancer) updateEndpointsLocked(endpoints []resolver.Endpoint) { + endpointSet := resolver.NewEndpointMap() + addressSet := resolver.NewAddressMap() + for _, endpoint := range endpoints { + endpointSet.Set(endpoint, nil) + for _, addr := range endpoint.Addresses { + addressSet.Set(addr, nil) + } + var ew *endpointWeight + if ewi, ok := b.endpointToWeight.Get(endpoint); ok { + ew = ewi.(*endpointWeight) + } else { + ew = &endpointWeight{ + logger: b.logger, + connectivityState: connectivity.Connecting, + // Initially, we set load reports to off, because they are not + // running upon initial endpointWeight creation. + cfg: &lbConfig{EnableOOBLoadReport: false}, + metricsRecorder: b.metricsRecorder, + target: b.target, + locality: b.locality, + } + for _, addr := range endpoint.Addresses { + b.addressWeights.Set(addr, ew) + } + b.endpointToWeight.Set(endpoint, ew) + } + ew.updateConfig(b.cfg) + } + + for _, endpoint := range b.endpointToWeight.Keys() { + if _, ok := endpointSet.Get(endpoint); ok { + // Existing endpoint also in new endpoint list; skip. + continue + } + b.endpointToWeight.Delete(endpoint) + for _, addr := range endpoint.Addresses { + if _, ok := addressSet.Get(addr); !ok { // old endpoints to be deleted can share addresses with new endpoints, so only delete if necessary + b.addressWeights.Delete(addr) + } + } + // SubConn map will get handled in updateSubConnState + // when receives SHUTDOWN signal. + } +} + // wrrBalancer implements the weighted round robin LB policy. type wrrBalancer struct { - // The following fields are immutable. - cc balancer.ClientConn - logger *grpclog.PrefixLogger - target string - metricsRecorder estats.MetricsRecorder - - // The following fields are only accessed on calls into the LB policy, and - // do not need a mutex. - cfg *lbConfig // active config - subConns *resolver.AddressMap // active weightedSubConns mapped by address - scMap map[balancer.SubConn]*weightedSubConn - connectivityState connectivity.State // aggregate state - csEvltr *balancer.ConnectivityStateEvaluator - resolverErr error // the last error reported by the resolver; cleared on successful resolution - connErr error // the last connection error; cleared upon leaving TransientFailure - stopPicker func() - locality string + // The following fields are set at initialization time and read only after that, + // so they do not need to be protected by a mutex. + child balancer.Balancer + balancer.ClientConn // Embed to intercept NewSubConn operation + logger *grpclog.PrefixLogger + target string + metricsRecorder estats.MetricsRecorder + + mu sync.Mutex + cfg *lbConfig // active config + locality string + stopPicker *grpcsync.Event + addressWeights *resolver.AddressMap // addr -> endpointWeight + endpointToWeight *resolver.EndpointMap // endpoint -> endpointWeight + scToWeight map[balancer.SubConn]*endpointWeight } func (b *wrrBalancer) UpdateClientConnState(ccs balancer.ClientConnState) error { b.logger.Infof("UpdateCCS: %v", ccs) - b.resolverErr = nil cfg, ok := ccs.BalancerConfig.(*lbConfig) if !ok { return fmt.Errorf("wrr: received nil or illegal BalancerConfig (type %T): %v", ccs.BalancerConfig, ccs.BalancerConfig) } + // Note: empty endpoints and duplicate addresses across endpoints won't + // explicitly error but will have undefined behavior. + b.mu.Lock() b.cfg = cfg b.locality = weightedtarget.LocalityFromResolverState(ccs.ResolverState) - b.updateAddresses(ccs.ResolverState.Addresses) - - if len(ccs.ResolverState.Addresses) == 0 { - b.ResolverError(errors.New("resolver produced zero addresses")) // will call regeneratePicker - return balancer.ErrBadResolverState - } + b.updateEndpointsLocked(ccs.ResolverState.Endpoints) + b.mu.Unlock() + + // This causes child to update picker inline and will thus cause inline + // picker update. + return b.child.UpdateClientConnState(balancer.ClientConnState{ + BalancerConfig: endpointShardingLBConfig, + ResolverState: ccs.ResolverState, + }) +} - b.regeneratePicker() +func (b *wrrBalancer) UpdateState(state balancer.State) { + b.mu.Lock() + defer b.mu.Unlock() - return nil -} + if b.stopPicker != nil { + b.stopPicker.Fire() + b.stopPicker = nil + } -func (b *wrrBalancer) updateAddresses(addrs []resolver.Address) { - addrsSet := resolver.NewAddressMap() + childStates := endpointsharding.ChildStatesFromPicker(state.Picker) - // Loop through new address list and create subconns for any new addresses. - for _, addr := range addrs { - if _, ok := addrsSet.Get(addr); ok { - // Redundant address; skip. - continue - } - addrsSet.Set(addr, nil) + var readyPickersWeight []pickerWeightedEndpoint - var wsc *weightedSubConn - wsci, ok := b.subConns.Get(addr) - if ok { - wsc = wsci.(*weightedSubConn) - } else { - // addr is a new address (not existing in b.subConns). - var sc balancer.SubConn - sc, err := b.cc.NewSubConn([]resolver.Address{addr}, balancer.NewSubConnOptions{ - StateListener: func(state balancer.SubConnState) { - b.updateSubConnState(sc, state) - }, - }) - if err != nil { - b.logger.Warningf("Failed to create new SubConn for address %v: %v", addr, err) + for _, childState := range childStates { + if childState.State.ConnectivityState == connectivity.Ready { + ewv, ok := b.endpointToWeight.Get(childState.Endpoint) + if !ok { + // Should never happen, simply continue and ignore this endpoint + // for READY pickers. continue } - wsc = &weightedSubConn{ - SubConn: sc, - logger: b.logger, - connectivityState: connectivity.Idle, - // Initially, we set load reports to off, because they are not - // running upon initial weightedSubConn creation. - cfg: &lbConfig{EnableOOBLoadReport: false}, - - metricsRecorder: b.metricsRecorder, - target: b.target, - locality: b.locality, - } - b.subConns.Set(addr, wsc) - b.scMap[sc] = wsc - b.csEvltr.RecordTransition(connectivity.Shutdown, connectivity.Idle) - sc.Connect() + ew := ewv.(*endpointWeight) + readyPickersWeight = append(readyPickersWeight, pickerWeightedEndpoint{ + picker: childState.State.Picker, + weightedEndpoint: ew, + }) } - // Update config for existing weightedSubConn or send update for first - // time to new one. Ensures an OOB listener is running if needed - // (and stops the existing one if applicable). - wsc.updateConfig(b.cfg) + } + // If no ready pickers are present, simply defer to the round robin picker + // from endpoint sharding, which will round robin across the most relevant + // pick first children in the highest precedence connectivity state. + if len(readyPickersWeight) == 0 { + b.ClientConn.UpdateState(balancer.State{ + ConnectivityState: state.ConnectivityState, + Picker: state.Picker, + }) + return } - // Loop through existing subconns and remove ones that are not in addrs. - for _, addr := range b.subConns.Keys() { - if _, ok := addrsSet.Get(addr); ok { - // Existing address also in new address list; skip. - continue - } - // addr was removed by resolver. Remove. - wsci, _ := b.subConns.Get(addr) - wsc := wsci.(*weightedSubConn) - wsc.SubConn.Shutdown() - b.subConns.Delete(addr) + p := &picker{ + v: rand.Uint32(), // start the scheduler at a random point + cfg: b.cfg, + weightedPickers: readyPickersWeight, + metricsRecorder: b.metricsRecorder, + locality: b.locality, + target: b.target, } + + b.stopPicker = grpcsync.NewEvent() + p.start(b.stopPicker) + + b.ClientConn.UpdateState(balancer.State{ + ConnectivityState: state.ConnectivityState, + Picker: p, + }) } -func (b *wrrBalancer) ResolverError(err error) { - b.resolverErr = err - if b.subConns.Len() == 0 { - b.connectivityState = connectivity.TransientFailure +type pickerWeightedEndpoint struct { + picker balancer.Picker + weightedEndpoint *endpointWeight +} + +func (b *wrrBalancer) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) { + addr := addrs[0] // The new pick first policy for DualStack will only ever create a SubConn with one address. + var sc balancer.SubConn + + oldListener := opts.StateListener + opts.StateListener = func(state balancer.SubConnState) { + b.updateSubConnState(sc, state) + oldListener(state) } - if b.connectivityState != connectivity.TransientFailure { - // No need to update the picker since no error is being returned. - return + + b.mu.Lock() + defer b.mu.Unlock() + ewi, ok := b.addressWeights.Get(addr) + if !ok { + // SubConn state updates can come in for a no longer relevant endpoint + // weight (from the old system after a new config update is applied). + return nil, fmt.Errorf("balancer is being closed; no new SubConns allowed") + } + sc, err := b.ClientConn.NewSubConn([]resolver.Address{addr}, opts) + if err != nil { + return nil, err } - b.regeneratePicker() + b.scToWeight[sc] = ewi.(*endpointWeight) + return sc, nil +} + +func (b *wrrBalancer) ResolverError(err error) { + // Will cause inline picker update from endpoint sharding. + b.child.ResolverError(err) } func (b *wrrBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { @@ -265,134 +338,84 @@ func (b *wrrBalancer) UpdateSubConnState(sc balancer.SubConn, state balancer.Sub } func (b *wrrBalancer) updateSubConnState(sc balancer.SubConn, state balancer.SubConnState) { - wsc := b.scMap[sc] - if wsc == nil { - b.logger.Errorf("UpdateSubConnState called with an unknown SubConn: %p, %v", sc, state) + b.mu.Lock() + ew := b.scToWeight[sc] + // updates from a no longer relevant SubConn update, nothing to do here but + // forward state to state listener, which happens in wrapped listener. Will + // eventually get cleared from scMap once receives Shutdown signal. + if ew == nil { + b.mu.Unlock() return } - if b.logger.V(2) { - logger.Infof("UpdateSubConnState(%+v, %+v)", sc, state) - } - - cs := state.ConnectivityState - - if cs == connectivity.TransientFailure { - // Save error to be reported via picker. - b.connErr = state.ConnectionError - } - - if cs == connectivity.Shutdown { - delete(b.scMap, sc) - // The subconn was removed from b.subConns when the address was removed - // in updateAddresses. + if state.ConnectivityState == connectivity.Shutdown { + delete(b.scToWeight, sc) + } + b.mu.Unlock() + + // On the first READY SubConn/Transition for an endpoint, set pickedSC, + // clear endpoint tracking weight state, and potentially start an OOB watch. + if state.ConnectivityState == connectivity.Ready && ew.pickedSC == nil { + ew.pickedSC = sc + ew.mu.Lock() + ew.nonEmptySince = time.Time{} + ew.lastUpdated = time.Time{} + cfg := ew.cfg + ew.mu.Unlock() + ew.updateORCAListener(cfg) + return } - oldCS := wsc.updateConnectivityState(cs) - b.connectivityState = b.csEvltr.RecordTransition(oldCS, cs) - - // Regenerate picker when one of the following happens: - // - this sc entered or left ready - // - the aggregated state of balancer is TransientFailure - // (may need to update error message) - if (cs == connectivity.Ready) != (oldCS == connectivity.Ready) || - b.connectivityState == connectivity.TransientFailure { - b.regeneratePicker() + // If the pickedSC (the one pick first uses for an endpoint) transitions out + // of READY, stop OOB listener if needed and clear pickedSC so the next + // created SubConn for the endpoint that goes READY will be chosen for + // endpoint as the active SubConn. + if state.ConnectivityState != connectivity.Ready && ew.pickedSC == sc { + // The first SubConn that goes READY for an endpoint is what pick first + // will pick. Only once that SubConn goes not ready will pick first + // restart this cycle of creating SubConns and using the first READY + // one. The lower level endpoint sharding will ping the Pick First once + // this occurs to ExitIdle which will trigger a connection attempt. + if ew.stopORCAListener != nil { + ew.stopORCAListener() + } + ew.pickedSC = nil } } // Close stops the balancer. It cancels any ongoing scheduler updates and // stops any ORCA listeners. func (b *wrrBalancer) Close() { + b.mu.Lock() if b.stopPicker != nil { - b.stopPicker() + b.stopPicker.Fire() b.stopPicker = nil } - for _, wsc := range b.scMap { - // Ensure any lingering OOB watchers are stopped. - wsc.updateConnectivityState(connectivity.Shutdown) - } -} - -// ExitIdle is ignored; we always connect to all backends. -func (b *wrrBalancer) ExitIdle() {} + b.mu.Unlock() -func (b *wrrBalancer) readySubConns() []*weightedSubConn { - var ret []*weightedSubConn - for _, v := range b.subConns.Values() { - wsc := v.(*weightedSubConn) - if wsc.connectivityState == connectivity.Ready { - ret = append(ret, wsc) + // Ensure any lingering OOB watchers are stopped. + for _, ewv := range b.endpointToWeight.Values() { + ew := ewv.(*endpointWeight) + if ew.stopORCAListener != nil { + ew.stopORCAListener() } } - return ret } -// mergeErrors builds an error from the last connection error and the last -// resolver error. Must only be called if b.connectivityState is -// TransientFailure. -func (b *wrrBalancer) mergeErrors() error { - // connErr must always be non-nil unless there are no SubConns, in which - // case resolverErr must be non-nil. - if b.connErr == nil { - return fmt.Errorf("last resolver error: %v", b.resolverErr) +func (b *wrrBalancer) ExitIdle() { + if ei, ok := b.child.(balancer.ExitIdler); ok { // Should always be ok, as child is endpoint sharding. + ei.ExitIdle() } - if b.resolverErr == nil { - return fmt.Errorf("last connection error: %v", b.connErr) - } - return fmt.Errorf("last connection error: %v; last resolver error: %v", b.connErr, b.resolverErr) -} - -func (b *wrrBalancer) regeneratePicker() { - if b.stopPicker != nil { - b.stopPicker() - b.stopPicker = nil - } - - switch b.connectivityState { - case connectivity.TransientFailure: - b.cc.UpdateState(balancer.State{ - ConnectivityState: connectivity.TransientFailure, - Picker: base.NewErrPicker(b.mergeErrors()), - }) - return - case connectivity.Connecting, connectivity.Idle: - // Idle could happen very briefly if all subconns are Idle and we've - // asked them to connect but they haven't reported Connecting yet. - // Report the same as Connecting since this is temporary. - b.cc.UpdateState(balancer.State{ - ConnectivityState: connectivity.Connecting, - Picker: base.NewErrPicker(balancer.ErrNoSubConnAvailable), - }) - return - case connectivity.Ready: - b.connErr = nil - } - - p := &picker{ - v: rand.Uint32(), // start the scheduler at a random point - cfg: b.cfg, - subConns: b.readySubConns(), - metricsRecorder: b.metricsRecorder, - locality: b.locality, - target: b.target, - } - var ctx context.Context - ctx, b.stopPicker = context.WithCancel(context.Background()) - p.start(ctx) - b.cc.UpdateState(balancer.State{ - ConnectivityState: b.connectivityState, - Picker: p, - }) } // picker is the WRR policy's picker. It uses live-updating backend weights to // update the scheduler periodically and ensure picks are routed proportional // to those weights. type picker struct { - scheduler unsafe.Pointer // *scheduler; accessed atomically - v uint32 // incrementing value used by the scheduler; accessed atomically - cfg *lbConfig // active config when picker created - subConns []*weightedSubConn // all READY subconns + scheduler unsafe.Pointer // *scheduler; accessed atomically + v uint32 // incrementing value used by the scheduler; accessed atomically + cfg *lbConfig // active config when picker created + + weightedPickers []pickerWeightedEndpoint // all READY pickers // The following fields are immutable. target string @@ -400,14 +423,39 @@ type picker struct { metricsRecorder estats.MetricsRecorder } -func (p *picker) scWeights(recordMetrics bool) []float64 { - ws := make([]float64, len(p.subConns)) +func (p *picker) endpointWeights(recordMetrics bool) []float64 { + wp := make([]float64, len(p.weightedPickers)) now := internal.TimeNow() - for i, wsc := range p.subConns { - ws[i] = wsc.weight(now, time.Duration(p.cfg.WeightExpirationPeriod), time.Duration(p.cfg.BlackoutPeriod), recordMetrics) + for i, wpi := range p.weightedPickers { + wp[i] = wpi.weightedEndpoint.weight(now, time.Duration(p.cfg.WeightExpirationPeriod), time.Duration(p.cfg.BlackoutPeriod), recordMetrics) } + return wp +} + +func (p *picker) Pick(info balancer.PickInfo) (balancer.PickResult, error) { + // Read the scheduler atomically. All scheduler operations are threadsafe, + // and if the scheduler is replaced during this usage, we want to use the + // scheduler that was live when the pick started. + sched := *(*scheduler)(atomic.LoadPointer(&p.scheduler)) - return ws + pickedPicker := p.weightedPickers[sched.nextIndex()] + pr, err := pickedPicker.picker.Pick(info) + if err != nil { + logger.Errorf("ready picker returned error: %v", err) + return balancer.PickResult{}, err + } + if !p.cfg.EnableOOBLoadReport { + oldDone := pr.Done + pr.Done = func(info balancer.DoneInfo) { + if load, ok := info.ServerLoad.(*v3orcapb.OrcaLoadReport); ok && load != nil { + pickedPicker.weightedEndpoint.OnLoadReport(load) + } + if oldDone != nil { + oldDone(info) + } + } + } + return pr, nil } func (p *picker) inc() uint32 { @@ -419,9 +467,9 @@ func (p *picker) regenerateScheduler() { atomic.StorePointer(&p.scheduler, unsafe.Pointer(&s)) } -func (p *picker) start(ctx context.Context) { +func (p *picker) start(stopPicker *grpcsync.Event) { p.regenerateScheduler() - if len(p.subConns) == 1 { + if len(p.weightedPickers) == 1 { // No need to regenerate weights with only one backend. return } @@ -431,7 +479,7 @@ func (p *picker) start(ctx context.Context) { defer ticker.Stop() for { select { - case <-ctx.Done(): + case <-stopPicker.Done(): return case <-ticker.C: p.regenerateScheduler() @@ -440,29 +488,12 @@ func (p *picker) start(ctx context.Context) { }() } -func (p *picker) Pick(balancer.PickInfo) (balancer.PickResult, error) { - // Read the scheduler atomically. All scheduler operations are threadsafe, - // and if the scheduler is replaced during this usage, we want to use the - // scheduler that was live when the pick started. - sched := *(*scheduler)(atomic.LoadPointer(&p.scheduler)) - - pickedSC := p.subConns[sched.nextIndex()] - pr := balancer.PickResult{SubConn: pickedSC.SubConn} - if !p.cfg.EnableOOBLoadReport { - pr.Done = func(info balancer.DoneInfo) { - if load, ok := info.ServerLoad.(*v3orcapb.OrcaLoadReport); ok && load != nil { - pickedSC.OnLoadReport(load) - } - } - } - return pr, nil -} - -// weightedSubConn is the wrapper of a subconn that holds the subconn and its -// weight (and other parameters relevant to computing the effective weight). -// When needed, it also tracks connectivity state, listens for metrics updates -// by implementing the orca.OOBListener interface and manages that listener. -type weightedSubConn struct { +// endpointWeight is the weight for an endpoint. It tracks the SubConn that will +// be picked for the endpoint, and other parameters relevant to computing the +// effective weight. When needed, it also tracks connectivity state, listens for +// metrics updates by implementing the orca.OOBListener interface and manages +// that listener. +type endpointWeight struct { // The following fields are immutable. balancer.SubConn logger *grpclog.PrefixLogger @@ -474,6 +505,11 @@ type weightedSubConn struct { // do not need a mutex. connectivityState connectivity.State stopORCAListener func() + // The first SubConn for the endpoint that goes READY when endpoint has no + // READY SubConns yet, cleared on that sc disconnecting (i.e. going out of + // READY). Represents what pick first will use as it's picked SubConn for + // this endpoint. + pickedSC balancer.SubConn // The following fields are accessed asynchronously and are protected by // mu. Note that mu may not be held when calling into the stopORCAListener @@ -487,11 +523,11 @@ type weightedSubConn struct { cfg *lbConfig } -func (w *weightedSubConn) OnLoadReport(load *v3orcapb.OrcaLoadReport) { +func (w *endpointWeight) OnLoadReport(load *v3orcapb.OrcaLoadReport) { if w.logger.V(2) { w.logger.Infof("Received load report for subchannel %v: %v", w.SubConn, load) } - // Update weights of this subchannel according to the reported load + // Update weights of this endpoint according to the reported load. utilization := load.ApplicationUtilization if utilization == 0 { utilization = load.CpuUtilization @@ -520,7 +556,7 @@ func (w *weightedSubConn) OnLoadReport(load *v3orcapb.OrcaLoadReport) { // updateConfig updates the parameters of the WRR policy and // stops/starts/restarts the ORCA OOB listener. -func (w *weightedSubConn) updateConfig(cfg *lbConfig) { +func (w *endpointWeight) updateConfig(cfg *lbConfig) { w.mu.Lock() oldCfg := w.cfg w.cfg = cfg @@ -533,14 +569,12 @@ func (w *weightedSubConn) updateConfig(cfg *lbConfig) { // load reporting disabled, OOBReportingPeriod is always 0.) return } - if w.connectivityState == connectivity.Ready { - // (Re)start the listener to use the new config's settings for OOB - // reporting. - w.updateORCAListener(cfg) - } + // (Re)start the listener to use the new config's settings for OOB + // reporting. + w.updateORCAListener(cfg) } -func (w *weightedSubConn) updateORCAListener(cfg *lbConfig) { +func (w *endpointWeight) updateORCAListener(cfg *lbConfig) { if w.stopORCAListener != nil { w.stopORCAListener() } @@ -548,57 +582,22 @@ func (w *weightedSubConn) updateORCAListener(cfg *lbConfig) { w.stopORCAListener = nil return } + if w.pickedSC == nil { // No picked SC for this endpoint yet, nothing to listen on. + return + } if w.logger.V(2) { - w.logger.Infof("Registering ORCA listener for %v with interval %v", w.SubConn, cfg.OOBReportingPeriod) + w.logger.Infof("Registering ORCA listener for %v with interval %v", w.pickedSC, cfg.OOBReportingPeriod) } opts := orca.OOBListenerOptions{ReportInterval: time.Duration(cfg.OOBReportingPeriod)} - w.stopORCAListener = orca.RegisterOOBListener(w.SubConn, w, opts) -} - -func (w *weightedSubConn) updateConnectivityState(cs connectivity.State) connectivity.State { - switch cs { - case connectivity.Idle: - // Always reconnect when idle. - w.SubConn.Connect() - case connectivity.Ready: - // If we transition back to READY state, reset nonEmptySince so that we - // apply the blackout period after we start receiving load data. Also - // reset lastUpdated to trigger endpoint weight not yet usable in the - // case endpoint gets asked what weight it is before receiving a new - // load report. Note that we cannot guarantee that we will never receive - // lingering callbacks for backend metric reports from the previous - // connection after the new connection has been established, but they - // should be masked by new backend metric reports from the new - // connection by the time the blackout period ends. - w.mu.Lock() - w.nonEmptySince = time.Time{} - w.lastUpdated = time.Time{} - cfg := w.cfg - w.mu.Unlock() - w.updateORCAListener(cfg) - } - - oldCS := w.connectivityState - - if oldCS == connectivity.TransientFailure && - (cs == connectivity.Connecting || cs == connectivity.Idle) { - // Once a subconn enters TRANSIENT_FAILURE, ignore subsequent IDLE or - // CONNECTING transitions to prevent the aggregated state from being - // always CONNECTING when many backends exist but are all down. - return oldCS - } - - w.connectivityState = cs - - return oldCS + w.stopORCAListener = orca.RegisterOOBListener(w.pickedSC, w, opts) } -// weight returns the current effective weight of the subconn, taking into +// weight returns the current effective weight of the endpoint, taking into // account the parameters. Returns 0 for blacked out or expired data, which // will cause the backend weight to be treated as the mean of the weights of the // other backends. If forScheduler is set to true, this function will emit // metrics through the metrics registry. -func (w *weightedSubConn) weight(now time.Time, weightExpirationPeriod, blackoutPeriod time.Duration, recordMetrics bool) (weight float64) { +func (w *endpointWeight) weight(now time.Time, weightExpirationPeriod, blackoutPeriod time.Duration, recordMetrics bool) (weight float64) { w.mu.Lock() defer w.mu.Unlock() @@ -608,7 +607,7 @@ func (w *weightedSubConn) weight(now time.Time, weightExpirationPeriod, blackout }() } - // The SubConn has not received a load report (i.e. just turned READY with + // The endpoint has not received a load report (i.e. just turned READY with // no load report). if w.lastUpdated.Equal(time.Time{}) { endpointWeightNotYetUsableMetric.Record(w.metricsRecorder, 1, w.target, w.locality) diff --git a/balancer/weightedroundrobin/balancer_test.go b/balancer/weightedroundrobin/balancer_test.go index 68d2d5a5c5c8..5e369780764e 100644 --- a/balancer/weightedroundrobin/balancer_test.go +++ b/balancer/weightedroundrobin/balancer_test.go @@ -460,6 +460,65 @@ func (s) TestBalancer_TwoAddresses_OOBThenPerCall(t *testing.T) { checkWeights(ctx, t, srvWeight{srv1, 10}, srvWeight{srv2, 1}) } +// TestEndpoints_SharedAddress tests the case where two endpoints have the same +// address. The expected behavior is undefined, however the program should not +// crash. +func (s) TestEndpoints_SharedAddress(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + srv := startServer(t, reportCall) + sc := svcConfig(t, perCallConfig) + if err := srv.StartClient(grpc.WithDefaultServiceConfig(sc)); err != nil { + t.Fatalf("Error starting client: %v", err) + } + + endpointsSharedAddress := []resolver.Endpoint{{Addresses: []resolver.Address{{Addr: srv.Address}}}, {Addresses: []resolver.Address{{Addr: srv.Address}}}} + srv.R.UpdateState(resolver.State{Endpoints: endpointsSharedAddress}) + + // Make some RPC's and make sure doesn't crash. It should go to one of the + // endpoints addresses, it's undefined which one it will choose and the load + // reporting might not work, but it should be able to make an RPC. + for i := 0; i < 10; i++ { + if _, err := srv.Client.EmptyCall(ctx, &testpb.Empty{}); err != nil { + t.Fatalf("EmptyCall failed with err: %v", err) + } + } +} + +// TestEndpoints_MultipleAddresses tests WRR on endpoints with numerous +// addresses. It configures WRR with two endpoints with one bad address followed +// by a good address. It configures two backends that each report per call +// metrics, each corresponding to the two endpoints good address. It then +// asserts load is distributed as expected corresponding to the call metrics +// received. +func (s) TestEndpoints_MultipleAddresses(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + srv1 := startServer(t, reportCall) + srv2 := startServer(t, reportCall) + + srv1.callMetrics.SetQPS(10.0) + srv1.callMetrics.SetApplicationUtilization(.1) + + srv2.callMetrics.SetQPS(10.0) + srv2.callMetrics.SetApplicationUtilization(1.0) + + sc := svcConfig(t, perCallConfig) + if err := srv1.StartClient(grpc.WithDefaultServiceConfig(sc)); err != nil { + t.Fatalf("Error starting client: %v", err) + } + + twoEndpoints := []resolver.Endpoint{{Addresses: []resolver.Address{{Addr: "bad-address-1"}, {Addr: srv1.Address}}}, {Addresses: []resolver.Address{{Addr: "bad-address-2"}, {Addr: srv2.Address}}}} + srv1.R.UpdateState(resolver.State{Endpoints: twoEndpoints}) + + // Call each backend once to ensure the weights have been received. + ensureReached(ctx, t, srv1.Client, 2) + // Wait for the weight update period to allow the new weights to be processed. + time.Sleep(weightUpdatePeriod) + checkWeights(ctx, t, srvWeight{srv1, 10}, srvWeight{srv2, 1}) +} + // Tests two addresses with OOB ORCA reporting enabled and a non-zero error // penalty applied. func (s) TestBalancer_TwoAddresses_ErrorPenalty(t *testing.T) { diff --git a/balancer/weightedroundrobin/metrics_test.go b/balancer/weightedroundrobin/metrics_test.go index 9794a65e044f..79e4d0a145a0 100644 --- a/balancer/weightedroundrobin/metrics_test.go +++ b/balancer/weightedroundrobin/metrics_test.go @@ -109,7 +109,7 @@ func (s) TestWRR_Metrics_SubConnWeight(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { tmr := stats.NewTestMetricsRecorder() - wsc := &weightedSubConn{ + wsc := &endpointWeight{ metricsRecorder: tmr, weightVal: 3, lastUpdated: test.lastUpdated, @@ -137,7 +137,7 @@ func (s) TestWRR_Metrics_SubConnWeight(t *testing.T) { // fallback. func (s) TestWRR_Metrics_Scheduler_RR_Fallback(t *testing.T) { tmr := stats.NewTestMetricsRecorder() - wsc := &weightedSubConn{ + ew := &endpointWeight{ metricsRecorder: tmr, weightVal: 0, } @@ -147,7 +147,7 @@ func (s) TestWRR_Metrics_Scheduler_RR_Fallback(t *testing.T) { BlackoutPeriod: iserviceconfig.Duration(10 * time.Second), WeightExpirationPeriod: iserviceconfig.Duration(3 * time.Minute), }, - subConns: []*weightedSubConn{wsc}, + weightedPickers: []pickerWeightedEndpoint{{weightedEndpoint: ew}}, metricsRecorder: tmr, } // There is only one SubConn, so no matter if the SubConn has a weight or @@ -160,12 +160,12 @@ func (s) TestWRR_Metrics_Scheduler_RR_Fallback(t *testing.T) { // With two SubConns, if neither of them have weights, it will also fallback // to round robin. - wsc2 := &weightedSubConn{ + ew2 := &endpointWeight{ target: "target", metricsRecorder: tmr, weightVal: 0, } - p.subConns = append(p.subConns, wsc2) + p.weightedPickers = append(p.weightedPickers, pickerWeightedEndpoint{weightedEndpoint: ew2}) p.regenerateScheduler() if got, _ := tmr.Metric("grpc.lb.wrr.rr_fallback"); got != 1 { t.Fatalf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.wrr.rr_fallback", got, 1) diff --git a/balancer/weightedroundrobin/scheduler.go b/balancer/weightedroundrobin/scheduler.go index 56aa15da10d2..7d3d6815eb7a 100644 --- a/balancer/weightedroundrobin/scheduler.go +++ b/balancer/weightedroundrobin/scheduler.go @@ -26,14 +26,14 @@ type scheduler interface { nextIndex() int } -// newScheduler uses scWeights to create a new scheduler for selecting subconns +// newScheduler uses scWeights to create a new scheduler for selecting endpoints // in a picker. It will return a round robin implementation if at least -// len(scWeights)-1 are zero or there is only a single subconn, otherwise it +// len(scWeights)-1 are zero or there is only a single endpoint, otherwise it // will return an Earliest Deadline First (EDF) scheduler implementation that -// selects the subchannels according to their weights. +// selects the endpoints according to their weights. func (p *picker) newScheduler(recordMetrics bool) scheduler { - scWeights := p.scWeights(recordMetrics) - n := len(scWeights) + epWeights := p.endpointWeights(recordMetrics) + n := len(epWeights) if n == 0 { return nil } @@ -46,7 +46,7 @@ func (p *picker) newScheduler(recordMetrics bool) scheduler { sum := float64(0) numZero := 0 max := float64(0) - for _, w := range scWeights { + for _, w := range epWeights { sum += w if w > max { max = w @@ -68,7 +68,7 @@ func (p *picker) newScheduler(recordMetrics bool) scheduler { weights := make([]uint16, n) allEqual := true - for i, w := range scWeights { + for i, w := range epWeights { if w == 0 { // Backends with weight = 0 use the mean. weights[i] = mean From 8b70aeb896f52f99d43aa00bd196cffaf9e1db5e Mon Sep 17 00:00:00 2001 From: Purnesh Dixit Date: Mon, 25 Nov 2024 11:03:13 +0530 Subject: [PATCH 06/57] stats/opentelemetry: introduce tracing propagator and carrier (#7677) --- go.mod | 2 +- .../grpc_trace_bin_propagator.go | 119 ++++++++++ .../grpc_trace_bin_propagator_test.go | 219 ++++++++++++++++++ .../opentelemetry/internal/tracing/carrier.go | 131 +++++++++++ .../internal/tracing/carrier_test.go | 190 +++++++++++++++ 5 files changed, 660 insertions(+), 1 deletion(-) create mode 100644 stats/opentelemetry/grpc_trace_bin_propagator.go create mode 100644 stats/opentelemetry/grpc_trace_bin_propagator_test.go create mode 100644 stats/opentelemetry/internal/tracing/carrier.go create mode 100644 stats/opentelemetry/internal/tracing/carrier_test.go diff --git a/go.mod b/go.mod index 1bbd024d22c1..9b3d296cc882 100644 --- a/go.mod +++ b/go.mod @@ -15,6 +15,7 @@ require ( go.opentelemetry.io/otel/metric v1.31.0 go.opentelemetry.io/otel/sdk v1.31.0 go.opentelemetry.io/otel/sdk/metric v1.31.0 + go.opentelemetry.io/otel/trace v1.31.0 golang.org/x/net v0.30.0 golang.org/x/oauth2 v0.23.0 golang.org/x/sync v0.8.0 @@ -32,7 +33,6 @@ require ( github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect - go.opentelemetry.io/otel/trace v1.31.0 // indirect golang.org/x/text v0.19.0 // indirect google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 // indirect ) diff --git a/stats/opentelemetry/grpc_trace_bin_propagator.go b/stats/opentelemetry/grpc_trace_bin_propagator.go new file mode 100644 index 000000000000..e8a3986d4f4a --- /dev/null +++ b/stats/opentelemetry/grpc_trace_bin_propagator.go @@ -0,0 +1,119 @@ +/* + * + * Copyright 2024 gRPC 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 opentelemetry + +import ( + "context" + + otelpropagation "go.opentelemetry.io/otel/propagation" + oteltrace "go.opentelemetry.io/otel/trace" +) + +// gRPCTraceBinHeaderKey is the gRPC metadata header key `grpc-trace-bin` used +// to propagate trace context in binary format. +const grpcTraceBinHeaderKey = "grpc-trace-bin" + +// GRPCTraceBinPropagator is an OpenTelemetry TextMapPropagator which is used +// to extract and inject trace context data from and into headers exchanged by +// gRPC applications. It propagates trace data in binary format using the +// `grpc-trace-bin` header. +type GRPCTraceBinPropagator struct{} + +// Inject sets OpenTelemetry span context from the Context into the carrier as +// a `grpc-trace-bin` header if span context is valid. +// +// If span context is not valid, it returns without setting `grpc-trace-bin` +// header. +func (GRPCTraceBinPropagator) Inject(ctx context.Context, carrier otelpropagation.TextMapCarrier) { + sc := oteltrace.SpanFromContext(ctx) + if !sc.SpanContext().IsValid() { + return + } + + bd := toBinary(sc.SpanContext()) + carrier.Set(grpcTraceBinHeaderKey, string(bd)) +} + +// Extract reads OpenTelemetry span context from the `grpc-trace-bin` header of +// carrier into the provided context, if present. +// +// If a valid span context is retrieved from `grpc-trace-bin`, it returns a new +// context containing the extracted OpenTelemetry span context marked as +// remote. +// +// If `grpc-trace-bin` header is not present, it returns the context as is. +func (GRPCTraceBinPropagator) Extract(ctx context.Context, carrier otelpropagation.TextMapCarrier) context.Context { + h := carrier.Get(grpcTraceBinHeaderKey) + if h == "" { + return ctx + } + + sc, ok := fromBinary([]byte(h)) + if !ok { + return ctx + } + return oteltrace.ContextWithRemoteSpanContext(ctx, sc) +} + +// Fields returns the keys whose values are set with Inject. +// +// GRPCTraceBinPropagator always returns a slice containing only +// `grpc-trace-bin` key because it only sets the `grpc-trace-bin` header for +// propagating trace context. +func (GRPCTraceBinPropagator) Fields() []string { + return []string{grpcTraceBinHeaderKey} +} + +// toBinary returns the binary format representation of a SpanContext. +// +// If sc is the zero value, returns nil. +func toBinary(sc oteltrace.SpanContext) []byte { + if sc.Equal(oteltrace.SpanContext{}) { + return nil + } + var b [29]byte + traceID := oteltrace.TraceID(sc.TraceID()) + copy(b[2:18], traceID[:]) + b[18] = 1 + spanID := oteltrace.SpanID(sc.SpanID()) + copy(b[19:27], spanID[:]) + b[27] = 2 + b[28] = byte(oteltrace.TraceFlags(sc.TraceFlags())) + return b[:] +} + +// fromBinary returns the SpanContext represented by b with Remote set to true. +// +// It returns with zero value SpanContext and false, if any of the +// below condition is not satisfied: +// - Valid header: len(b) = 29 +// - Valid version: b[0] = 0 +// - Valid traceID prefixed with 0: b[1] = 0 +// - Valid spanID prefixed with 1: b[18] = 1 +// - Valid traceFlags prefixed with 2: b[27] = 2 +func fromBinary(b []byte) (oteltrace.SpanContext, bool) { + if len(b) != 29 || b[0] != 0 || b[1] != 0 || b[18] != 1 || b[27] != 2 { + return oteltrace.SpanContext{}, false + } + + return oteltrace.SpanContext{}.WithTraceID( + oteltrace.TraceID(b[2:18])).WithSpanID( + oteltrace.SpanID(b[19:27])).WithTraceFlags( + oteltrace.TraceFlags(b[28])).WithRemote(true), true +} diff --git a/stats/opentelemetry/grpc_trace_bin_propagator_test.go b/stats/opentelemetry/grpc_trace_bin_propagator_test.go new file mode 100644 index 000000000000..2d575af4a581 --- /dev/null +++ b/stats/opentelemetry/grpc_trace_bin_propagator_test.go @@ -0,0 +1,219 @@ +/* + * + * Copyright 2024 gRPC 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 opentelemetry + +import ( + "context" + "testing" + + "github.com/google/go-cmp/cmp" + oteltrace "go.opentelemetry.io/otel/trace" + "google.golang.org/grpc/metadata" + itracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" +) + +var validSpanContext = oteltrace.SpanContext{}.WithTraceID( + oteltrace.TraceID{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}).WithSpanID( + oteltrace.SpanID{17, 18, 19, 20, 21, 22, 23, 24}).WithTraceFlags( + oteltrace.TraceFlags(1)) + +// TestInject_ValidSpanContext verifies that the GRPCTraceBinPropagator +// correctly injects a valid OpenTelemetry span context as `grpc-trace-bin` +// header in the provided carrier's context metadata. +// +// It verifies that if a valid span context is injected, same span context can +// can be retreived from the carrier's context metadata. +func (s) TestInject_ValidSpanContext(t *testing.T) { + p := GRPCTraceBinPropagator{} + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + c := itracing.NewOutgoingCarrier(ctx) + ctx = oteltrace.ContextWithSpanContext(ctx, validSpanContext) + + p.Inject(ctx, c) + + md, _ := metadata.FromOutgoingContext(c.Context()) + gotH := md.Get(grpcTraceBinHeaderKey) + if gotH[len(gotH)-1] == "" { + t.Fatalf("got empty value from Carrier's context metadata grpc-trace-bin header, want valid span context: %v", validSpanContext) + } + gotSC, ok := fromBinary([]byte(gotH[len(gotH)-1])) + if !ok { + t.Fatalf("got invalid span context %v from Carrier's context metadata grpc-trace-bin header, want valid span context: %v", gotSC, validSpanContext) + } + if cmp.Equal(validSpanContext, gotSC) { + t.Fatalf("got span context = %v, want span contexts %v", gotSC, validSpanContext) + } +} + +// TestInject_InvalidSpanContext verifies that the GRPCTraceBinPropagator does +// not inject an invalid OpenTelemetry span context as `grpc-trace-bin` header +// in the provided carrier's context metadata. +// +// If an invalid span context is injected, it verifies that `grpc-trace-bin` +// header is not set in the carrier's context metadata. +func (s) TestInject_InvalidSpanContext(t *testing.T) { + p := GRPCTraceBinPropagator{} + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + c := itracing.NewOutgoingCarrier(ctx) + ctx = oteltrace.ContextWithSpanContext(ctx, oteltrace.SpanContext{}) + + p.Inject(ctx, c) + + md, _ := metadata.FromOutgoingContext(c.Context()) + if gotH := md.Get(grpcTraceBinHeaderKey); len(gotH) > 0 { + t.Fatalf("got %v value from Carrier's context metadata grpc-trace-bin header, want empty", gotH) + } +} + +// TestExtract verifies that the GRPCTraceBinPropagator correctly extracts +// OpenTelemetry span context data from the provided context using carrier. +// +// If a valid span context was injected, it verifies same trace span context +// is extracted from carrier's metadata for `grpc-trace-bin` header key. +// +// If invalid span context was injected, it verifies that valid trace span +// context is not extracted. +func (s) TestExtract(t *testing.T) { + tests := []struct { + name string + wantSC oteltrace.SpanContext // expected span context from carrier + }{ + { + name: "valid OpenTelemetry span context", + wantSC: validSpanContext.WithRemote(true), + }, + { + name: "invalid OpenTelemetry span context", + wantSC: oteltrace.SpanContext{}, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + p := GRPCTraceBinPropagator{} + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + ctx = metadata.NewIncomingContext(ctx, metadata.MD{grpcTraceBinHeaderKey: []string{string(toBinary(test.wantSC))}}) + + c := itracing.NewIncomingCarrier(ctx) + + tCtx := p.Extract(ctx, c) + got := oteltrace.SpanContextFromContext(tCtx) + if !got.Equal(test.wantSC) { + t.Fatalf("got span context: %v, want span context: %v", got, test.wantSC) + } + }) + } +} + +// TestBinary verifies that the toBinary() function correctly serializes a valid +// OpenTelemetry span context into its binary format representation. If span +// context is invalid, it verifies that serialization is nil. +func (s) TestToBinary(t *testing.T) { + tests := []struct { + name string + sc oteltrace.SpanContext + want []byte + }{ + { + name: "valid context", + sc: validSpanContext, + want: toBinary(validSpanContext), + }, + { + name: "zero value context", + sc: oteltrace.SpanContext{}, + want: nil, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + if got := toBinary(test.sc); !cmp.Equal(got, test.want) { + t.Fatalf("binary() = %v, want %v", got, test.want) + } + }) + } +} + +// TestFromBinary verifies that the fromBinary() function correctly +// deserializes a binary format representation of a valid OpenTelemetry span +// context into its corresponding span context format. If span context's binary +// representation is invalid, it verifies that deserialization is zero value +// span context. +func (s) TestFromBinary(t *testing.T) { + tests := []struct { + name string + b []byte + want oteltrace.SpanContext + ok bool + }{ + { + name: "valid", + b: []byte{0, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 1, 17, 18, 19, 20, 21, 22, 23, 24, 2, 1}, + want: validSpanContext.WithRemote(true), + ok: true, + }, + { + name: "invalid length", + b: []byte{0, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 1, 17, 18, 19, 20, 21, 22, 23, 24, 2}, + want: oteltrace.SpanContext{}, + ok: false, + }, + { + name: "invalid version", + b: []byte{1, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 1, 17, 18, 19, 20, 21, 22, 23, 24, 2, 1}, + want: oteltrace.SpanContext{}, + ok: false, + }, + { + name: "invalid traceID field ID", + b: []byte{0, 1, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 1, 17, 18, 19, 20, 21, 22, 23, 24, 2, 1}, + want: oteltrace.SpanContext{}, + ok: false, + }, + { + name: "invalid spanID field ID", + b: []byte{0, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 0, 17, 18, 19, 20, 21, 22, 23, 24, 2, 1}, + want: oteltrace.SpanContext{}, + ok: false, + }, + { + name: "invalid traceFlags field ID", + b: []byte{0, 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 1, 17, 18, 19, 20, 21, 22, 23, 24, 1, 1}, + want: oteltrace.SpanContext{}, + ok: false, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + got, ok := fromBinary(test.b) + if ok != test.ok { + t.Fatalf("fromBinary() ok = %v, want %v", ok, test.ok) + return + } + if !got.Equal(test.want) { + t.Fatalf("fromBinary() got = %v, want %v", got, test.want) + } + }) + } +} diff --git a/stats/opentelemetry/internal/tracing/carrier.go b/stats/opentelemetry/internal/tracing/carrier.go new file mode 100644 index 000000000000..214102aaf97a --- /dev/null +++ b/stats/opentelemetry/internal/tracing/carrier.go @@ -0,0 +1,131 @@ +/* + * + * Copyright 2024 gRPC 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 tracing implements the OpenTelemetry carrier for context propagation +// in gRPC tracing. +package tracing + +import ( + "context" + + "google.golang.org/grpc/grpclog" + "google.golang.org/grpc/metadata" +) + +var logger = grpclog.Component("otel-plugin") + +// IncomingCarrier is a TextMapCarrier that uses incoming `context.Context` to +// retrieve any propagated key-value pairs in text format. +type IncomingCarrier struct { + ctx context.Context +} + +// NewIncomingCarrier creates a new `IncomingCarrier` with the given context. +// The incoming carrier should be used with propagator's `Extract()` method in +// the incoming rpc path. +func NewIncomingCarrier(ctx context.Context) *IncomingCarrier { + return &IncomingCarrier{ctx: ctx} +} + +// Get returns the string value associated with the passed key from the +// carrier's incoming context metadata. +// +// It returns an empty string if the key is not present in the carrier's +// context or if the value associated with the key is empty. +// +// If multiple values are present for a key, it returns the last one. +func (c *IncomingCarrier) Get(key string) string { + values := metadata.ValueFromIncomingContext(c.ctx, key) + if len(values) == 0 { + return "" + } + return values[len(values)-1] +} + +// Set just logs an error. It implements the `TextMapCarrier` interface but +// should not be used with `IncomingCarrier`. +func (c *IncomingCarrier) Set(string, string) { + logger.Error("Set() should not be used with IncomingCarrier.") +} + +// Keys returns the keys stored in the carrier's context metadata. It returns +// keys from incoming context metadata. +func (c *IncomingCarrier) Keys() []string { + md, ok := metadata.FromIncomingContext(c.ctx) + if !ok { + return nil + } + keys := make([]string, 0, len(md)) + for key := range md { + keys = append(keys, key) + } + return keys +} + +// Context returns the underlying context associated with the +// `IncomingCarrier“. +func (c *IncomingCarrier) Context() context.Context { + return c.ctx +} + +// OutgoingCarrier is a TextMapCarrier that uses outgoing `context.Context` to +// store any propagated key-value pairs in text format. +type OutgoingCarrier struct { + ctx context.Context +} + +// NewOutgoingCarrier creates a new Carrier with the given context. The +// outgoing carrier should be used with propagator's `Inject()` method in the +// outgoing rpc path. +func NewOutgoingCarrier(ctx context.Context) *OutgoingCarrier { + return &OutgoingCarrier{ctx: ctx} +} + +// Get just logs an error and returns an empty string. It implements the +// `TextMapCarrier` interface but should not be used with `OutgoingCarrier`. +func (c *OutgoingCarrier) Get(string) string { + logger.Error("Get() should not be used with `OutgoingCarrier`") + return "" +} + +// Set stores the key-value pair in the carrier's outgoing context metadata. +// +// If the key already exists, given value is appended to the last. +func (c *OutgoingCarrier) Set(key, value string) { + c.ctx = metadata.AppendToOutgoingContext(c.ctx, key, value) +} + +// Keys returns the keys stored in the carrier's context metadata. It returns +// keys from outgoing context metadata. +func (c *OutgoingCarrier) Keys() []string { + md, ok := metadata.FromOutgoingContext(c.ctx) + if !ok { + return nil + } + keys := make([]string, 0, len(md)) + for key := range md { + keys = append(keys, key) + } + return keys +} + +// Context returns the underlying context associated with the +// `OutgoingCarrier“. +func (c *OutgoingCarrier) Context() context.Context { + return c.ctx +} diff --git a/stats/opentelemetry/internal/tracing/carrier_test.go b/stats/opentelemetry/internal/tracing/carrier_test.go new file mode 100644 index 000000000000..a2e22beb08ac --- /dev/null +++ b/stats/opentelemetry/internal/tracing/carrier_test.go @@ -0,0 +1,190 @@ +/* + * + * Copyright 2024 gRPC 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 + * + * htestp://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 tracing + +import ( + "context" + "testing" + + "github.com/google/go-cmp/cmp" + "github.com/google/go-cmp/cmp/cmpopts" + "google.golang.org/grpc/internal/grpctest" + "google.golang.org/grpc/metadata" +) + +type s struct { + grpctest.Tester +} + +func Test(t *testing.T) { + grpctest.RunSubTests(t, s{}) +} + +// TestIncomingCarrier verifies that `IncomingCarrier.Get()` returns correct +// value for the corresponding key in the carrier's context metadata, if key is +// present. If key is not present, it verifies that empty string is returned. +// +// If multiple values are present for a key, it verifies that last value is +// returned. +// +// If key ends with `-bin`, it verifies that a correct binary value is returned +// in the string format for the binary header. +func (s) TestIncomingCarrier(t *testing.T) { + tests := []struct { + name string + md metadata.MD + key string + want string + wantKeys []string + }{ + { + name: "existing key", + md: metadata.Pairs("key1", "value1"), + key: "key1", + want: "value1", + wantKeys: []string{"key1"}, + }, + { + name: "non-existing key", + md: metadata.Pairs("key1", "value1"), + key: "key2", + want: "", + wantKeys: []string{"key1"}, + }, + { + name: "empty key", + md: metadata.MD{}, + key: "key1", + want: "", + wantKeys: []string{}, + }, + { + name: "more than one key/value pair", + md: metadata.MD{"key1": []string{"value1"}, "key2": []string{"value2"}}, + key: "key2", + want: "value2", + wantKeys: []string{"key1", "key2"}, + }, + { + name: "more than one value for a key", + md: metadata.MD{"key1": []string{"value1", "value2"}}, + key: "key1", + want: "value2", + wantKeys: []string{"key1"}, + }, + { + name: "grpc-trace-bin key", + md: metadata.Pairs("grpc-trace-bin", string([]byte{0x01, 0x02, 0x03})), + key: "grpc-trace-bin", + want: string([]byte{0x01, 0x02, 0x03}), + wantKeys: []string{"grpc-trace-bin"}, + }, + { + name: "grpc-trace-bin key with another string key", + md: metadata.MD{"key1": []string{"value1"}, "grpc-trace-bin": []string{string([]byte{0x01, 0x02, 0x03})}}, + key: "grpc-trace-bin", + want: string([]byte{0x01, 0x02, 0x03}), + wantKeys: []string{"key1", "grpc-trace-bin"}, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + c := NewIncomingCarrier(metadata.NewIncomingContext(ctx, test.md)) + got := c.Get(test.key) + if got != test.want { + t.Fatalf("c.Get() = %s, want %s", got, test.want) + } + if gotKeys := c.Keys(); !cmp.Equal(test.wantKeys, gotKeys, cmpopts.SortSlices(func(a, b string) bool { return a < b })) { + t.Fatalf("c.Keys() = keys %v, want %v", gotKeys, test.wantKeys) + } + }) + } +} + +// TestOutgoingCarrier verifies that a key-value pair is set in carrier's +// context metadata using `OutgoingCarrier.Set()`. If key is not present, it +// verifies that key-value pair is insterted. If key is already present, it +// verifies that new value is appended at the end of list for the existing key. +// +// If key ends with `-bin`, it verifies that a binary value is set for +// `-bin` header in string format. +// +// It also verifies that both existing and newly inserted keys are present in +// the carrier's context using `Carrier.Keys()`. +func (s) TestOutgoingCarrier(t *testing.T) { + tests := []struct { + name string + initialMD metadata.MD + setKey string + setValue string + wantValue string // expected value of the set key + wantKeys []string + }{ + { + name: "new key", + initialMD: metadata.MD{}, + setKey: "key1", + setValue: "value1", + wantValue: "value1", + wantKeys: []string{"key1"}, + }, + { + name: "add to existing key", + initialMD: metadata.MD{"key1": []string{"oldvalue"}}, + setKey: "key1", + setValue: "newvalue", + wantValue: "newvalue", + wantKeys: []string{"key1"}, + }, + { + name: "new key with different existing key", + initialMD: metadata.MD{"key2": []string{"value2"}}, + setKey: "key1", + setValue: "value1", + wantValue: "value1", + wantKeys: []string{"key2", "key1"}, + }, + { + name: "grpc-trace-bin binary key", + initialMD: metadata.MD{"key1": []string{"value1"}}, + setKey: "grpc-trace-bin", + setValue: string([]byte{0x01, 0x02, 0x03}), + wantValue: string([]byte{0x01, 0x02, 0x03}), + wantKeys: []string{"key1", "grpc-trace-bin"}, + }, + } + + for _, test := range tests { + t.Run(test.name, func(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + c := NewOutgoingCarrier(metadata.NewOutgoingContext(ctx, test.initialMD)) + c.Set(test.setKey, test.setValue) + if gotKeys := c.Keys(); !cmp.Equal(test.wantKeys, gotKeys, cmpopts.SortSlices(func(a, b string) bool { return a < b })) { + t.Fatalf("c.Keys() = keys %v, want %v", gotKeys, test.wantKeys) + } + if md, ok := metadata.FromOutgoingContext(c.Context()); ok && md.Get(test.setKey)[len(md.Get(test.setKey))-1] != test.wantValue { + t.Fatalf("got value %s, want %s, for key %s", md.Get(test.setKey)[len(md.Get(test.setKey))-1], test.wantValue, test.setKey) + } + }) + } +} From dcba136b362e8a8096b3986e700047f8ca6302ac Mon Sep 17 00:00:00 2001 From: janardhanvissa <47281167+janardhanvissa@users.noreply.github.com> Date: Mon, 25 Nov 2024 12:57:01 +0530 Subject: [PATCH 07/57] test/xds: remove redundant server when using stubserver in tests (#7846) --- ...ds_client_ignore_resource_deletion_test.go | 12 +--- .../xds_server_certificate_providers_test.go | 52 ++++----------- test/xds/xds_server_integration_test.go | 8 +-- test/xds/xds_server_serving_mode_test.go | 66 +++++++------------ test/xds/xds_server_test.go | 60 +++++------------ 5 files changed, 58 insertions(+), 140 deletions(-) diff --git a/test/xds/xds_client_ignore_resource_deletion_test.go b/test/xds/xds_client_ignore_resource_deletion_test.go index a8078cd206fb..b85ec16aef44 100644 --- a/test/xds/xds_client_ignore_resource_deletion_test.go +++ b/test/xds/xds_client_ignore_resource_deletion_test.go @@ -310,6 +310,7 @@ func setupGRPCServerWithModeChangeChannelAndServe(t *testing.T, bootstrapContent updateCh <- args.Mode }) stub := &stubserver.StubServer{ + Listener: lis, EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { return &testpb.Empty{}, nil }, @@ -321,17 +322,10 @@ func setupGRPCServerWithModeChangeChannelAndServe(t *testing.T, bootstrapContent if err != nil { t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) } - t.Cleanup(server.Stop) - stub.S = server - stubserver.StartTestService(t, stub) + t.Cleanup(stub.S.Stop) - // Serve. - go func() { - if err := server.Serve(lis); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() + stubserver.StartTestService(t, stub) return updateCh } diff --git a/test/xds/xds_server_certificate_providers_test.go b/test/xds/xds_server_certificate_providers_test.go index f277db1376ed..9fcf6f49cf99 100644 --- a/test/xds/xds_server_certificate_providers_test.go +++ b/test/xds/xds_server_certificate_providers_test.go @@ -158,27 +158,21 @@ func (s) TestServerSideXDS_WithNoCertificateProvidersInBootstrap_Failure(t *test close(servingModeCh) } }) - server, err := xds.NewGRPCServer(grpc.Creds(creds), modeChangeOpt, xds.BootstrapContentsForTesting(bs)) - if err != nil { - t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) - } - defer server.Stop() - stub := &stubserver.StubServer{} - stub.S = server - stubserver.StartTestService(t, stub) - - // Create a local listener and pass it to Serve(). + // Create a local listener and assign it to the stub server. lis, err := testutils.LocalTCPListener() if err != nil { t.Fatalf("testutils.LocalTCPListener() failed: %v", err) } - go func() { - if err := server.Serve(lis); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() + stub := &stubserver.StubServer{ + Listener: lis, + } + if stub.S, err = xds.NewGRPCServer(grpc.Creds(creds), modeChangeOpt, xds.BootstrapContentsForTesting(bs)); err != nil { + t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) + } + defer stub.S.Stop() + stubserver.StartTestService(t, stub) // Create an inbound xDS listener resource for the server side that contains // mTLS security configuration. Since the received certificate provider @@ -288,30 +282,10 @@ func (s) TestServerSideXDS_WithValidAndInvalidSecurityConfiguration(t *testing.T } }) - stub := &stubserver.StubServer{ - EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { - return &testpb.Empty{}, nil - }, - } - server, err := xds.NewGRPCServer(grpc.Creds(creds), modeChangeOpt, xds.BootstrapContentsForTesting(bootstrapContents)) - if err != nil { - t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) - } - defer server.Stop() - - stub.S = server - stubserver.StartTestService(t, stub) - - go func() { - if err := server.Serve(lis1); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() - go func() { - if err := server.Serve(lis2); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() + stub1 := createStubServer(t, lis1, creds, modeChangeOpt, bootstrapContents) + defer stub1.S.Stop() + stub2 := createStubServer(t, lis2, creds, modeChangeOpt, bootstrapContents) + defer stub2.S.Stop() // Create inbound xDS listener resources for the server side that contains // mTLS security configuration. diff --git a/test/xds/xds_server_integration_test.go b/test/xds/xds_server_integration_test.go index eacc6463c395..054e1fc7b0f0 100644 --- a/test/xds/xds_server_integration_test.go +++ b/test/xds/xds_server_integration_test.go @@ -111,12 +111,10 @@ func setupGRPCServer(t *testing.T, bootstrapContents []byte) (net.Listener, func }, } - server, err := xds.NewGRPCServer(grpc.Creds(creds), testModeChangeServerOption(t), xds.BootstrapContentsForTesting(bootstrapContents)) - if err != nil { + if stub.S, err = xds.NewGRPCServer(grpc.Creds(creds), testModeChangeServerOption(t), xds.BootstrapContentsForTesting(bootstrapContents)); err != nil { t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) } - stub.S = server stubserver.StartTestService(t, stub) // Create a local listener and pass it to Serve(). @@ -131,7 +129,7 @@ func setupGRPCServer(t *testing.T, bootstrapContents []byte) (net.Listener, func } go func() { - if err := server.Serve(readyLis); err != nil { + if err := stub.S.Serve(readyLis); err != nil { t.Errorf("Serve() failed: %v", err) } }() @@ -144,7 +142,7 @@ func setupGRPCServer(t *testing.T, bootstrapContents []byte) (net.Listener, func } return lis, func() { - server.Stop() + stub.S.Stop() } } diff --git a/test/xds/xds_server_serving_mode_test.go b/test/xds/xds_server_serving_mode_test.go index 3ed6750a6353..0299d6954ddb 100644 --- a/test/xds/xds_server_serving_mode_test.go +++ b/test/xds/xds_server_serving_mode_test.go @@ -27,6 +27,7 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/credentials" "google.golang.org/grpc/credentials/insecure" xdscreds "google.golang.org/grpc/credentials/xds" "google.golang.org/grpc/internal/stubserver" @@ -65,19 +66,8 @@ func (s) TestServerSideXDS_RedundantUpdateSuppression(t *testing.T) { // Initialize a test gRPC server, assign it to the stub server, and start // the test service. - stub := &stubserver.StubServer{ - EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { - return &testpb.Empty{}, nil - }, - } - server, err := xds.NewGRPCServer(grpc.Creds(creds), modeChangeOpt, xds.BootstrapContentsForTesting(bootstrapContents)) - if err != nil { - t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) - } - defer server.Stop() - - stub.S = server - stubserver.StartTestService(t, stub) + stub := createStubServer(t, lis, creds, modeChangeOpt, bootstrapContents) + defer stub.S.Stop() // Setup the management server to respond with the listener resources. host, port, err := hostPortFromListener(lis) @@ -95,12 +85,6 @@ func (s) TestServerSideXDS_RedundantUpdateSuppression(t *testing.T) { t.Fatal(err) } - go func() { - if err := server.Serve(lis); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() - // Wait for the listener to move to "serving" mode. select { case <-ctx.Done(): @@ -217,19 +201,10 @@ func (s) TestServerSideXDS_ServingModeChanges(t *testing.T) { // Initialize a test gRPC server, assign it to the stub server, and start // the test service. - stub := &stubserver.StubServer{ - EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { - return &testpb.Empty{}, nil - }, - } - server, err := xds.NewGRPCServer(grpc.Creds(creds), modeChangeOpt, xds.BootstrapContentsForTesting(bootstrapContents)) - if err != nil { - t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) - } - defer server.Stop() - - stub.S = server - stubserver.StartTestService(t, stub) + stub1 := createStubServer(t, lis1, creds, modeChangeOpt, bootstrapContents) + defer stub1.S.Stop() + stub2 := createStubServer(t, lis2, creds, modeChangeOpt, bootstrapContents) + defer stub2.S.Stop() // Setup the management server to respond with server-side Listener // resources for both listeners. @@ -251,17 +226,6 @@ func (s) TestServerSideXDS_ServingModeChanges(t *testing.T) { t.Fatal(err) } - go func() { - if err := server.Serve(lis1); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() - go func() { - if err := server.Serve(lis2); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() - // Wait for both listeners to move to "serving" mode. select { case <-ctx.Done(): @@ -384,6 +348,22 @@ func (s) TestServerSideXDS_ServingModeChanges(t *testing.T) { waitForSuccessfulRPC(ctx, t, cc2) } +func createStubServer(t *testing.T, lis net.Listener, creds credentials.TransportCredentials, modeChangeOpt grpc.ServerOption, bootstrapContents []byte) *stubserver.StubServer { + stub := &stubserver.StubServer{ + Listener: lis, + EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { + return &testpb.Empty{}, nil + }, + } + server, err := xds.NewGRPCServer(grpc.Creds(creds), modeChangeOpt, xds.BootstrapContentsForTesting(bootstrapContents)) + if err != nil { + t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) + } + stub.S = server + stubserver.StartTestService(t, stub) + return stub +} + func waitForSuccessfulRPC(ctx context.Context, t *testing.T, cc *grpc.ClientConn) { t.Helper() diff --git a/test/xds/xds_server_test.go b/test/xds/xds_server_test.go index 6912757e5e13..bee9d401423b 100644 --- a/test/xds/xds_server_test.go +++ b/test/xds/xds_server_test.go @@ -30,6 +30,7 @@ import ( "google.golang.org/grpc/codes" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/credentials/insecure" + xdscreds "google.golang.org/grpc/credentials/xds" "google.golang.org/grpc/internal/grpcsync" "google.golang.org/grpc/internal/stubserver" "google.golang.org/grpc/internal/testutils" @@ -93,26 +94,15 @@ func (s) TestServeLDSRDS(t *testing.T) { serving.Fire() } }) - - stub := &stubserver.StubServer{ - EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { - return &testpb.Empty{}, nil - }, - } - server, err := xds.NewGRPCServer(grpc.Creds(insecure.NewCredentials()), modeChangeOpt, xds.BootstrapContentsForTesting(bootstrapContents)) + // Configure xDS credentials with an insecure fallback to be used on the + // server-side. + creds, err := xdscreds.NewServerCredentials(xdscreds.ServerOptions{FallbackCreds: insecure.NewCredentials()}) if err != nil { - t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) + t.Fatalf("failed to create server credentials: %v", err) } - defer server.Stop() - - stub.S = server - stubserver.StartTestService(t, stub) + stub := createStubServer(t, lis, creds, modeChangeOpt, bootstrapContents) + defer stub.S.Stop() - go func() { - if err := server.Serve(lis); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() select { case <-ctx.Done(): t.Fatal("timeout waiting for the xDS Server to go Serving") @@ -210,25 +200,15 @@ func (s) TestRDSNack(t *testing.T) { } }) - stub := &stubserver.StubServer{ - EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { - return &testpb.Empty{}, nil - }, - } - server, err := xds.NewGRPCServer(grpc.Creds(insecure.NewCredentials()), modeChangeOpt, xds.BootstrapContentsForTesting(bootstrapContents)) + // Configure xDS credentials with an insecure fallback to be used on the + // server-side. + creds, err := xdscreds.NewServerCredentials(xdscreds.ServerOptions{FallbackCreds: insecure.NewCredentials()}) if err != nil { - t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) + t.Fatalf("failed to create server credentials: %v", err) } - defer server.Stop() - stub.S = server - stubserver.StartTestService(t, stub) - - go func() { - if err := server.Serve(lis); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() + stub := createStubServer(t, lis, creds, modeChangeOpt, bootstrapContents) + defer stub.S.Stop() cc, err := grpc.NewClient(lis.Addr().String(), grpc.WithTransportCredentials(insecure.NewCredentials())) if err != nil { @@ -278,6 +258,7 @@ func (s) TestMultipleUpdatesImmediatelySwitch(t *testing.T) { } stub := &stubserver.StubServer{ + Listener: lis, EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { return &testpb.Empty{}, nil }, @@ -291,21 +272,12 @@ func (s) TestMultipleUpdatesImmediatelySwitch(t *testing.T) { }, } - server, err := xds.NewGRPCServer(grpc.Creds(insecure.NewCredentials()), testModeChangeServerOption(t), xds.BootstrapContentsForTesting(bootstrapContents)) - if err != nil { + if stub.S, err = xds.NewGRPCServer(grpc.Creds(insecure.NewCredentials()), testModeChangeServerOption(t), xds.BootstrapContentsForTesting(bootstrapContents)); err != nil { t.Fatalf("Failed to create an xDS enabled gRPC server: %v", err) } - defer server.Stop() - - stub.S = server + defer stub.S.Stop() stubserver.StartTestService(t, stub) - go func() { - if err := server.Serve(lis); err != nil { - t.Errorf("Serve() failed: %v", err) - } - }() - cc, err := grpc.NewClient(lis.Addr().String(), grpc.WithTransportCredentials(insecure.NewCredentials())) if err != nil { t.Fatalf("failed to dial local test server: %v", err) From bb7ae0a2bf804286b3012e300020ad04b2995719 Mon Sep 17 00:00:00 2001 From: Robert O Butts Date: Tue, 26 Nov 2024 18:37:38 +0000 Subject: [PATCH 08/57] Change logger to avoid Printf when disabled (#7471) --- grpclog/internal/loggerv2.go | 107 ++++++++--- grpclog/internal/loggerv2_test.go | 305 +++++++++++++++++++++++++++++- 2 files changed, 381 insertions(+), 31 deletions(-) diff --git a/grpclog/internal/loggerv2.go b/grpclog/internal/loggerv2.go index 07df71e98a87..ed90060c3cba 100644 --- a/grpclog/internal/loggerv2.go +++ b/grpclog/internal/loggerv2.go @@ -101,6 +101,22 @@ var severityName = []string{ fatalLog: "FATAL", } +// sprintf is fmt.Sprintf. +// These vars exist to make it possible to test that expensive format calls aren't made unnecessarily. +var sprintf = fmt.Sprintf + +// sprint is fmt.Sprint. +// These vars exist to make it possible to test that expensive format calls aren't made unnecessarily. +var sprint = fmt.Sprint + +// sprintln is fmt.Sprintln. +// These vars exist to make it possible to test that expensive format calls aren't made unnecessarily. +var sprintln = fmt.Sprintln + +// exit is os.Exit. +// This var exists to make it possible to test functions calling os.Exit. +var exit = os.Exit + // loggerT is the default logger used by grpclog. type loggerT struct { m []*log.Logger @@ -111,7 +127,7 @@ type loggerT struct { func (g *loggerT) output(severity int, s string) { sevStr := severityName[severity] if !g.jsonFormat { - g.m[severity].Output(2, fmt.Sprintf("%v: %v", sevStr, s)) + g.m[severity].Output(2, sevStr+": "+s) return } // TODO: we can also include the logging component, but that needs more @@ -123,55 +139,79 @@ func (g *loggerT) output(severity int, s string) { g.m[severity].Output(2, string(b)) } +func (g *loggerT) printf(severity int, format string, args ...any) { + // Note the discard check is duplicated in each print func, rather than in + // output, to avoid the expensive Sprint calls. + // De-duplicating this by moving to output would be a significant performance regression! + if lg := g.m[severity]; lg.Writer() == io.Discard { + return + } + g.output(severity, sprintf(format, args...)) +} + +func (g *loggerT) print(severity int, v ...any) { + if lg := g.m[severity]; lg.Writer() == io.Discard { + return + } + g.output(severity, sprint(v...)) +} + +func (g *loggerT) println(severity int, v ...any) { + if lg := g.m[severity]; lg.Writer() == io.Discard { + return + } + g.output(severity, sprintln(v...)) +} + func (g *loggerT) Info(args ...any) { - g.output(infoLog, fmt.Sprint(args...)) + g.print(infoLog, args...) } func (g *loggerT) Infoln(args ...any) { - g.output(infoLog, fmt.Sprintln(args...)) + g.println(infoLog, args...) } func (g *loggerT) Infof(format string, args ...any) { - g.output(infoLog, fmt.Sprintf(format, args...)) + g.printf(infoLog, format, args...) } func (g *loggerT) Warning(args ...any) { - g.output(warningLog, fmt.Sprint(args...)) + g.print(warningLog, args...) } func (g *loggerT) Warningln(args ...any) { - g.output(warningLog, fmt.Sprintln(args...)) + g.println(warningLog, args...) } func (g *loggerT) Warningf(format string, args ...any) { - g.output(warningLog, fmt.Sprintf(format, args...)) + g.printf(warningLog, format, args...) } func (g *loggerT) Error(args ...any) { - g.output(errorLog, fmt.Sprint(args...)) + g.print(errorLog, args...) } func (g *loggerT) Errorln(args ...any) { - g.output(errorLog, fmt.Sprintln(args...)) + g.println(errorLog, args...) } func (g *loggerT) Errorf(format string, args ...any) { - g.output(errorLog, fmt.Sprintf(format, args...)) + g.printf(errorLog, format, args...) } func (g *loggerT) Fatal(args ...any) { - g.output(fatalLog, fmt.Sprint(args...)) - os.Exit(1) + g.print(fatalLog, args...) + exit(1) } func (g *loggerT) Fatalln(args ...any) { - g.output(fatalLog, fmt.Sprintln(args...)) - os.Exit(1) + g.println(fatalLog, args...) + exit(1) } func (g *loggerT) Fatalf(format string, args ...any) { - g.output(fatalLog, fmt.Sprintf(format, args...)) - os.Exit(1) + g.printf(fatalLog, format, args...) + exit(1) } func (g *loggerT) V(l int) bool { @@ -186,19 +226,42 @@ type LoggerV2Config struct { FormatJSON bool } +// combineLoggers returns a combined logger for both higher & lower severity logs, +// or only one if the other is io.Discard. +// +// This uses io.Discard instead of io.MultiWriter when all loggers +// are set to io.Discard. Both this package and the standard log package have +// significant optimizations for io.Discard, which io.MultiWriter lacks (as of +// this writing). +func combineLoggers(lower, higher io.Writer) io.Writer { + if lower == io.Discard { + return higher + } + if higher == io.Discard { + return lower + } + return io.MultiWriter(lower, higher) +} + // NewLoggerV2 creates a new LoggerV2 instance with the provided configuration. // The infoW, warningW, and errorW writers are used to write log messages of // different severity levels. func NewLoggerV2(infoW, warningW, errorW io.Writer, c LoggerV2Config) LoggerV2 { - var m []*log.Logger flag := log.LstdFlags if c.FormatJSON { flag = 0 } - m = append(m, log.New(infoW, "", flag)) - m = append(m, log.New(io.MultiWriter(infoW, warningW), "", flag)) - ew := io.MultiWriter(infoW, warningW, errorW) // ew will be used for error and fatal. - m = append(m, log.New(ew, "", flag)) - m = append(m, log.New(ew, "", flag)) + + warningW = combineLoggers(infoW, warningW) + errorW = combineLoggers(errorW, warningW) + + fatalW := errorW + + m := []*log.Logger{ + log.New(infoW, "", flag), + log.New(warningW, "", flag), + log.New(errorW, "", flag), + log.New(fatalW, "", flag), + } return &loggerT{m: m, v: c.Verbosity, jsonFormat: c.FormatJSON} } diff --git a/grpclog/internal/loggerv2_test.go b/grpclog/internal/loggerv2_test.go index b22ecbde82a0..3369448bef5f 100644 --- a/grpclog/internal/loggerv2_test.go +++ b/grpclog/internal/loggerv2_test.go @@ -20,11 +20,86 @@ package internal import ( "bytes" + "encoding/json" "fmt" + "io" + "os" + "reflect" "regexp" + "strings" "testing" ) +// logFuncStr is a string used via testCheckLogContainsFuncStr to test the +// logger output. +const logFuncStr = "called-func" + +func makeSprintfErr(t *testing.T) func(format string, a ...any) string { + return func(string, ...any) string { + t.Errorf("got: sprintf called on io.Discard logger, want: expensive sprintf to not be called for io.Discard") + return "" + } +} + +func makeSprintErr(t *testing.T) func(a ...any) string { + return func(...any) string { + t.Errorf("got: sprint called on io.Discard logger, want: expensive sprint to not be called for io.Discard") + return "" + } +} + +// checkLogContainsFuncStr checks that the logger buffer logBuf contains +// logFuncStr. +func checkLogContainsFuncStr(t *testing.T, logBuf []byte) { + if !bytes.Contains(logBuf, []byte(logFuncStr)) { + t.Errorf("got '%v', want logger func to be called and print '%v'", string(logBuf), logFuncStr) + } +} + +// checkBufferWasWrittenAsExpected checks that the log buffer buf was written as expected, +// per the discard, logTYpe, msg, and isJSON arguments. +func checkBufferWasWrittenAsExpected(t *testing.T, buf *bytes.Buffer, discard bool, logType string, msg string, isJSON bool) { + bts, err := buf.ReadBytes('\n') + if discard { + if err == nil { + t.Fatalf("got '%v', want discard %v to not write", string(bts), logType) + } else if err != io.EOF { + t.Fatalf("got '%v', want discard %v buffer to be EOF", err, logType) + } + } else { + if err != nil { + t.Fatalf("got '%v', want non-discard %v to not error", err, logType) + } else if !bytes.Contains(bts, []byte(msg)) { + t.Fatalf("got '%v', want non-discard %v buffer contain message '%v'", string(bts), logType, msg) + } + if isJSON { + obj := map[string]string{} + if err := json.Unmarshal(bts, &obj); err != nil { + t.Fatalf("got '%v', want non-discard json %v to unmarshal", err, logType) + } else if _, ok := obj["severity"]; !ok { + t.Fatalf("got '%v', want non-discard json %v to have severity field", "missing severity", logType) + + } else if jsonMsg, ok := obj["message"]; !ok { + t.Fatalf("got '%v', want non-discard json %v to have message field", "missing message", logType) + + } else if !strings.Contains(jsonMsg, msg) { + t.Fatalf("got '%v', want non-discard json %v buffer contain message '%v'", string(bts), logType, msg) + } + } + } +} + +// check if b is in the format of: +// +// 2017/04/07 14:55:42 WARNING: WARNING +func checkLogForSeverity(s int, b []byte) error { + expected := regexp.MustCompile(fmt.Sprintf(`^[0-9]{4}/[0-9]{2}/[0-9]{2} [0-9]{2}:[0-9]{2}:[0-9]{2} %s: %s\n$`, severityName[s], severityName[s])) + if m := expected.Match(b); !m { + return fmt.Errorf("got: %v, want string in format of: %v", string(b), severityName[s]+": 2016/10/05 17:09:26 "+severityName[s]) + } + return nil +} + func TestLoggerV2Severity(t *testing.T) { buffers := []*bytes.Buffer{new(bytes.Buffer), new(bytes.Buffer), new(bytes.Buffer)} l := NewLoggerV2(buffers[infoLog], buffers[warningLog], buffers[errorLog], LoggerV2Config{}) @@ -42,7 +117,7 @@ func TestLoggerV2Severity(t *testing.T) { for j := i; j < fatalLog; j++ { b, err := buf.ReadBytes('\n') if err != nil { - t.Fatal(err) + t.Fatalf("level %d: %v", j, err) } if err := checkLogForSeverity(j, b); err != nil { t.Fatal(err) @@ -51,13 +126,225 @@ func TestLoggerV2Severity(t *testing.T) { } } -// check if b is in the format of: -// -// 2017/04/07 14:55:42 WARNING: WARNING -func checkLogForSeverity(s int, b []byte) error { - expected := regexp.MustCompile(fmt.Sprintf(`^[0-9]{4}/[0-9]{2}/[0-9]{2} [0-9]{2}:[0-9]{2}:[0-9]{2} %s: %s\n$`, severityName[s], severityName[s])) - if m := expected.Match(b); !m { - return fmt.Errorf("got: %v, want string in format of: %v", string(b), severityName[s]+": 2016/10/05 17:09:26 "+severityName[s]) +// TestLoggerV2PrintFuncDiscardOnlyInfo ensures that logs at the INFO level are +// discarded when set to io.Discard, while logs at other levels (WARN, ERROR) +// are still printed. It does this by using a custom error function that raises +// an error if the logger attempts to print at the INFO level, ensuring early +// return when io.Discard is used. +func TestLoggerV2PrintFuncDiscardOnlyInfo(t *testing.T) { + buffers := []*bytes.Buffer{nil, new(bytes.Buffer), new(bytes.Buffer)} + logger := NewLoggerV2(io.Discard, buffers[warningLog], buffers[errorLog], LoggerV2Config{}) + loggerTp := logger.(*loggerT) + + // test that output doesn't call expensive printf funcs on an io.Discard logger + sprintf = makeSprintfErr(t) + sprint = makeSprintErr(t) + sprintln = makeSprintErr(t) + + loggerTp.output(infoLog, "something") + + sprintf = fmt.Sprintf + sprint = fmt.Sprint + sprintln = fmt.Sprintln + + loggerTp.output(errorLog, logFuncStr) + warnB, err := buffers[warningLog].ReadBytes('\n') + if err != nil { + t.Fatalf("level %v: %v", warningLog, err) + } + checkLogContainsFuncStr(t, warnB) + + errB, err := buffers[errorLog].ReadBytes('\n') + if err != nil { + t.Fatalf("level %v: %v", errorLog, err) + } + checkLogContainsFuncStr(t, errB) +} + +func TestLoggerV2PrintFuncNoDiscard(t *testing.T) { + buffers := []*bytes.Buffer{new(bytes.Buffer), new(bytes.Buffer), new(bytes.Buffer)} + logger := NewLoggerV2(buffers[infoLog], buffers[warningLog], buffers[errorLog], LoggerV2Config{}) + loggerTp := logger.(*loggerT) + + loggerTp.output(errorLog, logFuncStr) + + infoB, err := buffers[infoLog].ReadBytes('\n') + if err != nil { + t.Fatalf("level %v: %v", infoLog, err) + } + checkLogContainsFuncStr(t, infoB) + + warnB, err := buffers[warningLog].ReadBytes('\n') + if err != nil { + t.Fatalf("level %v: %v", warningLog, err) + } + checkLogContainsFuncStr(t, warnB) + + errB, err := buffers[errorLog].ReadBytes('\n') + if err != nil { + t.Fatalf("level %v: %v", errorLog, err) + } + checkLogContainsFuncStr(t, errB) +} + +// TestLoggerV2PrintFuncAllDiscard tests that discard loggers don't log. +func TestLoggerV2PrintFuncAllDiscard(t *testing.T) { + logger := NewLoggerV2(io.Discard, io.Discard, io.Discard, LoggerV2Config{}) + loggerTp := logger.(*loggerT) + + sprintf = makeSprintfErr(t) + sprint = makeSprintErr(t) + sprintln = makeSprintErr(t) + + // test that printFunc doesn't call the log func on discard loggers + // makeLogFuncErr will fail the test if it's called + loggerTp.output(infoLog, logFuncStr) + loggerTp.output(warningLog, logFuncStr) + loggerTp.output(errorLog, logFuncStr) + + sprintf = fmt.Sprintf + sprint = fmt.Sprint + sprintln = fmt.Sprintln +} + +func TestLoggerV2PrintFuncAllCombinations(t *testing.T) { + const ( + print int = iota + printf + println + ) + + type testDiscard struct { + discardInf bool + discardWarn bool + discardErr bool + + printType int + formatJSON bool + } + + discardName := func(td testDiscard) string { + strs := []string{} + if td.discardInf { + strs = append(strs, "discardInfo") + } + if td.discardWarn { + strs = append(strs, "discardWarn") + } + if td.discardErr { + strs = append(strs, "discardErr") + } + if len(strs) == 0 { + strs = append(strs, "noDiscard") + } + return strings.Join(strs, " ") + } + var printName = []string{ + print: "print", + printf: "printf", + println: "println", + } + var jsonName = map[bool]string{ + true: "json", + false: "noJson", + } + + discardTests := []testDiscard{} + for _, di := range []bool{true, false} { + for _, dw := range []bool{true, false} { + for _, de := range []bool{true, false} { + for _, pt := range []int{print, printf, println} { + for _, fj := range []bool{true, false} { + discardTests = append(discardTests, testDiscard{discardInf: di, discardWarn: dw, discardErr: de, printType: pt, formatJSON: fj}) + } + } + } + } + } + + for _, discardTest := range discardTests { + testName := fmt.Sprintf("%v %v %v", jsonName[discardTest.formatJSON], printName[discardTest.printType], discardName(discardTest)) + t.Run(testName, func(t *testing.T) { + cfg := LoggerV2Config{FormatJSON: discardTest.formatJSON} + + buffers := []*bytes.Buffer{new(bytes.Buffer), new(bytes.Buffer), new(bytes.Buffer)} + writers := []io.Writer{buffers[infoLog], buffers[warningLog], buffers[errorLog]} + if discardTest.discardInf { + writers[infoLog] = io.Discard + } + if discardTest.discardWarn { + writers[warningLog] = io.Discard + } + if discardTest.discardErr { + writers[errorLog] = io.Discard + } + logger := NewLoggerV2(writers[infoLog], writers[warningLog], writers[errorLog], cfg) + + msgInf := "someinfo" + msgWarn := "somewarn" + msgErr := "someerr" + if discardTest.printType == print { + logger.Info(msgInf) + logger.Warning(msgWarn) + logger.Error(msgErr) + } else if discardTest.printType == printf { + logger.Infof("%v", msgInf) + logger.Warningf("%v", msgWarn) + logger.Errorf("%v", msgErr) + } else if discardTest.printType == println { + logger.Infoln(msgInf) + logger.Warningln(msgWarn) + logger.Errorln(msgErr) + } + + // verify the test Discard, log type, message, and json arguments were logged as-expected + + checkBufferWasWrittenAsExpected(t, buffers[infoLog], discardTest.discardInf, "info", msgInf, cfg.FormatJSON) + checkBufferWasWrittenAsExpected(t, buffers[warningLog], discardTest.discardWarn, "warning", msgWarn, cfg.FormatJSON) + checkBufferWasWrittenAsExpected(t, buffers[errorLog], discardTest.discardErr, "error", msgErr, cfg.FormatJSON) + }) + } +} + +func TestLoggerV2Fatal(t *testing.T) { + const ( + print = "print" + println = "println" + printf = "printf" + ) + printFuncs := []string{print, println, printf} + + exitCalls := []int{} + + if reflect.ValueOf(exit).Pointer() != reflect.ValueOf(os.Exit).Pointer() { + t.Error("got: exit isn't os.Exit, want exit var to be os.Exit") + } + + exit = func(code int) { + exitCalls = append(exitCalls, code) + } + defer func() { + exit = os.Exit + }() + + for _, printFunc := range printFuncs { + buffers := []*bytes.Buffer{new(bytes.Buffer), new(bytes.Buffer), new(bytes.Buffer)} + logger := NewLoggerV2(buffers[infoLog], buffers[warningLog], buffers[errorLog], LoggerV2Config{}) + switch printFunc { + case print: + logger.Fatal("fatal") + case println: + logger.Fatalln("fatalln") + case printf: + logger.Fatalf("fatalf %d", 42) + default: + t.Errorf("unknown print type '%v'", printFunc) + } + + checkBufferWasWrittenAsExpected(t, buffers[errorLog], false, "fatal", "fatal", false) + if len(exitCalls) == 0 { + t.Error("got: no exit call, want fatal log to call exit") + } + exitCalls = []int{} } - return nil } From 967ba461405304ca9acc47c40eb55a4e87abb514 Mon Sep 17 00:00:00 2001 From: Zach Reyes <39203661+zasweq@users.noreply.github.com> Date: Tue, 26 Nov 2024 13:56:48 -0500 Subject: [PATCH 09/57] balancer/pickfirst: Add pick first metrics (#7839) --- balancer/pickfirst/pickfirst_test.go | 5 +- .../pickfirst/pickfirstleaf/metrics_test.go | 273 ++++++++++++++++++ .../pickfirst/pickfirstleaf/pickfirstleaf.go | 57 +++- .../pickfirstleaf/pickfirstleaf_ext_test.go | 48 ++- .../pickfirstleaf/pickfirstleaf_test.go | 3 +- gcp/observability/go.sum | 1 + internal/balancergroup/balancergroup_test.go | 2 + .../testutils/stats/test_metrics_recorder.go | 17 ++ interop/observability/go.sum | 1 + security/advancedtls/examples/go.sum | 16 + security/advancedtls/go.sum | 16 + stats/opencensus/go.sum | 8 + .../clustermanager/clustermanager_test.go | 2 + 13 files changed, 434 insertions(+), 15 deletions(-) create mode 100644 balancer/pickfirst/pickfirstleaf/metrics_test.go diff --git a/balancer/pickfirst/pickfirst_test.go b/balancer/pickfirst/pickfirst_test.go index 0b360c3b31ed..1da680fb4cf7 100644 --- a/balancer/pickfirst/pickfirst_test.go +++ b/balancer/pickfirst/pickfirst_test.go @@ -29,6 +29,7 @@ import ( "google.golang.org/grpc/connectivity" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/testutils" + "google.golang.org/grpc/internal/testutils/stats" "google.golang.org/grpc/resolver" ) @@ -55,7 +56,7 @@ func (s) TestPickFirst_InitialResolverError(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() cc := testutils.NewBalancerClientConn(t) - bal := balancer.Get(Name).Build(cc, balancer.BuildOptions{}) + bal := balancer.Get(Name).Build(cc, balancer.BuildOptions{MetricsRecorder: &stats.NoopMetricsRecorder{}}) defer bal.Close() bal.ResolverError(errors.New("resolution failed: test error")) @@ -88,7 +89,7 @@ func (s) TestPickFirst_ResolverErrorinTF(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() cc := testutils.NewBalancerClientConn(t) - bal := balancer.Get(Name).Build(cc, balancer.BuildOptions{}) + bal := balancer.Get(Name).Build(cc, balancer.BuildOptions{MetricsRecorder: &stats.NoopMetricsRecorder{}}) defer bal.Close() // After sending a valid update, the LB policy should report CONNECTING. diff --git a/balancer/pickfirst/pickfirstleaf/metrics_test.go b/balancer/pickfirst/pickfirstleaf/metrics_test.go new file mode 100644 index 000000000000..90beca6adc42 --- /dev/null +++ b/balancer/pickfirst/pickfirstleaf/metrics_test.go @@ -0,0 +1,273 @@ +/* + * + * Copyright 2024 gRPC 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 pickfirstleaf_test + +import ( + "context" + "fmt" + "testing" + + "google.golang.org/grpc" + "google.golang.org/grpc/balancer/pickfirst/pickfirstleaf" + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/stubserver" + "google.golang.org/grpc/internal/testutils" + "google.golang.org/grpc/internal/testutils/stats" + testgrpc "google.golang.org/grpc/interop/grpc_testing" + testpb "google.golang.org/grpc/interop/grpc_testing" + "google.golang.org/grpc/resolver" + "google.golang.org/grpc/resolver/manual" + "google.golang.org/grpc/serviceconfig" + "google.golang.org/grpc/stats/opentelemetry" + + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/sdk/metric" + "go.opentelemetry.io/otel/sdk/metric/metricdata" + "go.opentelemetry.io/otel/sdk/metric/metricdata/metricdatatest" +) + +var pfConfig string + +func init() { + pfConfig = fmt.Sprintf(`{ + "loadBalancingConfig": [ + { + %q: { + } + } + ] + }`, pickfirstleaf.Name) +} + +// TestPickFirstMetrics tests pick first metrics. It configures a pick first +// balancer, causes it to connect and then disconnect, and expects the +// subsequent metrics to emit from that. +func (s) TestPickFirstMetrics(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + ss := &stubserver.StubServer{ + EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { + return &testpb.Empty{}, nil + }, + } + ss.StartServer() + defer ss.Stop() + + sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(pfConfig) + + r := manual.NewBuilderWithScheme("whatever") + r.InitialState(resolver.State{ + ServiceConfig: sc, + Addresses: []resolver.Address{{Addr: ss.Address}}}, + ) + + tmr := stats.NewTestMetricsRecorder() + cc, err := grpc.NewClient(r.Scheme()+":///", grpc.WithStatsHandler(tmr), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(r)) + if err != nil { + t.Fatalf("NewClient() failed with error: %v", err) + } + defer cc.Close() + + tsc := testgrpc.NewTestServiceClient(cc) + if _, err := tsc.EmptyCall(ctx, &testpb.Empty{}); err != nil { + t.Fatalf("EmptyCall() failed: %v", err) + } + + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_succeeded"); got != 1 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_succeeded", got, 1) + } + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_failed"); got != 0 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_failed", got, 0) + } + if got, _ := tmr.Metric("grpc.lb.pick_first.disconnections"); got != 0 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.disconnections", got, 0) + } + + ss.Stop() + testutils.AwaitState(ctx, t, cc, connectivity.Idle) + if got, _ := tmr.Metric("grpc.lb.pick_first.disconnections"); got != 1 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.disconnections", got, 1) + } +} + +// TestPickFirstMetricsFailure tests the connection attempts failed metric. It +// configures a channel and scenario that causes a pick first connection attempt +// to fail, and then expects that metric to emit. +func (s) TestPickFirstMetricsFailure(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(pfConfig) + + r := manual.NewBuilderWithScheme("whatever") + r.InitialState(resolver.State{ + ServiceConfig: sc, + Addresses: []resolver.Address{{Addr: "bad address"}}}, + ) + grpcTarget := r.Scheme() + ":///" + tmr := stats.NewTestMetricsRecorder() + cc, err := grpc.NewClient(grpcTarget, grpc.WithStatsHandler(tmr), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(r)) + if err != nil { + t.Fatalf("NewClient() failed with error: %v", err) + } + defer cc.Close() + + tsc := testgrpc.NewTestServiceClient(cc) + if _, err := tsc.EmptyCall(ctx, &testpb.Empty{}); err == nil { + t.Fatalf("EmptyCall() passed when expected to fail") + } + + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_succeeded"); got != 0 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_succeeded", got, 0) + } + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_failed"); got != 1 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_failed", got, 1) + } + if got, _ := tmr.Metric("grpc.lb.pick_first.disconnections"); got != 0 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.disconnections", got, 0) + } +} + +// TestPickFirstMetricsE2E tests the pick first metrics end to end. It +// configures a channel with an OpenTelemetry plugin, induces all 3 pick first +// metrics to emit, and makes sure the correct OpenTelemetry metrics atoms emit. +func (s) TestPickFirstMetricsE2E(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + ss := &stubserver.StubServer{ + EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { + return &testpb.Empty{}, nil + }, + } + ss.StartServer() + defer ss.Stop() + + sc := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(pfConfig) + r := manual.NewBuilderWithScheme("whatever") + r.InitialState(resolver.State{ + ServiceConfig: sc, + Addresses: []resolver.Address{{Addr: "bad address"}}}, + ) // Will trigger connection failed. + + grpcTarget := r.Scheme() + ":///" + reader := metric.NewManualReader() + provider := metric.NewMeterProvider(metric.WithReader(reader)) + mo := opentelemetry.MetricsOptions{ + MeterProvider: provider, + Metrics: opentelemetry.DefaultMetrics().Add("grpc.lb.pick_first.disconnections", "grpc.lb.pick_first.connection_attempts_succeeded", "grpc.lb.pick_first.connection_attempts_failed"), + } + + cc, err := grpc.NewClient(grpcTarget, opentelemetry.DialOption(opentelemetry.Options{MetricsOptions: mo}), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(r)) + if err != nil { + t.Fatalf("NewClient() failed with error: %v", err) + } + defer cc.Close() + + tsc := testgrpc.NewTestServiceClient(cc) + if _, err := tsc.EmptyCall(ctx, &testpb.Empty{}); err == nil { + t.Fatalf("EmptyCall() passed when expected to fail") + } + + r.UpdateState(resolver.State{ + ServiceConfig: sc, + Addresses: []resolver.Address{{Addr: ss.Address}}, + }) // Will trigger successful connection metric. + if _, err := tsc.EmptyCall(ctx, &testpb.Empty{}, grpc.WaitForReady(true)); err != nil { + t.Fatalf("EmptyCall() failed: %v", err) + } + + // Stop the server, that should send signal to disconnect, which will + // eventually emit disconnection metric before ClientConn goes IDLE. + ss.Stop() + testutils.AwaitState(ctx, t, cc, connectivity.Idle) + wantMetrics := []metricdata.Metrics{ + { + Name: "grpc.lb.pick_first.connection_attempts_succeeded", + Description: "EXPERIMENTAL. Number of successful connection attempts.", + Unit: "attempt", + Data: metricdata.Sum[int64]{ + DataPoints: []metricdata.DataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("grpc.target", grpcTarget)), + Value: 1, + }, + }, + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: true, + }, + }, + { + Name: "grpc.lb.pick_first.connection_attempts_failed", + Description: "EXPERIMENTAL. Number of failed connection attempts.", + Unit: "attempt", + Data: metricdata.Sum[int64]{ + DataPoints: []metricdata.DataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("grpc.target", grpcTarget)), + Value: 1, + }, + }, + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: true, + }, + }, + { + Name: "grpc.lb.pick_first.disconnections", + Description: "EXPERIMENTAL. Number of times the selected subchannel becomes disconnected.", + Unit: "disconnection", + Data: metricdata.Sum[int64]{ + DataPoints: []metricdata.DataPoint[int64]{ + { + Attributes: attribute.NewSet(attribute.String("grpc.target", grpcTarget)), + Value: 1, + }, + }, + Temporality: metricdata.CumulativeTemporality, + IsMonotonic: true, + }, + }, + } + + gotMetrics := metricsDataFromReader(ctx, reader) + for _, metric := range wantMetrics { + val, ok := gotMetrics[metric.Name] + if !ok { + t.Fatalf("Metric %v not present in recorded metrics", metric.Name) + } + if !metricdatatest.AssertEqual(t, metric, val, metricdatatest.IgnoreTimestamp(), metricdatatest.IgnoreExemplars()) { + t.Fatalf("Metrics data type not equal for metric: %v", metric.Name) + } + } +} + +func metricsDataFromReader(ctx context.Context, reader *metric.ManualReader) map[string]metricdata.Metrics { + rm := &metricdata.ResourceMetrics{} + reader.Collect(ctx, rm) + gotMetrics := map[string]metricdata.Metrics{} + for _, sm := range rm.ScopeMetrics { + for _, m := range sm.Metrics { + gotMetrics[m.Name] = m + } + } + return gotMetrics +} diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index aaec87497fd4..1ebf7cea5e94 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -36,6 +36,7 @@ import ( "google.golang.org/grpc/balancer" "google.golang.org/grpc/balancer/pickfirst/internal" "google.golang.org/grpc/connectivity" + expstats "google.golang.org/grpc/experimental/stats" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/internal/envconfig" internalgrpclog "google.golang.org/grpc/internal/grpclog" @@ -57,7 +58,28 @@ var ( // Name is the name of the pick_first_leaf balancer. // It is changed to "pick_first" in init() if this balancer is to be // registered as the default pickfirst. - Name = "pick_first_leaf" + Name = "pick_first_leaf" + disconnectionsMetric = expstats.RegisterInt64Count(expstats.MetricDescriptor{ + Name: "grpc.lb.pick_first.disconnections", + Description: "EXPERIMENTAL. Number of times the selected subchannel becomes disconnected.", + Unit: "disconnection", + Labels: []string{"grpc.target"}, + Default: false, + }) + connectionAttemptsSucceededMetric = expstats.RegisterInt64Count(expstats.MetricDescriptor{ + Name: "grpc.lb.pick_first.connection_attempts_succeeded", + Description: "EXPERIMENTAL. Number of successful connection attempts.", + Unit: "attempt", + Labels: []string{"grpc.target"}, + Default: false, + }) + connectionAttemptsFailedMetric = expstats.RegisterInt64Count(expstats.MetricDescriptor{ + Name: "grpc.lb.pick_first.connection_attempts_failed", + Description: "EXPERIMENTAL. Number of failed connection attempts.", + Unit: "attempt", + Labels: []string{"grpc.target"}, + Default: false, + }) ) const ( @@ -80,9 +102,12 @@ const ( type pickfirstBuilder struct{} -func (pickfirstBuilder) Build(cc balancer.ClientConn, _ balancer.BuildOptions) balancer.Balancer { +func (pickfirstBuilder) Build(cc balancer.ClientConn, bo balancer.BuildOptions) balancer.Balancer { b := &pickfirstBalancer{ - cc: cc, + cc: cc, + target: bo.Target.String(), + metricsRecorder: bo.MetricsRecorder, // ClientConn will always create a Metrics Recorder. + addressList: addressList{}, subConns: resolver.NewAddressMap(), state: connectivity.Connecting, @@ -147,8 +172,10 @@ func (b *pickfirstBalancer) newSCData(addr resolver.Address) (*scData, error) { type pickfirstBalancer struct { // The following fields are initialized at build time and read-only after // that and therefore do not need to be guarded by a mutex. - logger *internalgrpclog.PrefixLogger - cc balancer.ClientConn + logger *internalgrpclog.PrefixLogger + cc balancer.ClientConn + target string + metricsRecorder expstats.MetricsRecorder // guaranteed to be non nil // The mutex is used to ensure synchronization of updates triggered // from the idle picker and the already serialized resolver, @@ -532,10 +559,6 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub b.mu.Lock() defer b.mu.Unlock() oldState := sd.state - // Record a connection attempt when exiting CONNECTING. - if newState.ConnectivityState == connectivity.TransientFailure { - sd.connectionFailedInFirstPass = true - } sd.state = newState.ConnectivityState // Previously relevant SubConns can still callback with state updates. // To prevent pickers from returning these obsolete SubConns, this logic @@ -548,7 +571,14 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub return } + // Record a connection attempt when exiting CONNECTING. + if newState.ConnectivityState == connectivity.TransientFailure { + sd.connectionFailedInFirstPass = true + connectionAttemptsFailedMetric.Record(b.metricsRecorder, 1, b.target) + } + if newState.ConnectivityState == connectivity.Ready { + connectionAttemptsSucceededMetric.Record(b.metricsRecorder, 1, b.target) b.shutdownRemainingLocked(sd) if !b.addressList.seekTo(sd.addr) { // This should not fail as we should have only one SubConn after @@ -575,6 +605,15 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub // the first address when the picker is used. b.shutdownRemainingLocked(sd) b.state = connectivity.Idle + // READY SubConn interspliced in between CONNECTING and IDLE, need to + // account for that. + if oldState == connectivity.Connecting { + // A known issue (https://github.com/grpc/grpc-go/issues/7862) + // causes a race that prevents the READY state change notification. + // This works around it. + connectionAttemptsSucceededMetric.Record(b.metricsRecorder, 1, b.target) + } + disconnectionsMetric.Record(b.metricsRecorder, 1, b.target) b.addressList.reset() b.cc.UpdateState(balancer.State{ ConnectivityState: connectivity.Idle, diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index bf957f98b119..007157249689 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -39,6 +39,7 @@ import ( "google.golang.org/grpc/internal/stubserver" "google.golang.org/grpc/internal/testutils" "google.golang.org/grpc/internal/testutils/pickfirst" + "google.golang.org/grpc/internal/testutils/stats" "google.golang.org/grpc/resolver" "google.golang.org/grpc/resolver/manual" "google.golang.org/grpc/status" @@ -863,10 +864,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_AfterEndOfList(t *testing.T) { triggerTimer, timeAfter := mockTimer() pfinternal.TimeAfterFunc = timeAfter + tmr := stats.NewTestMetricsRecorder() dialer := testutils.NewBlockingDialer() opts := []grpc.DialOption{ grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), grpc.WithContextDialer(dialer.DialContext), + grpc.WithStatsHandler(tmr), } cc, rb, bm := setupPickFirstLeaf(t, 3, opts...) addrs := bm.resolverAddrs() @@ -906,6 +909,7 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_AfterEndOfList(t *testing.T) { // First SubConn Fails. holds[0].Fail(fmt.Errorf("test error")) + tmr.WaitForInt64CountIncr(ctx, 1) // No TF should be reported until the first pass is complete. shortCtx, shortCancel := context.WithTimeout(ctx, defaultTestShortTimeout) @@ -916,11 +920,24 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_AfterEndOfList(t *testing.T) { shortCtx, shortCancel = context.WithTimeout(ctx, defaultTestShortTimeout) defer shortCancel() holds[2].Fail(fmt.Errorf("test error")) + tmr.WaitForInt64CountIncr(ctx, 1) testutils.AwaitNotState(shortCtx, t, cc, connectivity.TransientFailure) // Last SubConn fails, this should result in a TF update. holds[1].Fail(fmt.Errorf("test error")) + tmr.WaitForInt64CountIncr(ctx, 1) testutils.AwaitState(ctx, t, cc, connectivity.TransientFailure) + + // Only connection attempt fails in this test. + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_succeeded"); got != 0 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_succeeded", got, 0) + } + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_failed"); got != 1 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_failed", got, 1) + } + if got, _ := tmr.Metric("grpc.lb.pick_first.disconnections"); got != 0 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.disconnections", got, 0) + } } // Test verifies that pickfirst attempts to connect to the second backend once @@ -936,10 +953,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { triggerTimer, timeAfter := mockTimer() pfinternal.TimeAfterFunc = timeAfter + tmr := stats.NewTestMetricsRecorder() dialer := testutils.NewBlockingDialer() opts := []grpc.DialOption{ grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), grpc.WithContextDialer(dialer.DialContext), + grpc.WithStatsHandler(tmr), } cc, rb, bm := setupPickFirstLeaf(t, 2, opts...) addrs := bm.resolverAddrs() @@ -968,6 +987,17 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TriggerConnectionDelay(t *testing.T) { // that the channel becomes READY. holds[1].Resume() testutils.AwaitState(ctx, t, cc, connectivity.Ready) + + // Only connection attempt successes in this test. + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_succeeded"); got != 1 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_succeeded", got, 1) + } + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_failed"); got != 0 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_failed", got, 0) + } + if got, _ := tmr.Metric("grpc.lb.pick_first.disconnections"); got != 0 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.disconnections", got, 0) + } } // Test tests the pickfirst balancer by causing a SubConn to fail and then @@ -983,10 +1013,12 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_ThenTimerFires(t *testing.T) { triggerTimer, timeAfter := mockTimer() pfinternal.TimeAfterFunc = timeAfter + tmr := stats.NewTestMetricsRecorder() dialer := testutils.NewBlockingDialer() opts := []grpc.DialOption{ grpc.WithDefaultServiceConfig(fmt.Sprintf(`{"loadBalancingConfig": [{"%s":{}}]}`, pickfirstleaf.Name)), grpc.WithContextDialer(dialer.DialContext), + grpc.WithStatsHandler(tmr), } cc, rb, bm := setupPickFirstLeaf(t, 3, opts...) addrs := bm.resolverAddrs() @@ -1014,6 +1046,9 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_ThenTimerFires(t *testing.T) { if holds[1].Wait(ctx) != true { t.Fatalf("Timeout waiting for server %d with address %q to be contacted", 1, addrs[1]) } + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_failed"); got != 1 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_failed", got, 1) + } if holds[2].IsStarted() != false { t.Fatalf("Server %d with address %q contacted unexpectedly", 2, addrs[2]) } @@ -1030,13 +1065,20 @@ func (s) TestPickFirstLeaf_HappyEyeballs_TF_ThenTimerFires(t *testing.T) { // that the channel becomes READY. holds[1].Resume() testutils.AwaitState(ctx, t, cc, connectivity.Ready) + + if got, _ := tmr.Metric("grpc.lb.pick_first.connection_attempts_succeeded"); got != 1 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.connection_attempts_succeeded", got, 1) + } + if got, _ := tmr.Metric("grpc.lb.pick_first.disconnections"); got != 0 { + t.Errorf("Unexpected data for metric %v, got: %v, want: %v", "grpc.lb.pick_first.disconnections", got, 0) + } } func (s) TestPickFirstLeaf_InterleavingIPV4Preffered(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() cc := testutils.NewBalancerClientConn(t) - bal := balancer.Get(pickfirstleaf.Name).Build(cc, balancer.BuildOptions{}) + bal := balancer.Get(pickfirstleaf.Name).Build(cc, balancer.BuildOptions{MetricsRecorder: &stats.NoopMetricsRecorder{}}) defer bal.Close() ccState := balancer.ClientConnState{ ResolverState: resolver.State{ @@ -1082,7 +1124,7 @@ func (s) TestPickFirstLeaf_InterleavingIPv6Preffered(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() cc := testutils.NewBalancerClientConn(t) - bal := balancer.Get(pickfirstleaf.Name).Build(cc, balancer.BuildOptions{}) + bal := balancer.Get(pickfirstleaf.Name).Build(cc, balancer.BuildOptions{MetricsRecorder: &stats.NoopMetricsRecorder{}}) defer bal.Close() ccState := balancer.ClientConnState{ ResolverState: resolver.State{ @@ -1126,7 +1168,7 @@ func (s) TestPickFirstLeaf_InterleavingUnknownPreffered(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() cc := testutils.NewBalancerClientConn(t) - bal := balancer.Get(pickfirstleaf.Name).Build(cc, balancer.BuildOptions{}) + bal := balancer.Get(pickfirstleaf.Name).Build(cc, balancer.BuildOptions{MetricsRecorder: &stats.NoopMetricsRecorder{}}) defer bal.Close() ccState := balancer.ClientConnState{ ResolverState: resolver.State{ diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go index 71984a238cd5..f269a71a7a97 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_test.go @@ -29,6 +29,7 @@ import ( "google.golang.org/grpc/connectivity" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/testutils" + "google.golang.org/grpc/internal/testutils/stats" "google.golang.org/grpc/resolver" ) @@ -195,7 +196,7 @@ func (s) TestPickFirstLeaf_TFPickerUpdate(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() cc := testutils.NewBalancerClientConn(t) - bal := pickfirstBuilder{}.Build(cc, balancer.BuildOptions{}) + bal := pickfirstBuilder{}.Build(cc, balancer.BuildOptions{MetricsRecorder: &stats.NoopMetricsRecorder{}}) defer bal.Close() ccState := balancer.ClientConnState{ ResolverState: resolver.State{ diff --git a/gcp/observability/go.sum b/gcp/observability/go.sum index 30e984fb4343..472ac41d57ad 100644 --- a/gcp/observability/go.sum +++ b/gcp/observability/go.sum @@ -1107,6 +1107,7 @@ go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozR go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= +go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= diff --git a/internal/balancergroup/balancergroup_test.go b/internal/balancergroup/balancergroup_test.go index c154c029d8f2..e49e8135a1b7 100644 --- a/internal/balancergroup/balancergroup_test.go +++ b/internal/balancergroup/balancergroup_test.go @@ -33,6 +33,7 @@ import ( "google.golang.org/grpc/internal/channelz" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/testutils" + "google.golang.org/grpc/internal/testutils/stats" "google.golang.org/grpc/resolver" ) @@ -603,6 +604,7 @@ func (s) TestBalancerGracefulSwitch(t *testing.T) { childPolicyName := t.Name() stub.Register(childPolicyName, stub.BalancerFuncs{ Init: func(bd *stub.BalancerData) { + bd.BuildOptions.MetricsRecorder = &stats.NoopMetricsRecorder{} bd.Data = balancer.Get(pickfirst.Name).Build(bd.ClientConn, bd.BuildOptions) }, Close: func(bd *stub.BalancerData) { diff --git a/internal/testutils/stats/test_metrics_recorder.go b/internal/testutils/stats/test_metrics_recorder.go index 72a20c1cbf44..e13013e38d53 100644 --- a/internal/testutils/stats/test_metrics_recorder.go +++ b/internal/testutils/stats/test_metrics_recorder.go @@ -63,6 +63,8 @@ func NewTestMetricsRecorder() *TestMetricsRecorder { // Metric returns the most recent data for a metric, and whether this recorder // has received data for a metric. func (r *TestMetricsRecorder) Metric(name string) (float64, bool) { + r.mu.Lock() + defer r.mu.Unlock() data, ok := r.data[estats.Metric(name)] return data, ok } @@ -102,6 +104,21 @@ func (r *TestMetricsRecorder) WaitForInt64Count(ctx context.Context, metricsData return nil } +// WaitForInt64CountIncr waits for an int64 count metric to be recorded and +// verifies that the recorded metrics data incr matches the expected incr. +// Returns an error if failed to wait or received wrong data. +func (r *TestMetricsRecorder) WaitForInt64CountIncr(ctx context.Context, incrWant int64) error { + got, err := r.intCountCh.Receive(ctx) + if err != nil { + return fmt.Errorf("timeout waiting for int64Count") + } + metricsDataGot := got.(MetricsData) + if diff := cmp.Diff(metricsDataGot.IntIncr, incrWant); diff != "" { + return fmt.Errorf("int64count metricsData received unexpected value (-got, +want): %v", diff) + } + return nil +} + // RecordInt64Count sends the metrics data to the intCountCh channel and updates // the internal data map with the recorded value. func (r *TestMetricsRecorder) RecordInt64Count(handle *estats.Int64CountHandle, incr int64, labels ...string) { diff --git a/interop/observability/go.sum b/interop/observability/go.sum index a749b30fb223..4cdbd27b0fc9 100644 --- a/interop/observability/go.sum +++ b/interop/observability/go.sum @@ -1109,6 +1109,7 @@ go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozR go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= +go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= diff --git a/security/advancedtls/examples/go.sum b/security/advancedtls/examples/go.sum index 9102af782ca0..2192e85919d7 100644 --- a/security/advancedtls/examples/go.sum +++ b/security/advancedtls/examples/go.sum @@ -1,7 +1,23 @@ +github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= +github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= +go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= +go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= +go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= +go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= +go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= +go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= +go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= +go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= +go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= diff --git a/security/advancedtls/go.sum b/security/advancedtls/go.sum index 9102af782ca0..2192e85919d7 100644 --- a/security/advancedtls/go.sum +++ b/security/advancedtls/go.sum @@ -1,7 +1,23 @@ +github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= +github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= +github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= +github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= +go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= +go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= +go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= +go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= +go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= +go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= +go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= +go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= +go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= diff --git a/stats/opencensus/go.sum b/stats/opencensus/go.sum index 447a12f3eb15..2e88e8bf1877 100644 --- a/stats/opencensus/go.sum +++ b/stats/opencensus/go.sum @@ -821,7 +821,9 @@ github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2 github.com/go-latex/latex v0.0.0-20210118124228-b3d85cf34e07/go.mod h1:CO1AlKB2CSIqUrmQPqA0gdRIlnLEY0gK5JGjh37zN5U= github.com/go-latex/latex v0.0.0-20210823091927-c0d11ff05a81/go.mod h1:SX0U8uGpxhq9o2S/CELCSUxEWWAuoCUcVCQWv7G2OCk= github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= +github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= +github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-pdf/fpdf v0.5.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/go-pdf/fpdf v0.6.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= @@ -914,6 +916,7 @@ github.com/google/s2a-go v0.1.3/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkj github.com/google/s2a-go v0.1.4/go.mod h1:Ej+mSEMGRnqRzjc7VtF+jdBwYG5fuJfiZ8ELkjEwM0A= github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= +github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.0.0-20220520183353-fd19c99a87aa/go.mod h1:17drOmN3MwGY7t0e+Ei9b45FFGA3fBs3x36SsCg1hq8= github.com/googleapis/enterprise-certificate-proxy v0.1.0/go.mod h1:17drOmN3MwGY7t0e+Ei9b45FFGA3fBs3x36SsCg1hq8= @@ -1037,10 +1040,15 @@ go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= go.opentelemetry.io/contrib/detectors/gcp v1.31.0/go.mod h1:tzQL6E1l+iV44YFTkcAeNQqzXUiekSYP9jjJjXwEd00= +go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= +go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= +go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= +go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= +go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= diff --git a/xds/internal/balancer/clustermanager/clustermanager_test.go b/xds/internal/balancer/clustermanager/clustermanager_test.go index b606cb9e5e34..6ef8738dfcf4 100644 --- a/xds/internal/balancer/clustermanager/clustermanager_test.go +++ b/xds/internal/balancer/clustermanager/clustermanager_test.go @@ -34,6 +34,7 @@ import ( "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/hierarchy" "google.golang.org/grpc/internal/testutils" + "google.golang.org/grpc/internal/testutils/stats" "google.golang.org/grpc/resolver" "google.golang.org/grpc/status" ) @@ -643,6 +644,7 @@ func TestClusterGracefulSwitch(t *testing.T) { childPolicyName := t.Name() stub.Register(childPolicyName, stub.BalancerFuncs{ Init: func(bd *stub.BalancerData) { + bd.BuildOptions.MetricsRecorder = &stats.NoopMetricsRecorder{} bd.Data = balancer.Get(pickfirst.Name).Build(bd.ClientConn, bd.BuildOptions) }, Close: func(bd *stub.BalancerData) { From 4c07bca27377feb808912b844b3fa95ad10f946b Mon Sep 17 00:00:00 2001 From: Ismail Gjevori Date: Wed, 27 Nov 2024 00:08:44 +0100 Subject: [PATCH 10/57] stream: add jitter to retry backoff in accordance with gRFC A6 (#7869) --- service_config.go | 5 +++-- stream.go | 11 +++++------ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/service_config.go b/service_config.go index 2671c5ef69f0..7e83027d1994 100644 --- a/service_config.go +++ b/service_config.go @@ -168,6 +168,7 @@ func init() { return parseServiceConfig(js, defaultMaxCallAttempts) } } + func parseServiceConfig(js string, maxAttempts int) *serviceconfig.ParseResult { if len(js) == 0 { return &serviceconfig.ParseResult{Err: fmt.Errorf("no JSON service config provided")} @@ -297,7 +298,7 @@ func convertRetryPolicy(jrp *jsonRetryPolicy, maxAttempts int) (p *internalservi return rp, nil } -func min(a, b *int) *int { +func minPointers(a, b *int) *int { if *a < *b { return a } @@ -309,7 +310,7 @@ func getMaxSize(mcMax, doptMax *int, defaultVal int) *int { return &defaultVal } if mcMax != nil && doptMax != nil { - return min(mcMax, doptMax) + return minPointers(mcMax, doptMax) } if mcMax != nil { return mcMax diff --git a/stream.go b/stream.go index 6d10d0ac8713..17e2267b3320 100644 --- a/stream.go +++ b/stream.go @@ -218,7 +218,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth var mc serviceconfig.MethodConfig var onCommit func() - var newStream = func(ctx context.Context, done func()) (iresolver.ClientStream, error) { + newStream := func(ctx context.Context, done func()) (iresolver.ClientStream, error) { return newClientStreamWithParams(ctx, desc, cc, method, mc, onCommit, done, opts...) } @@ -708,11 +708,10 @@ func (a *csAttempt) shouldRetry(err error) (bool, error) { cs.numRetriesSincePushback = 0 } else { fact := math.Pow(rp.BackoffMultiplier, float64(cs.numRetriesSincePushback)) - cur := float64(rp.InitialBackoff) * fact - if max := float64(rp.MaxBackoff); cur > max { - cur = max - } - dur = time.Duration(rand.Int64N(int64(cur))) + cur := min(float64(rp.InitialBackoff)*fact, float64(rp.MaxBackoff)) + // Apply jitter by multiplying with a random factor between 0.8 and 1.2 + cur *= 0.8 + 0.4*rand.Float64() + dur = time.Duration(int64(cur)) cs.numRetriesSincePushback++ } From 3c0586a427dd1fb8ba66f8c4dbd038273a406bbe Mon Sep 17 00:00:00 2001 From: Zach Reyes <39203661+zasweq@users.noreply.github.com> Date: Mon, 2 Dec 2024 12:19:40 -0500 Subject: [PATCH 11/57] stats/opentelemetry: Cleanup OpenTelemetry API's before stabilization (#7874) Co-authored-by: Doug Fawley --- experimental/stats/metricregistry.go | 27 ++++--- experimental/stats/metrics.go | 75 ----------------- .../testutils/stats/test_metrics_recorder.go | 8 +- stats/metrics.go | 81 +++++++++++++++++++ stats/opentelemetry/client_metrics.go | 29 +++---- stats/opentelemetry/example_test.go | 8 +- stats/opentelemetry/opentelemetry.go | 48 ++++++----- stats/opentelemetry/server_metrics.go | 22 ++--- 8 files changed, 156 insertions(+), 142 deletions(-) create mode 100644 stats/metrics.go diff --git a/experimental/stats/metricregistry.go b/experimental/stats/metricregistry.go index 1d827dd5d9d4..ad75313a18e1 100644 --- a/experimental/stats/metricregistry.go +++ b/experimental/stats/metricregistry.go @@ -23,6 +23,7 @@ import ( "google.golang.org/grpc/grpclog" "google.golang.org/grpc/internal" + "google.golang.org/grpc/stats" ) func init() { @@ -34,7 +35,7 @@ var logger = grpclog.Component("metrics-registry") // DefaultMetrics are the default metrics registered through global metrics // registry. This is written to at initialization time only, and is read only // after initialization. -var DefaultMetrics = NewMetrics() +var DefaultMetrics = stats.NewMetricSet() // MetricDescriptor is the data for a registered metric. type MetricDescriptor struct { @@ -42,7 +43,7 @@ type MetricDescriptor struct { // (including any per call metrics). See // https://github.com/grpc/proposal/blob/master/A79-non-per-call-metrics-architecture.md#metric-instrument-naming-conventions // for metric naming conventions. - Name Metric + Name string // The description of this metric. Description string // The unit (e.g. entries, seconds) of this metric. @@ -154,27 +155,27 @@ func (h *Int64GaugeHandle) Record(recorder MetricsRecorder, incr int64, labels . } // registeredMetrics are the registered metric descriptor names. -var registeredMetrics = make(map[Metric]bool) +var registeredMetrics = make(map[string]bool) // metricsRegistry contains all of the registered metrics. // // This is written to only at init time, and read only after that. -var metricsRegistry = make(map[Metric]*MetricDescriptor) +var metricsRegistry = make(map[string]*MetricDescriptor) // DescriptorForMetric returns the MetricDescriptor from the global registry. // // Returns nil if MetricDescriptor not present. -func DescriptorForMetric(metric Metric) *MetricDescriptor { - return metricsRegistry[metric] +func DescriptorForMetric(metricName string) *MetricDescriptor { + return metricsRegistry[metricName] } -func registerMetric(name Metric, def bool) { - if registeredMetrics[name] { - logger.Fatalf("metric %v already registered", name) +func registerMetric(metricName string, def bool) { + if registeredMetrics[metricName] { + logger.Fatalf("metric %v already registered", metricName) } - registeredMetrics[name] = true + registeredMetrics[metricName] = true if def { - DefaultMetrics = DefaultMetrics.Add(name) + DefaultMetrics = DefaultMetrics.Add(metricName) } } @@ -256,8 +257,8 @@ func snapshotMetricsRegistryForTesting() func() { oldRegisteredMetrics := registeredMetrics oldMetricsRegistry := metricsRegistry - registeredMetrics = make(map[Metric]bool) - metricsRegistry = make(map[Metric]*MetricDescriptor) + registeredMetrics = make(map[string]bool) + metricsRegistry = make(map[string]*MetricDescriptor) maps.Copy(registeredMetrics, registeredMetrics) maps.Copy(metricsRegistry, metricsRegistry) diff --git a/experimental/stats/metrics.go b/experimental/stats/metrics.go index 3221f7a633a3..bf9e7f987b04 100644 --- a/experimental/stats/metrics.go +++ b/experimental/stats/metrics.go @@ -19,8 +19,6 @@ // Package stats contains experimental metrics/stats API's. package stats -import "maps" - // MetricsRecorder records on metrics derived from metric registry. type MetricsRecorder interface { // RecordInt64Count records the measurement alongside labels on the int @@ -39,76 +37,3 @@ type MetricsRecorder interface { // gauge associated with the provided handle. RecordInt64Gauge(handle *Int64GaugeHandle, incr int64, labels ...string) } - -// Metric is an identifier for a metric. -type Metric string - -// Metrics is a set of metrics to record. Once created, Metrics is immutable, -// however Add and Remove can make copies with specific metrics added or -// removed, respectively. -// -// Do not construct directly; use NewMetrics instead. -type Metrics struct { - // metrics are the set of metrics to initialize. - metrics map[Metric]bool -} - -// NewMetrics returns a Metrics containing Metrics. -func NewMetrics(metrics ...Metric) *Metrics { - newMetrics := make(map[Metric]bool) - for _, metric := range metrics { - newMetrics[metric] = true - } - return &Metrics{ - metrics: newMetrics, - } -} - -// Metrics returns the metrics set. The returned map is read-only and must not -// be modified. -func (m *Metrics) Metrics() map[Metric]bool { - return m.metrics -} - -// Add adds the metrics to the metrics set and returns a new copy with the -// additional metrics. -func (m *Metrics) Add(metrics ...Metric) *Metrics { - newMetrics := make(map[Metric]bool) - for metric := range m.metrics { - newMetrics[metric] = true - } - - for _, metric := range metrics { - newMetrics[metric] = true - } - return &Metrics{ - metrics: newMetrics, - } -} - -// Join joins the metrics passed in with the metrics set, and returns a new copy -// with the merged metrics. -func (m *Metrics) Join(metrics *Metrics) *Metrics { - newMetrics := make(map[Metric]bool) - maps.Copy(newMetrics, m.metrics) - maps.Copy(newMetrics, metrics.metrics) - return &Metrics{ - metrics: newMetrics, - } -} - -// Remove removes the metrics from the metrics set and returns a new copy with -// the metrics removed. -func (m *Metrics) Remove(metrics ...Metric) *Metrics { - newMetrics := make(map[Metric]bool) - for metric := range m.metrics { - newMetrics[metric] = true - } - - for _, metric := range metrics { - delete(newMetrics, metric) - } - return &Metrics{ - metrics: newMetrics, - } -} diff --git a/internal/testutils/stats/test_metrics_recorder.go b/internal/testutils/stats/test_metrics_recorder.go index e13013e38d53..e1a03b8d8008 100644 --- a/internal/testutils/stats/test_metrics_recorder.go +++ b/internal/testutils/stats/test_metrics_recorder.go @@ -44,7 +44,7 @@ type TestMetricsRecorder struct { // mu protects data. mu sync.Mutex // data is the most recent update for each metric name. - data map[estats.Metric]float64 + data map[string]float64 } // NewTestMetricsRecorder returns a new TestMetricsRecorder. @@ -56,7 +56,7 @@ func NewTestMetricsRecorder() *TestMetricsRecorder { floatHistoCh: testutils.NewChannelWithSize(10), intGaugeCh: testutils.NewChannelWithSize(10), - data: make(map[estats.Metric]float64), + data: make(map[string]float64), } } @@ -65,7 +65,7 @@ func NewTestMetricsRecorder() *TestMetricsRecorder { func (r *TestMetricsRecorder) Metric(name string) (float64, bool) { r.mu.Lock() defer r.mu.Unlock() - data, ok := r.data[estats.Metric(name)] + data, ok := r.data[name] return data, ok } @@ -73,7 +73,7 @@ func (r *TestMetricsRecorder) Metric(name string) (float64, bool) { func (r *TestMetricsRecorder) ClearMetrics() { r.mu.Lock() defer r.mu.Unlock() - r.data = make(map[estats.Metric]float64) + r.data = make(map[string]float64) } // MetricsData represents data associated with a metric. diff --git a/stats/metrics.go b/stats/metrics.go new file mode 100644 index 000000000000..641c8e9794a5 --- /dev/null +++ b/stats/metrics.go @@ -0,0 +1,81 @@ +/* + * Copyright 2024 gRPC 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 stats + +import "maps" + +// MetricSet is a set of metrics to record. Once created, MetricSet is immutable, +// however Add and Remove can make copies with specific metrics added or +// removed, respectively. +// +// Do not construct directly; use NewMetricSet instead. +type MetricSet struct { + // metrics are the set of metrics to initialize. + metrics map[string]bool +} + +// NewMetricSet returns a MetricSet containing metricNames. +func NewMetricSet(metricNames ...string) *MetricSet { + newMetrics := make(map[string]bool) + for _, metric := range metricNames { + newMetrics[metric] = true + } + return &MetricSet{metrics: newMetrics} +} + +// Metrics returns the metrics set. The returned map is read-only and must not +// be modified. +func (m *MetricSet) Metrics() map[string]bool { + return m.metrics +} + +// Add adds the metricNames to the metrics set and returns a new copy with the +// additional metrics. +func (m *MetricSet) Add(metricNames ...string) *MetricSet { + newMetrics := make(map[string]bool) + for metric := range m.metrics { + newMetrics[metric] = true + } + + for _, metric := range metricNames { + newMetrics[metric] = true + } + return &MetricSet{metrics: newMetrics} +} + +// Join joins the metrics passed in with the metrics set, and returns a new copy +// with the merged metrics. +func (m *MetricSet) Join(metrics *MetricSet) *MetricSet { + newMetrics := make(map[string]bool) + maps.Copy(newMetrics, m.metrics) + maps.Copy(newMetrics, metrics.metrics) + return &MetricSet{metrics: newMetrics} +} + +// Remove removes the metricNames from the metrics set and returns a new copy +// with the metrics removed. +func (m *MetricSet) Remove(metricNames ...string) *MetricSet { + newMetrics := make(map[string]bool) + for metric := range m.metrics { + newMetrics[metric] = true + } + + for _, metric := range metricNames { + delete(newMetrics, metric) + } + return &MetricSet{metrics: newMetrics} +} diff --git a/stats/opentelemetry/client_metrics.go b/stats/opentelemetry/client_metrics.go index 4af7f933c8ba..265791e5a261 100644 --- a/stats/opentelemetry/client_metrics.go +++ b/stats/opentelemetry/client_metrics.go @@ -260,18 +260,19 @@ func (h *clientStatsHandler) processRPCEnd(ctx context.Context, ai *attemptInfo, } const ( - // ClientAttemptStarted is the number of client call attempts started. - ClientAttemptStarted estats.Metric = "grpc.client.attempt.started" - // ClientAttemptDuration is the end-to-end time taken to complete a client - // call attempt. - ClientAttemptDuration estats.Metric = "grpc.client.attempt.duration" - // ClientAttemptSentCompressedTotalMessageSize is the compressed message - // bytes sent per client call attempt. - ClientAttemptSentCompressedTotalMessageSize estats.Metric = "grpc.client.attempt.sent_total_compressed_message_size" - // ClientAttemptRcvdCompressedTotalMessageSize is the compressed message - // bytes received per call attempt. - ClientAttemptRcvdCompressedTotalMessageSize estats.Metric = "grpc.client.attempt.rcvd_total_compressed_message_size" - // ClientCallDuration is the time taken by gRPC to complete an RPC from - // application's perspective. - ClientCallDuration estats.Metric = "grpc.client.call.duration" + // ClientAttemptStartedMetricName is the number of client call attempts + // started. + ClientAttemptStartedMetricName string = "grpc.client.attempt.started" + // ClientAttemptDurationMetricName is the end-to-end time taken to complete + // a client call attempt. + ClientAttemptDurationMetricName string = "grpc.client.attempt.duration" + // ClientAttemptSentCompressedTotalMessageSizeMetricName is the compressed + // message bytes sent per client call attempt. + ClientAttemptSentCompressedTotalMessageSizeMetricName string = "grpc.client.attempt.sent_total_compressed_message_size" + // ClientAttemptRcvdCompressedTotalMessageSizeMetricName is the compressed + // message bytes received per call attempt. + ClientAttemptRcvdCompressedTotalMessageSizeMetricName string = "grpc.client.attempt.rcvd_total_compressed_message_size" + // ClientCallDurationMetricName is the time taken by gRPC to complete an RPC + // from application's perspective. + ClientCallDurationMetricName string = "grpc.client.call.duration" ) diff --git a/stats/opentelemetry/example_test.go b/stats/opentelemetry/example_test.go index 66676f3c035b..e87e4ebb65b9 100644 --- a/stats/opentelemetry/example_test.go +++ b/stats/opentelemetry/example_test.go @@ -19,7 +19,7 @@ package opentelemetry_test import ( "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" - estats "google.golang.org/grpc/experimental/stats" + "google.golang.org/grpc/stats" "google.golang.org/grpc/stats/opentelemetry" "go.opentelemetry.io/otel/sdk/metric" @@ -88,7 +88,7 @@ func ExampleMetrics_excludeSome() { // To exclude specific metrics, initialize Options as follows: opts := opentelemetry.Options{ MetricsOptions: opentelemetry.MetricsOptions{ - Metrics: opentelemetry.DefaultMetrics().Remove(opentelemetry.ClientAttemptDuration, opentelemetry.ClientAttemptRcvdCompressedTotalMessageSize), + Metrics: opentelemetry.DefaultMetrics().Remove(opentelemetry.ClientAttemptDurationMetricName, opentelemetry.ClientAttemptRcvdCompressedTotalMessageSizeMetricName), }, } do := opentelemetry.DialOption(opts) @@ -103,7 +103,7 @@ func ExampleMetrics_disableAll() { // To disable all metrics, initialize Options as follows: opts := opentelemetry.Options{ MetricsOptions: opentelemetry.MetricsOptions{ - Metrics: estats.NewMetrics(), // Distinct to nil, which creates default metrics. This empty set creates no metrics. + Metrics: stats.NewMetricSet(), // Distinct to nil, which creates default metrics. This empty set creates no metrics. }, } do := opentelemetry.DialOption(opts) @@ -118,7 +118,7 @@ func ExampleMetrics_enableSome() { // To only create specific metrics, initialize Options as follows: opts := opentelemetry.Options{ MetricsOptions: opentelemetry.MetricsOptions{ - Metrics: estats.NewMetrics(opentelemetry.ClientAttemptDuration, opentelemetry.ClientAttemptRcvdCompressedTotalMessageSize), // only create these metrics + Metrics: stats.NewMetricSet(opentelemetry.ClientAttemptDurationMetricName, opentelemetry.ClientAttemptRcvdCompressedTotalMessageSizeMetricName), // only create these metrics }, } do := opentelemetry.DialOption(opts) diff --git a/stats/opentelemetry/opentelemetry.go b/stats/opentelemetry/opentelemetry.go index cc5ad387fb4c..dcc424775f14 100644 --- a/stats/opentelemetry/opentelemetry.go +++ b/stats/opentelemetry/opentelemetry.go @@ -16,6 +16,10 @@ // Package opentelemetry implements opentelemetry instrumentation code for // gRPC-Go clients and servers. +// +// For details on configuring opentelemetry and various instruments that this +// package creates, see +// [gRPC OpenTelemetry Metrics](https://grpc.io/docs/guides/opentelemetry-metrics/). package opentelemetry import ( @@ -28,6 +32,7 @@ import ( estats "google.golang.org/grpc/experimental/stats" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/internal" + "google.golang.org/grpc/stats" otelinternal "google.golang.org/grpc/stats/opentelemetry/internal" otelattribute "go.opentelemetry.io/otel/attribute" @@ -57,27 +62,28 @@ type Options struct { type MetricsOptions struct { // MeterProvider is the MeterProvider instance that will be used to create // instruments. To enable metrics collection, set a meter provider. If - // unset, no metrics will be recorded. Any implementation knobs (i.e. views, - // bounds) set in the MeterProvider take precedence over the API calls from - // this interface. (i.e. it will create default views for unset views). + // unset, no metrics will be recorded. MeterProvider otelmetric.MeterProvider // Metrics are the metrics to instrument. Will create instrument and record telemetry // for corresponding metric supported by the client and server // instrumentation components if applicable. If not set, the default metrics // will be recorded. - Metrics *estats.Metrics - - // MethodAttributeFilter is to record the method name of RPCs handled by - // grpc.UnknownServiceHandler, but take care to limit the values allowed, as - // allowing too many will increase cardinality and could cause severe memory - // or performance problems. On Client Side, pass a - // grpc.StaticMethodCallOption as a call option into Invoke or NewStream. - // This only applies for server side metrics. + Metrics *stats.MetricSet + + // MethodAttributeFilter is a function that determines whether to record the + // method name of RPCs as an attribute, or to bucket into "other". Take care + // to limit the values allowed, as allowing too many will increase + // cardinality and could cause severe memory or performance problems. + // + // This only applies for server-side metrics. For clients, to record the + // method name in the attributes, pass grpc.StaticMethodCallOption to Invoke + // or NewStream. Note that when using protobuf generated clients, this + // CallOption is included automatically. MethodAttributeFilter func(string) bool - // OptionalLabels are labels received from LB Policies that this component - // should add to metrics that record after receiving incoming metadata. + // OptionalLabels specifies a list of optional labels to enable on any + // metrics that support them. OptionalLabels []string // pluginOption is used to get labels to attach to certain metrics, if set. @@ -207,7 +213,7 @@ type serverMetrics struct { callDuration otelmetric.Float64Histogram } -func createInt64Counter(setOfMetrics map[estats.Metric]bool, metricName estats.Metric, meter otelmetric.Meter, options ...otelmetric.Int64CounterOption) otelmetric.Int64Counter { +func createInt64Counter(setOfMetrics map[string]bool, metricName string, meter otelmetric.Meter, options ...otelmetric.Int64CounterOption) otelmetric.Int64Counter { if _, ok := setOfMetrics[metricName]; !ok { return noop.Int64Counter{} } @@ -219,7 +225,7 @@ func createInt64Counter(setOfMetrics map[estats.Metric]bool, metricName estats.M return ret } -func createFloat64Counter(setOfMetrics map[estats.Metric]bool, metricName estats.Metric, meter otelmetric.Meter, options ...otelmetric.Float64CounterOption) otelmetric.Float64Counter { +func createFloat64Counter(setOfMetrics map[string]bool, metricName string, meter otelmetric.Meter, options ...otelmetric.Float64CounterOption) otelmetric.Float64Counter { if _, ok := setOfMetrics[metricName]; !ok { return noop.Float64Counter{} } @@ -231,7 +237,7 @@ func createFloat64Counter(setOfMetrics map[estats.Metric]bool, metricName estats return ret } -func createInt64Histogram(setOfMetrics map[estats.Metric]bool, metricName estats.Metric, meter otelmetric.Meter, options ...otelmetric.Int64HistogramOption) otelmetric.Int64Histogram { +func createInt64Histogram(setOfMetrics map[string]bool, metricName string, meter otelmetric.Meter, options ...otelmetric.Int64HistogramOption) otelmetric.Int64Histogram { if _, ok := setOfMetrics[metricName]; !ok { return noop.Int64Histogram{} } @@ -243,7 +249,7 @@ func createInt64Histogram(setOfMetrics map[estats.Metric]bool, metricName estats return ret } -func createFloat64Histogram(setOfMetrics map[estats.Metric]bool, metricName estats.Metric, meter otelmetric.Meter, options ...otelmetric.Float64HistogramOption) otelmetric.Float64Histogram { +func createFloat64Histogram(setOfMetrics map[string]bool, metricName string, meter otelmetric.Meter, options ...otelmetric.Float64HistogramOption) otelmetric.Float64Histogram { if _, ok := setOfMetrics[metricName]; !ok { return noop.Float64Histogram{} } @@ -255,7 +261,7 @@ func createFloat64Histogram(setOfMetrics map[estats.Metric]bool, metricName esta return ret } -func createInt64Gauge(setOfMetrics map[estats.Metric]bool, metricName estats.Metric, meter otelmetric.Meter, options ...otelmetric.Int64GaugeOption) otelmetric.Int64Gauge { +func createInt64Gauge(setOfMetrics map[string]bool, metricName string, meter otelmetric.Meter, options ...otelmetric.Int64GaugeOption) otelmetric.Int64Gauge { if _, ok := setOfMetrics[metricName]; !ok { return noop.Int64Gauge{} } @@ -298,7 +304,7 @@ type registryMetrics struct { optionalLabels []string } -func (rm *registryMetrics) registerMetrics(metrics *estats.Metrics, meter otelmetric.Meter) { +func (rm *registryMetrics) registerMetrics(metrics *stats.MetricSet, meter otelmetric.Meter) { rm.intCounts = make(map[*estats.MetricDescriptor]otelmetric.Int64Counter) rm.floatCounts = make(map[*estats.MetricDescriptor]otelmetric.Float64Counter) rm.intHistos = make(map[*estats.MetricDescriptor]otelmetric.Int64Histogram) @@ -379,12 +385,12 @@ var ( // DefaultSizeBounds are the default bounds for metrics which record size. DefaultSizeBounds = []float64{0, 1024, 2048, 4096, 16384, 65536, 262144, 1048576, 4194304, 16777216, 67108864, 268435456, 1073741824, 4294967296} // defaultPerCallMetrics are the default metrics provided by this module. - defaultPerCallMetrics = estats.NewMetrics(ClientAttemptStarted, ClientAttemptDuration, ClientAttemptSentCompressedTotalMessageSize, ClientAttemptRcvdCompressedTotalMessageSize, ClientCallDuration, ServerCallStarted, ServerCallSentCompressedTotalMessageSize, ServerCallRcvdCompressedTotalMessageSize, ServerCallDuration) + defaultPerCallMetrics = stats.NewMetricSet(ClientAttemptStartedMetricName, ClientAttemptDurationMetricName, ClientAttemptSentCompressedTotalMessageSizeMetricName, ClientAttemptRcvdCompressedTotalMessageSizeMetricName, ClientCallDurationMetricName, ServerCallStartedMetricName, ServerCallSentCompressedTotalMessageSizeMetricName, ServerCallRcvdCompressedTotalMessageSizeMetricName, ServerCallDurationMetricName) ) // DefaultMetrics returns a set of default OpenTelemetry metrics. // // This should only be invoked after init time. -func DefaultMetrics() *estats.Metrics { +func DefaultMetrics() *stats.MetricSet { return defaultPerCallMetrics.Join(estats.DefaultMetrics) } diff --git a/stats/opentelemetry/server_metrics.go b/stats/opentelemetry/server_metrics.go index eaea559b2c10..4765afa8ed53 100644 --- a/stats/opentelemetry/server_metrics.go +++ b/stats/opentelemetry/server_metrics.go @@ -264,15 +264,15 @@ func (h *serverStatsHandler) processRPCEnd(ctx context.Context, ai *attemptInfo, } const ( - // ServerCallStarted is the number of server calls started. - ServerCallStarted estats.Metric = "grpc.server.call.started" - // ServerCallSentCompressedTotalMessageSize is the compressed message bytes - // sent per server call. - ServerCallSentCompressedTotalMessageSize estats.Metric = "grpc.server.call.sent_total_compressed_message_size" - // ServerCallRcvdCompressedTotalMessageSize is the compressed message bytes - // received per server call. - ServerCallRcvdCompressedTotalMessageSize estats.Metric = "grpc.server.call.rcvd_total_compressed_message_size" - // ServerCallDuration is the end-to-end time taken to complete a call from - // server transport's perspective. - ServerCallDuration estats.Metric = "grpc.server.call.duration" + // ServerCallStartedMetricName is the number of server calls started. + ServerCallStartedMetricName string = "grpc.server.call.started" + // ServerCallSentCompressedTotalMessageSizeMetricName is the compressed + // message bytes sent per server call. + ServerCallSentCompressedTotalMessageSizeMetricName string = "grpc.server.call.sent_total_compressed_message_size" + // ServerCallRcvdCompressedTotalMessageSizeMetricName is the compressed + // message bytes received per server call. + ServerCallRcvdCompressedTotalMessageSizeMetricName string = "grpc.server.call.rcvd_total_compressed_message_size" + // ServerCallDurationMetricName is the end-to-end time taken to complete a + // call from server transport's perspective. + ServerCallDurationMetricName string = "grpc.server.call.duration" ) From 3ce87dd3800b90da028f08d75023144c03f5706f Mon Sep 17 00:00:00 2001 From: Halvard Skogsrud Date: Tue, 3 Dec 2024 05:27:55 +1100 Subject: [PATCH 12/57] credentials/google: Add cloud-platform scope for ADC (#7887) --- credentials/google/google.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/credentials/google/google.go b/credentials/google/google.go index d18e370df4e7..5a9c9461f0e4 100644 --- a/credentials/google/google.go +++ b/credentials/google/google.go @@ -30,6 +30,8 @@ import ( "google.golang.org/grpc/internal" ) +const defaultCloudPlatformScope = "https://www.googleapis.com/auth/cloud-platform" + var logger = grpclog.Component("credentials") // DefaultCredentialsOptions constructs options to build DefaultCredentials. @@ -120,7 +122,7 @@ var ( return alts.NewClientCreds(alts.DefaultClientOptions()) } newADC = func(ctx context.Context) (credentials.PerRPCCredentials, error) { - return oauth.NewApplicationDefault(ctx) + return oauth.NewApplicationDefault(ctx, defaultCloudPlatformScope) } ) From ab189b0af7ef5e7aa6e20b5293612bd51ab10fe2 Mon Sep 17 00:00:00 2001 From: Zach Reyes <39203661+zasweq@users.noreply.github.com> Date: Mon, 2 Dec 2024 14:48:15 -0500 Subject: [PATCH 13/57] examples/features/csm_observability: Add xDS Credentials (#7875) --- examples/features/csm_observability/client/main.go | 9 ++++++++- examples/features/csm_observability/server/main.go | 14 +++++++++++++- 2 files changed, 21 insertions(+), 2 deletions(-) diff --git a/examples/features/csm_observability/client/main.go b/examples/features/csm_observability/client/main.go index 20b357c2faad..cf33e6cb641d 100644 --- a/examples/features/csm_observability/client/main.go +++ b/examples/features/csm_observability/client/main.go @@ -29,6 +29,7 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" + xdscreds "google.golang.org/grpc/credentials/xds" "google.golang.org/grpc/examples/features/proto/echo" "google.golang.org/grpc/stats/opentelemetry" "google.golang.org/grpc/stats/opentelemetry/csm" @@ -56,7 +57,13 @@ func main() { cleanup := csm.EnableObservability(context.Background(), opentelemetry.Options{MetricsOptions: opentelemetry.MetricsOptions{MeterProvider: provider}}) defer cleanup() - cc, err := grpc.NewClient(*target, grpc.WithTransportCredentials(insecure.NewCredentials())) + // Set up xds credentials that fall back to insecure as described in: + // https://cloud.google.com/service-mesh/docs/service-routing/security-proxyless-setup#workloads_are_unable_to_communicate_in_the_security_setup. + creds, err := xdscreds.NewClientCredentials(xdscreds.ClientOptions{FallbackCreds: insecure.NewCredentials()}) + if err != nil { + log.Fatalf("Failed to create xDS credentials: %v", err) + } + cc, err := grpc.NewClient(*target, grpc.WithTransportCredentials(creds)) if err != nil { log.Fatalf("Failed to start NewClient: %v", err) } diff --git a/examples/features/csm_observability/server/main.go b/examples/features/csm_observability/server/main.go index b87f859aa1c0..3cafe23316b3 100644 --- a/examples/features/csm_observability/server/main.go +++ b/examples/features/csm_observability/server/main.go @@ -28,9 +28,12 @@ import ( "net/http" "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + xdscreds "google.golang.org/grpc/credentials/xds" pb "google.golang.org/grpc/examples/features/proto/echo" "google.golang.org/grpc/stats/opentelemetry" "google.golang.org/grpc/stats/opentelemetry/csm" + "google.golang.org/grpc/xds" "github.com/prometheus/client_golang/prometheus/promhttp" "go.opentelemetry.io/otel/exporters/prometheus" @@ -67,7 +70,16 @@ func main() { if err != nil { log.Fatalf("Failed to listen: %v", err) } - s := grpc.NewServer() + // Set up xds credentials that fall back to insecure as described in: + // https://cloud.google.com/service-mesh/docs/service-routing/security-proxyless-setup#workloads_are_unable_to_communicate_in_the_security_setup. + creds, err := xdscreds.NewServerCredentials(xdscreds.ServerOptions{FallbackCreds: insecure.NewCredentials()}) + if err != nil { + log.Fatalf("Failed to create xDS credentials: %v", err) + } + s, err := xds.NewGRPCServer(grpc.Creds(creds)) + if err != nil { + log.Fatalf("Failed to start xDS Server: %v", err) + } pb.RegisterEchoServer(s, &echoServer{addr: ":" + *port}) log.Printf("Serving on %s\n", *port) From 17d08f746b51fa70911754d36a799c247a2c4b79 Mon Sep 17 00:00:00 2001 From: Doug Fawley Date: Mon, 2 Dec 2024 13:03:58 -0800 Subject: [PATCH 14/57] scripts/gen-deps: filter out grpc modules (#7890) --- scripts/common.sh | 7 +++++++ scripts/gen-deps.sh | 4 +++- scripts/vet.sh | 7 ------- 3 files changed, 10 insertions(+), 8 deletions(-) diff --git a/scripts/common.sh b/scripts/common.sh index dede0789840f..3e641bae4c6c 100755 --- a/scripts/common.sh +++ b/scripts/common.sh @@ -11,6 +11,13 @@ not() { ! "$@" } +# noret_grep will return 0 if zero or more lines were selected, and >1 if an +# error occurred. Suppresses grep's return code of 1 when there are no matches +# (for eg, empty file). +noret_grep() { + grep "$@" || [[ $? == 1 ]] +} + die() { echo "$@" >&2 exit 1 diff --git a/scripts/gen-deps.sh b/scripts/gen-deps.sh index bc647c4d8cba..f4d5ec63e9c6 100755 --- a/scripts/gen-deps.sh +++ b/scripts/gen-deps.sh @@ -3,6 +3,8 @@ set -e # Exit on error set -o pipefail # Fail a pipe if any sub-command fails. +source "$(dirname $0)/common.sh" + if [[ "$#" -ne 1 || ! -d "$1" ]]; then echo "Specify a valid output directory as the first parameter." exit 1 @@ -16,6 +18,6 @@ cd "${SCRIPTS_DIR}/.." git ls-files -- '*.go' | grep -v '\(^\|/\)\(internal\|examples\|benchmark\|interop\|test\|testdata\)\(/\|$\)' | xargs dirname | sort -u | while read d; do pushd "$d" > /dev/null pkg="$(echo "$d" | sed 's;\.;grpc;' | sed 's;/;_;g')" - go list -deps . | sort >| "${OUTPUT_DIR}/$pkg" + go list -deps . | sort | noret_grep -v 'google.golang.org/grpc' >| "${OUTPUT_DIR}/$pkg" popd > /dev/null done diff --git a/scripts/vet.sh b/scripts/vet.sh index 8db7e19c5364..8c63af4cf008 100755 --- a/scripts/vet.sh +++ b/scripts/vet.sh @@ -8,13 +8,6 @@ source "$(dirname $0)/common.sh" # Check to make sure it's safe to modify the user's git repo. git status --porcelain | fail_on_output -# noret_grep will return 0 if zero or more lines were selected, and >1 if an -# error occurred. Suppresses grep's return code of 1 when there are no matches -# (for eg, empty file). -noret_grep() { - grep "$@" || [[ $? == 1 ]] -} - # Undo any edits made by this script. cleanup() { git reset --hard HEAD From 00272e8024ccd4adce8500a2f7daf974ab7fe7d9 Mon Sep 17 00:00:00 2001 From: Arjan Singh Bal <46515553+arjan-bal@users.noreply.github.com> Date: Tue, 3 Dec 2024 11:06:10 +0530 Subject: [PATCH 15/57] dns: Support link local IPv6 addresses (#7889) --- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 10 +++-- .../pickfirstleaf/pickfirstleaf_ext_test.go | 12 +++--- channelz/service/service_test.go | 11 +++--- .../credentials/xds/handshake_info_test.go | 15 ++++++-- internal/resolver/dns/dns_resolver.go | 37 ++++++++++--------- internal/resolver/dns/dns_resolver_test.go | 11 +++++- internal/testutils/fakegrpclb/server.go | 9 +++-- internal/xds/rbac/matchers.go | 7 +++- scripts/vet.sh | 4 ++ security/advancedtls/crl_test.go | 3 +- test/local_creds_test.go | 5 ++- .../xdsresource/filter_chain_test.go | 7 ++-- 12 files changed, 82 insertions(+), 49 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index 1ebf7cea5e94..7a250e6e3217 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -30,6 +30,7 @@ import ( "errors" "fmt" "net" + "net/netip" "sync" "time" @@ -417,11 +418,14 @@ func addressFamily(address string) ipAddrFamily { if err != nil { return ipAddrFamilyUnknown } - ip := net.ParseIP(host) + ip, err := netip.ParseAddr(host) + if err != nil { + return ipAddrFamilyUnknown + } switch { - case ip.To4() != nil: + case ip.Is4() || ip.Is4In6(): return ipAddrFamilyV4 - case ip.To16() != nil: + case ip.Is6(): return ipAddrFamilyV6 default: return ipAddrFamilyUnknown diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 007157249689..69461625fbea 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -1091,7 +1091,7 @@ func (s) TestPickFirstLeaf_InterleavingIPV4Preffered(t *testing.T) { {Addresses: []resolver.Address{{Addr: "[::FFFF:192.168.0.1]:2222"}}}, {Addresses: []resolver.Address{{Addr: "[0001:0001:0001:0001:0001:0001:0001:0001]:8080"}}}, {Addresses: []resolver.Address{{Addr: "[0002:0002:0002:0002:0002:0002:0002:0002]:8080"}}}, - {Addresses: []resolver.Address{{Addr: "[0003:0003:0003:0003:0003:0003:0003:0003]:3333"}}}, + {Addresses: []resolver.Address{{Addr: "[fe80::1%eth0]:3333"}}}, {Addresses: []resolver.Address{{Addr: "grpc.io:80"}}}, // not an IP. }, }, @@ -1107,7 +1107,7 @@ func (s) TestPickFirstLeaf_InterleavingIPV4Preffered(t *testing.T) { {Addr: "2.2.2.2:2"}, {Addr: "[0002:0002:0002:0002:0002:0002:0002:0002]:8080"}, {Addr: "3.3.3.3:3"}, - {Addr: "[0003:0003:0003:0003:0003:0003:0003:0003]:3333"}, + {Addr: "[fe80::1%eth0]:3333"}, {Addr: "[::FFFF:192.168.0.1]:2222"}, } @@ -1135,7 +1135,7 @@ func (s) TestPickFirstLeaf_InterleavingIPv6Preffered(t *testing.T) { {Addresses: []resolver.Address{{Addr: "3.3.3.3:3"}}}, {Addresses: []resolver.Address{{Addr: "[::FFFF:192.168.0.1]:2222"}}}, {Addresses: []resolver.Address{{Addr: "[0002:0002:0002:0002:0002:0002:0002:0002]:2222"}}}, - {Addresses: []resolver.Address{{Addr: "[0003:0003:0003:0003:0003:0003:0003:0003]:3333"}}}, + {Addresses: []resolver.Address{{Addr: "[fe80::1%eth0]:3333"}}}, {Addresses: []resolver.Address{{Addr: "grpc.io:80"}}}, // not an IP. }, }, @@ -1150,7 +1150,7 @@ func (s) TestPickFirstLeaf_InterleavingIPv6Preffered(t *testing.T) { {Addr: "grpc.io:80"}, {Addr: "[0002:0002:0002:0002:0002:0002:0002:0002]:2222"}, {Addr: "2.2.2.2:2"}, - {Addr: "[0003:0003:0003:0003:0003:0003:0003:0003]:3333"}, + {Addr: "[fe80::1%eth0]:3333"}, {Addr: "3.3.3.3:3"}, {Addr: "[::FFFF:192.168.0.1]:2222"}, } @@ -1180,7 +1180,7 @@ func (s) TestPickFirstLeaf_InterleavingUnknownPreffered(t *testing.T) { {Addresses: []resolver.Address{{Addr: "[::FFFF:192.168.0.1]:2222"}}}, {Addresses: []resolver.Address{{Addr: "[0001:0001:0001:0001:0001:0001:0001:0001]:8080"}}}, {Addresses: []resolver.Address{{Addr: "[0002:0002:0002:0002:0002:0002:0002:0002]:8080"}}}, - {Addresses: []resolver.Address{{Addr: "[0003:0003:0003:0003:0003:0003:0003:0003]:3333"}}}, + {Addresses: []resolver.Address{{Addr: "[fe80::1%eth0]:3333"}}}, {Addresses: []resolver.Address{{Addr: "example.com:80"}}}, // not an IP. }, }, @@ -1197,7 +1197,7 @@ func (s) TestPickFirstLeaf_InterleavingUnknownPreffered(t *testing.T) { {Addr: "2.2.2.2:2"}, {Addr: "[0002:0002:0002:0002:0002:0002:0002:0002]:8080"}, {Addr: "3.3.3.3:3"}, - {Addr: "[0003:0003:0003:0003:0003:0003:0003:0003]:3333"}, + {Addr: "[fe80::1%eth0]:3333"}, {Addr: "[::FFFF:192.168.0.1]:2222"}, } diff --git a/channelz/service/service_test.go b/channelz/service/service_test.go index 78a5f2f81fa5..111e5d17e74b 100644 --- a/channelz/service/service_test.go +++ b/channelz/service/service_test.go @@ -22,6 +22,7 @@ import ( "context" "fmt" "net" + "net/netip" "strconv" "strings" "testing" @@ -605,8 +606,8 @@ func (s) TestGetSocket(t *testing.T) { lastMessageReceivedTimestamp: time.Unix(3, 0), localFlowControlWindow: 65536, remoteFlowControlWindow: 1024, - localAddr: &net.TCPAddr{IP: net.ParseIP("1.0.0.1"), Port: 10001}, - remoteAddr: &net.TCPAddr{IP: net.ParseIP("12.0.0.1"), Port: 10002}, + localAddr: &net.TCPAddr{IP: netip.MustParseAddr("1.0.0.1").AsSlice(), Port: 10001}, + remoteAddr: &net.TCPAddr{IP: netip.MustParseAddr("12.0.0.1").AsSlice(), Port: 10002}, remoteName: "remote.remote", }), newSocket(czSocket{ streamsStarted: 10, @@ -637,11 +638,11 @@ func (s) TestGetSocket(t *testing.T) { lastMessageReceivedTimestamp: time.Unix(0, 0), localFlowControlWindow: 65536, remoteFlowControlWindow: 10240, - localAddr: &net.IPAddr{IP: net.ParseIP("1.0.0.1")}, - remoteAddr: &net.IPAddr{IP: net.ParseIP("9.0.0.1")}, + localAddr: &net.IPAddr{IP: netip.MustParseAddr("1.0.0.1").AsSlice()}, + remoteAddr: &net.IPAddr{IP: netip.MustParseAddr("9.0.0.1").AsSlice()}, remoteName: "", }), newSocket(czSocket{ - localAddr: &net.TCPAddr{IP: net.ParseIP("127.0.0.1"), Port: 10001}, + localAddr: &net.TCPAddr{IP: netip.MustParseAddr("127.0.0.1").AsSlice(), Port: 10001}, }), newSocket(czSocket{ security: &credentials.TLSChannelzSecurityValue{ StandardName: "TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256", diff --git a/internal/credentials/xds/handshake_info_test.go b/internal/credentials/xds/handshake_info_test.go index a68024052bbe..a3e598be3db3 100644 --- a/internal/credentials/xds/handshake_info_test.go +++ b/internal/credentials/xds/handshake_info_test.go @@ -21,6 +21,7 @@ package xds import ( "crypto/x509" "net" + "net/netip" "net/url" "regexp" "testing" @@ -142,8 +143,11 @@ func TestMatchingSANExists_FailureCases(t *testing.T) { inputCert := &x509.Certificate{ DNSNames: []string{"foo.bar.example.com", "bar.baz.test.com", "*.example.com"}, EmailAddresses: []string{"foobar@example.com", "barbaz@test.com"}, - IPAddresses: []net.IP{net.ParseIP("192.0.0.1"), net.ParseIP("2001:db8::68")}, - URIs: []*url.URL{url1, url2}, + IPAddresses: []net.IP{ + netip.MustParseAddr("192.0.0.1").AsSlice(), + netip.MustParseAddr("2001:db8::68").AsSlice(), + }, + URIs: []*url.URL{url1, url2}, } tests := []struct { @@ -214,8 +218,11 @@ func TestMatchingSANExists_Success(t *testing.T) { inputCert := &x509.Certificate{ DNSNames: []string{"baz.test.com", "*.example.com"}, EmailAddresses: []string{"foobar@example.com", "barbaz@test.com"}, - IPAddresses: []net.IP{net.ParseIP("192.0.0.1"), net.ParseIP("2001:db8::68")}, - URIs: []*url.URL{url1, url2}, + IPAddresses: []net.IP{ + netip.MustParseAddr("192.0.0.1").AsSlice(), + netip.MustParseAddr("2001:db8::68").AsSlice(), + }, + URIs: []*url.URL{url1, url2}, } tests := []struct { diff --git a/internal/resolver/dns/dns_resolver.go b/internal/resolver/dns/dns_resolver.go index b080ae30bc1b..ba5c5a95d0d7 100644 --- a/internal/resolver/dns/dns_resolver.go +++ b/internal/resolver/dns/dns_resolver.go @@ -26,6 +26,7 @@ import ( "fmt" rand "math/rand/v2" "net" + "net/netip" "os" "strconv" "strings" @@ -122,7 +123,7 @@ func (b *dnsBuilder) Build(target resolver.Target, cc resolver.ClientConn, opts } // IP address. - if ipAddr, ok := formatIP(host); ok { + if ipAddr, err := formatIP(host); err == nil { addr := []resolver.Address{{Addr: ipAddr + ":" + port}} cc.UpdateState(resolver.State{Addresses: addr}) return deadResolver{}, nil @@ -260,9 +261,9 @@ func (d *dnsResolver) lookupSRV(ctx context.Context) ([]resolver.Address, error) return nil, err } for _, a := range lbAddrs { - ip, ok := formatIP(a) - if !ok { - return nil, fmt.Errorf("dns: error parsing A record IP address %v", a) + ip, err := formatIP(a) + if err != nil { + return nil, fmt.Errorf("dns: error parsing A record IP address %v: %v", a, err) } addr := ip + ":" + strconv.Itoa(int(s.Port)) newAddrs = append(newAddrs, resolver.Address{Addr: addr, ServerName: s.Target}) @@ -322,9 +323,9 @@ func (d *dnsResolver) lookupHost(ctx context.Context) ([]resolver.Address, error } newAddrs := make([]resolver.Address, 0, len(addrs)) for _, a := range addrs { - ip, ok := formatIP(a) - if !ok { - return nil, fmt.Errorf("dns: error parsing A record IP address %v", a) + ip, err := formatIP(a) + if err != nil { + return nil, fmt.Errorf("dns: error parsing A record IP address %v: %v", a, err) } addr := ip + ":" + d.port newAddrs = append(newAddrs, resolver.Address{Addr: addr}) @@ -351,19 +352,19 @@ func (d *dnsResolver) lookup() (*resolver.State, error) { return &state, nil } -// formatIP returns ok = false if addr is not a valid textual representation of -// an IP address. If addr is an IPv4 address, return the addr and ok = true. +// formatIP returns an error if addr is not a valid textual representation of +// an IP address. If addr is an IPv4 address, return the addr and error = nil. // If addr is an IPv6 address, return the addr enclosed in square brackets and -// ok = true. -func formatIP(addr string) (addrIP string, ok bool) { - ip := net.ParseIP(addr) - if ip == nil { - return "", false +// error = nil. +func formatIP(addr string) (string, error) { + ip, err := netip.ParseAddr(addr) + if err != nil { + return "", err } - if ip.To4() != nil { - return addr, true + if ip.Is4() { + return addr, nil } - return "[" + addr + "]", true + return "[" + addr + "]", nil } // parseTarget takes the user input target string and default port, returns @@ -379,7 +380,7 @@ func parseTarget(target, defaultPort string) (host, port string, err error) { if target == "" { return "", "", internal.ErrMissingAddr } - if ip := net.ParseIP(target); ip != nil { + if _, err := netip.ParseAddr(target); err == nil { // target is an IPv4 or IPv6(without brackets) address return target, defaultPort, nil } diff --git a/internal/resolver/dns/dns_resolver_test.go b/internal/resolver/dns/dns_resolver_test.go index 4e6a4d544629..47285052e622 100644 --- a/internal/resolver/dns/dns_resolver_test.go +++ b/internal/resolver/dns/dns_resolver_test.go @@ -755,7 +755,16 @@ func (s) TestIPResolver(t *testing.T) { target: "[2001:db8::1]:http", wantAddr: []resolver.Address{{Addr: "[2001:db8::1]:http"}}, }, - // TODO(yuxuanli): zone support? + { + name: "ipv6 with zone and port", + target: "[fe80::1%25eth0]:1234", + wantAddr: []resolver.Address{{Addr: "[fe80::1%eth0]:1234"}}, + }, + { + name: "ipv6 with zone and default port", + target: "fe80::1%25eth0", + wantAddr: []resolver.Address{{Addr: "[fe80::1%eth0]:443"}}, + }, } for _, test := range tests { diff --git a/internal/testutils/fakegrpclb/server.go b/internal/testutils/fakegrpclb/server.go index 82be2c1af1a4..b19e863f2eb1 100644 --- a/internal/testutils/fakegrpclb/server.go +++ b/internal/testutils/fakegrpclb/server.go @@ -24,6 +24,7 @@ import ( "fmt" "io" "net" + "net/netip" "strconv" "sync" "time" @@ -85,9 +86,9 @@ func NewServer(params ServerParams) (*Server, error) { if err != nil { return nil, fmt.Errorf("failed to parse list of backend address %q: %v", addr, err) } - ip := net.ParseIP(ipStr) - if ip == nil { - return nil, fmt.Errorf("failed to parse ip: %q", ipStr) + ip, err := netip.ParseAddr(ipStr) + if err != nil { + return nil, fmt.Errorf("failed to parse ip %q: %v", ipStr, err) } port, err := strconv.Atoi(portStr) if err != nil { @@ -95,7 +96,7 @@ func NewServer(params ServerParams) (*Server, error) { } logger.Infof("Adding backend ip: %q, port: %d to server list", ip.String(), port) servers = append(servers, &lbpb.Server{ - IpAddress: ip, + IpAddress: ip.AsSlice(), Port: int32(port), }) } diff --git a/internal/xds/rbac/matchers.go b/internal/xds/rbac/matchers.go index e1c15018bde0..07bb59cee501 100644 --- a/internal/xds/rbac/matchers.go +++ b/internal/xds/rbac/matchers.go @@ -20,6 +20,7 @@ import ( "errors" "fmt" "net" + "net/netip" "regexp" v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3" @@ -344,7 +345,8 @@ func newRemoteIPMatcher(cidrRange *v3corepb.CidrRange) (*remoteIPMatcher, error) } func (sim *remoteIPMatcher) match(data *rpcData) bool { - return sim.ipNet.Contains(net.IP(net.ParseIP(data.peerInfo.Addr.String()))) + ip, _ := netip.ParseAddr(data.peerInfo.Addr.String()) + return sim.ipNet.Contains(net.IP(ip.AsSlice())) } type localIPMatcher struct { @@ -361,7 +363,8 @@ func newLocalIPMatcher(cidrRange *v3corepb.CidrRange) (*localIPMatcher, error) { } func (dim *localIPMatcher) match(data *rpcData) bool { - return dim.ipNet.Contains(net.IP(net.ParseIP(data.localAddr.String()))) + ip, _ := netip.ParseAddr(data.localAddr.String()) + return dim.ipNet.Contains(net.IP(ip.AsSlice())) } // portMatcher matches on whether the destination port of the RPC matches the diff --git a/scripts/vet.sh b/scripts/vet.sh index 8c63af4cf008..4867e8c677b6 100755 --- a/scripts/vet.sh +++ b/scripts/vet.sh @@ -82,6 +82,10 @@ git grep '"github.com/envoyproxy/go-control-plane/envoy' -- '*.go' ':(exclude)*. git grep -e 'context.Background()' --or -e 'context.TODO()' -- "*_test.go" | grep -v "benchmark/primitives/context_test.go" | grep -v "credential s/google" | grep -v "internal/transport/" | grep -v "xds/internal/" | grep -v "security/advancedtls" | grep -v 'context.WithTimeout(' | not grep -v 'context.WithCancel(' +# Disallow usage of net.ParseIP in favour of netip.ParseAddr as the former +# can't parse link local IPv6 addresses. +not git grep 'net.ParseIP' -- '*.go' + misspell -error . # - gofmt, goimports, go vet, go mod tidy. diff --git a/security/advancedtls/crl_test.go b/security/advancedtls/crl_test.go index e24a8f9b0dc4..450ab561d6e7 100644 --- a/security/advancedtls/crl_test.go +++ b/security/advancedtls/crl_test.go @@ -30,6 +30,7 @@ import ( "fmt" "math/big" "net" + "net/netip" "os" "path" "strings" @@ -463,7 +464,7 @@ func setupTLSConn(t *testing.T) (net.Listener, *x509.Certificate, *ecdsa.Private Subject: pkix.Name{CommonName: "test-cert"}, KeyUsage: x509.KeyUsageCertSign | x509.KeyUsageCRLSign, ExtKeyUsage: []x509.ExtKeyUsage{x509.ExtKeyUsageServerAuth, x509.ExtKeyUsageClientAuth}, - IPAddresses: []net.IP{net.ParseIP("::1")}, + IPAddresses: []net.IP{netip.MustParseAddr("::1").AsSlice()}, CRLDistributionPoints: []string{"http://static.corp.google.com/crl/campus-sln/borg"}, } diff --git a/test/local_creds_test.go b/test/local_creds_test.go index ac32ab520995..241f6306e2b4 100644 --- a/test/local_creds_test.go +++ b/test/local_creds_test.go @@ -22,6 +22,7 @@ import ( "context" "fmt" "net" + "net/netip" "strings" "testing" "time" @@ -181,11 +182,11 @@ func testLocalCredsE2EFail(dopts []grpc.DialOption) error { var fakeClientAddr, fakeServerAddr net.Addr fakeClientAddr = &net.IPAddr{ - IP: net.ParseIP("10.8.9.10"), + IP: netip.MustParseAddr("10.8.9.10").AsSlice(), Zone: "", } fakeServerAddr = &net.IPAddr{ - IP: net.ParseIP("10.8.9.11"), + IP: netip.MustParseAddr("10.8.9.11").AsSlice(), Zone: "", } diff --git a/xds/internal/xdsclient/xdsresource/filter_chain_test.go b/xds/internal/xdsclient/xdsresource/filter_chain_test.go index 1897e387a96b..cdcc86601ceb 100644 --- a/xds/internal/xdsclient/xdsresource/filter_chain_test.go +++ b/xds/internal/xdsclient/xdsresource/filter_chain_test.go @@ -22,6 +22,7 @@ import ( "errors" "fmt" "net" + "net/netip" "strings" "testing" @@ -2438,7 +2439,7 @@ func (s) TestLookup_Successes(t *testing.T) { lis: lisWithoutDefaultChain, params: FilterChainLookupParams{ IsUnspecifiedListener: true, - DestAddr: net.ParseIP("2001:68::db8"), + DestAddr: netip.MustParseAddr("2001:68::db8").AsSlice(), SourceAddr: net.IPv4(10, 1, 1, 1), SourcePort: 1, }, @@ -2468,8 +2469,8 @@ func (s) TestLookup_Successes(t *testing.T) { lis: lisWithoutDefaultChain, params: FilterChainLookupParams{ IsUnspecifiedListener: true, - DestAddr: net.ParseIP("2001:68::1"), - SourceAddr: net.ParseIP("2001:68::2"), + DestAddr: netip.MustParseAddr("2001:68::1").AsSlice(), + SourceAddr: netip.MustParseAddr("2001:68::2").AsSlice(), SourcePort: 1, }, wantFC: &FilterChain{ From 78aa51be7ea153a6773d421c0ac9852e3c5cd22c Mon Sep 17 00:00:00 2001 From: Arjan Singh Bal <46515553+arjan-bal@users.noreply.github.com> Date: Tue, 3 Dec 2024 11:13:29 +0530 Subject: [PATCH 16/57] pickfirst: Stop test servers without closing listeners (#7872) --- .../pickfirstleaf/pickfirstleaf_ext_test.go | 99 +++++++++++-------- 1 file changed, 56 insertions(+), 43 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 69461625fbea..9e835a6731b8 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -67,20 +67,54 @@ func Test(t *testing.T) { grpctest.RunSubTests(t, s{}) } +// testServer is a server than can be stopped and resumed without closing +// the listener. This guarantees the same port number (and address) is used +// after restart. When a server is stopped, it accepts and closes all tcp +// connections from clients. +type testServer struct { + stubserver.StubServer + lis *testutils.RestartableListener +} + +func (s *testServer) stop() { + s.lis.Stop() +} + +func (s *testServer) resume() { + s.lis.Restart() +} + +func newTestServer(t *testing.T) *testServer { + l, err := testutils.LocalTCPListener() + if err != nil { + t.Fatalf("Failed to create listener: %v", err) + } + rl := testutils.NewRestartableListener(l) + ss := stubserver.StubServer{ + EmptyCallF: func(context.Context, *testpb.Empty) (*testpb.Empty, error) { return &testpb.Empty{}, nil }, + Listener: rl, + } + return &testServer{ + StubServer: ss, + lis: rl, + } +} + // setupPickFirstLeaf performs steps required for pick_first tests. It starts a // bunch of backends exporting the TestService, and creates a ClientConn to them. func setupPickFirstLeaf(t *testing.T, backendCount int, opts ...grpc.DialOption) (*grpc.ClientConn, *manual.Resolver, *backendManager) { t.Helper() r := manual.NewBuilderWithScheme("whatever") - backends := make([]*stubserver.StubServer, backendCount) + backends := make([]*testServer, backendCount) addrs := make([]resolver.Address, backendCount) for i := 0; i < backendCount; i++ { - backend := stubserver.StartTestService(t, nil) + server := newTestServer(t) + backend := stubserver.StartTestService(t, &server.StubServer) t.Cleanup(func() { backend.Stop() }) - backends[i] = backend + backends[i] = server addrs[i] = resolver.Address{Addr: backend.Address} } @@ -264,8 +298,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_DisjointLists(t *testing.T) { stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) - bm.backends[0].S.Stop() - bm.backends[0].S = nil + bm.backends[0].stop() r.UpdateState(resolver.State{Addresses: []resolver.Address{addrs[0], addrs[1]}}) var bal *stateStoringBalancer select { @@ -287,8 +320,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_DisjointLists(t *testing.T) { t.Errorf("SubConn states mismatch (-want +got):\n%s", diff) } - bm.backends[2].S.Stop() - bm.backends[2].S = nil + bm.backends[2].stop() r.UpdateState(resolver.State{Addresses: []resolver.Address{addrs[2], addrs[3]}}) if err := pickfirst.CheckRPCsToBackend(ctx, cc, addrs[3]); err != nil { @@ -327,8 +359,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_ActiveBackendInUpdatedList(t *testing stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) - bm.backends[0].S.Stop() - bm.backends[0].S = nil + bm.backends[0].stop() r.UpdateState(resolver.State{Addresses: []resolver.Address{addrs[0], addrs[1]}}) var bal *stateStoringBalancer select { @@ -350,8 +381,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_ActiveBackendInUpdatedList(t *testing t.Errorf("SubConn states mismatch (-want +got):\n%s", diff) } - bm.backends[2].S.Stop() - bm.backends[2].S = nil + bm.backends[2].stop() r.UpdateState(resolver.State{Addresses: []resolver.Address{addrs[2], addrs[1]}}) // Verify that the ClientConn stays in READY. @@ -391,8 +421,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_InActiveBackendInUpdatedList(t *testi stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) - bm.backends[0].S.Stop() - bm.backends[0].S = nil + bm.backends[0].stop() r.UpdateState(resolver.State{Addresses: []resolver.Address{addrs[0], addrs[1]}}) var bal *stateStoringBalancer select { @@ -414,11 +443,9 @@ func (s) TestPickFirstLeaf_ResolverUpdates_InActiveBackendInUpdatedList(t *testi t.Errorf("SubConn states mismatch (-want +got):\n%s", diff) } - bm.backends[2].S.Stop() - bm.backends[2].S = nil - if err := bm.backends[0].StartServer(); err != nil { - t.Fatalf("Failed to re-start test backend: %v", err) - } + bm.backends[2].stop() + bm.backends[0].resume() + r.UpdateState(resolver.State{Addresses: []resolver.Address{addrs[0], addrs[2]}}) if err := pickfirst.CheckRPCsToBackend(ctx, cc, addrs[0]); err != nil { @@ -456,8 +483,7 @@ func (s) TestPickFirstLeaf_ResolverUpdates_IdenticalLists(t *testing.T) { stateSubscriber := &ccStateSubscriber{} internal.SubscribeToConnectivityStateChanges.(func(cc *grpc.ClientConn, s grpcsync.Subscriber) func())(cc, stateSubscriber) - bm.backends[0].S.Stop() - bm.backends[0].S = nil + bm.backends[0].stop() r.UpdateState(resolver.State{Addresses: []resolver.Address{addrs[0], addrs[1]}}) var bal *stateStoringBalancer select { @@ -554,14 +580,11 @@ func (s) TestPickFirstLeaf_StopConnectedServer_FirstServerRestart(t *testing.T) } // Shut down the connected server. - bm.backends[0].S.Stop() - bm.backends[0].S = nil + bm.backends[0].stop() testutils.AwaitState(ctx, t, cc, connectivity.Idle) // Start the new target server. - if err := bm.backends[0].StartServer(); err != nil { - t.Fatalf("Failed to start server: %v", err) - } + bm.backends[0].resume() if err := pickfirst.CheckRPCsToBackend(ctx, cc, addrs[0]); err != nil { t.Fatal(err) @@ -620,14 +643,11 @@ func (s) TestPickFirstLeaf_StopConnectedServer_SecondServerRestart(t *testing.T) } // Shut down the connected server. - bm.backends[1].S.Stop() - bm.backends[1].S = nil + bm.backends[1].stop() testutils.AwaitState(ctx, t, cc, connectivity.Idle) // Start the new target server. - if err := bm.backends[1].StartServer(); err != nil { - t.Fatalf("Failed to start server: %v", err) - } + bm.backends[1].resume() if err := pickfirst.CheckRPCsToBackend(ctx, cc, addrs[1]); err != nil { t.Fatal(err) @@ -692,14 +712,11 @@ func (s) TestPickFirstLeaf_StopConnectedServer_SecondServerToFirst(t *testing.T) } // Shut down the connected server. - bm.backends[1].S.Stop() - bm.backends[1].S = nil + bm.backends[1].stop() testutils.AwaitState(ctx, t, cc, connectivity.Idle) // Start the new target server. - if err := bm.backends[0].StartServer(); err != nil { - t.Fatalf("Failed to start server: %v", err) - } + bm.backends[0].resume() if err := pickfirst.CheckRPCsToBackend(ctx, cc, addrs[0]); err != nil { t.Fatal(err) @@ -763,14 +780,11 @@ func (s) TestPickFirstLeaf_StopConnectedServer_FirstServerToSecond(t *testing.T) } // Shut down the connected server. - bm.backends[0].S.Stop() - bm.backends[0].S = nil + bm.backends[0].stop() testutils.AwaitState(ctx, t, cc, connectivity.Idle) // Start the new target server. - if err := bm.backends[1].StartServer(); err != nil { - t.Fatalf("Failed to start server: %v", err) - } + bm.backends[1].resume() if err := pickfirst.CheckRPCsToBackend(ctx, cc, addrs[1]); err != nil { t.Fatal(err) @@ -1308,14 +1322,13 @@ type scState struct { } type backendManager struct { - backends []*stubserver.StubServer + backends []*testServer } func (b *backendManager) stopAllExcept(index int) { for idx, b := range b.backends { if idx != index { - b.S.Stop() - b.S = nil + b.stop() } } } From 634497b75899887567d48ff134059443069b8196 Mon Sep 17 00:00:00 2001 From: Arjan Singh Bal <46515553+arjan-bal@users.noreply.github.com> Date: Tue, 3 Dec 2024 12:30:29 +0530 Subject: [PATCH 17/57] test: Split import paths for generated message and service code (#7891) --- test/clientconn_state_transition_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/test/clientconn_state_transition_test.go b/test/clientconn_state_transition_test.go index a0a61099aee6..547b5ffb9a24 100644 --- a/test/clientconn_state_transition_test.go +++ b/test/clientconn_state_transition_test.go @@ -39,6 +39,7 @@ import ( "google.golang.org/grpc/internal/stubserver" "google.golang.org/grpc/internal/testutils" testgrpc "google.golang.org/grpc/interop/grpc_testing" + testpb "google.golang.org/grpc/interop/grpc_testing" "google.golang.org/grpc/resolver" "google.golang.org/grpc/resolver/manual" ) @@ -678,7 +679,7 @@ func (s) TestChannelStateTransitionWithRPC(t *testing.T) { // Make an RPC call to transition the channel to CONNECTING. go func() { - _, err := testgrpc.NewTestServiceClient(cc).EmptyCall(ctx, &testgrpc.Empty{}) + _, err := testgrpc.NewTestServiceClient(cc).EmptyCall(ctx, &testpb.Empty{}) if err == nil { t.Errorf("Expected RPC to fail, but it succeeded") } From 5565631455d326987ad3f9dba6d45478cc426c9f Mon Sep 17 00:00:00 2001 From: hanut19 <50198451+hanut19@users.noreply.github.com> Date: Wed, 4 Dec 2024 11:01:22 +0530 Subject: [PATCH 18/57] balancer/pickfirst: replace grpc.Dial with grpc.NewClient in tests (#7879) --- balancer/pickfirst/pickfirst_ext_test.go | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/balancer/pickfirst/pickfirst_ext_test.go b/balancer/pickfirst/pickfirst_ext_test.go index faa6e7dd274e..4354009fcaec 100644 --- a/balancer/pickfirst/pickfirst_ext_test.go +++ b/balancer/pickfirst/pickfirst_ext_test.go @@ -376,12 +376,12 @@ func (s) TestPickFirst_StickyTransientFailure(t *testing.T) { }, }), } - cc, err := grpc.Dial(lis.Addr().String(), dopts...) + cc, err := grpc.NewClient(lis.Addr().String(), dopts...) if err != nil { - t.Fatalf("Failed to dial server at %q: %v", lis.Addr(), err) + t.Fatalf("Failed to create new client: %v", err) } t.Cleanup(func() { cc.Close() }) - + cc.Connect() testutils.AwaitState(ctx, t, cc, connectivity.TransientFailure) // Spawn a goroutine to ensure that the channel stays in TransientFailure. @@ -837,12 +837,12 @@ func (s) TestPickFirst_ResolverError_WithPreviousUpdate_Connecting(t *testing.T) grpc.WithResolvers(r), grpc.WithDefaultServiceConfig(pickFirstServiceConfig), } - cc, err := grpc.Dial(r.Scheme()+":///test.server", dopts...) + cc, err := grpc.NewClient(r.Scheme()+":///test.server", dopts...) if err != nil { - t.Fatalf("grpc.Dial() failed: %v", err) + t.Fatalf("grpc.NewClient() failed: %v", err) } t.Cleanup(func() { cc.Close() }) - + cc.Connect() addrs := []resolver.Address{{Addr: lis.Addr().String()}} r.UpdateState(resolver.State{Addresses: addrs}) testutils.AwaitState(ctx, t, cc, connectivity.Connecting) @@ -892,12 +892,12 @@ func (s) TestPickFirst_ResolverError_WithPreviousUpdate_TransientFailure(t *test grpc.WithResolvers(r), grpc.WithDefaultServiceConfig(pickFirstServiceConfig), } - cc, err := grpc.Dial(r.Scheme()+":///test.server", dopts...) + cc, err := grpc.NewClient(r.Scheme()+":///test.server", dopts...) if err != nil { - t.Fatalf("grpc.Dial() failed: %v", err) + t.Fatalf("grpc.NewClient() failed: %v", err) } t.Cleanup(func() { cc.Close() }) - + cc.Connect() addrs := []resolver.Address{{Addr: lis.Addr().String()}} r.UpdateState(resolver.State{Addresses: addrs}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) From 317271b232677b7869576a49855b01b9f4775d67 Mon Sep 17 00:00:00 2001 From: Arjan Singh Bal <46515553+arjan-bal@users.noreply.github.com> Date: Thu, 5 Dec 2024 10:50:07 +0530 Subject: [PATCH 19/57] pickfirst: Register a health listener when used as a leaf policy (#7832) --- .../pickfirst/pickfirstleaf/pickfirstleaf.go | 173 +++++++++--- .../pickfirstleaf/pickfirstleaf_ext_test.go | 248 ++++++++++++++++++ 2 files changed, 384 insertions(+), 37 deletions(-) diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go index 7a250e6e3217..2fc0a71f9441 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf.go @@ -54,6 +54,10 @@ func init() { balancer.Register(pickfirstBuilder{}) } +// enableHealthListenerKeyType is a unique key type used in resolver attributes +// to indicate whether the health listener usage is enabled. +type enableHealthListenerKeyType struct{} + var ( logger = grpclog.Component("pick-first-leaf-lb") // Name is the name of the pick_first_leaf balancer. @@ -109,10 +113,8 @@ func (pickfirstBuilder) Build(cc balancer.ClientConn, bo balancer.BuildOptions) target: bo.Target.String(), metricsRecorder: bo.MetricsRecorder, // ClientConn will always create a Metrics Recorder. - addressList: addressList{}, subConns: resolver.NewAddressMap(), state: connectivity.Connecting, - mu: sync.Mutex{}, cancelConnectionTimer: func() {}, } b.logger = internalgrpclog.NewPrefixLogger(logger, fmt.Sprintf(logPrefix, b)) @@ -131,6 +133,13 @@ func (pickfirstBuilder) ParseConfig(js json.RawMessage) (serviceconfig.LoadBalan return cfg, nil } +// EnableHealthListener updates the state to configure pickfirst for using a +// generic health listener. +func EnableHealthListener(state resolver.State) resolver.State { + state.Attributes = state.Attributes.WithValue(enableHealthListenerKeyType{}, true) + return state +} + type pfConfig struct { serviceconfig.LoadBalancingConfig `json:"-"` @@ -148,15 +157,19 @@ type scData struct { subConn balancer.SubConn addr resolver.Address - state connectivity.State + rawConnectivityState connectivity.State + // The effective connectivity state based on raw connectivity, health state + // and after following sticky TransientFailure behaviour defined in A62. + effectiveState connectivity.State lastErr error connectionFailedInFirstPass bool } func (b *pickfirstBalancer) newSCData(addr resolver.Address) (*scData, error) { sd := &scData{ - state: connectivity.Idle, - addr: addr, + rawConnectivityState: connectivity.Idle, + effectiveState: connectivity.Idle, + addr: addr, } sc, err := b.cc.NewSubConn([]resolver.Address{addr}, balancer.NewSubConnOptions{ StateListener: func(state balancer.SubConnState) { @@ -181,7 +194,9 @@ type pickfirstBalancer struct { // The mutex is used to ensure synchronization of updates triggered // from the idle picker and the already serialized resolver, // SubConn state updates. - mu sync.Mutex + mu sync.Mutex + // State reported to the channel based on SubConn states and resolver + // updates. state connectivity.State // scData for active subonns mapped by address. subConns *resolver.AddressMap @@ -189,6 +204,7 @@ type pickfirstBalancer struct { firstPass bool numTF int cancelConnectionTimer func() + healthCheckingEnabled bool } // ResolverError is called by the ClientConn when the name resolver produces @@ -214,7 +230,7 @@ func (b *pickfirstBalancer) resolverErrorLocked(err error) { return } - b.cc.UpdateState(balancer.State{ + b.updateBalancerState(balancer.State{ ConnectivityState: connectivity.TransientFailure, Picker: &picker{err: fmt.Errorf("name resolver error: %v", err)}, }) @@ -227,12 +243,12 @@ func (b *pickfirstBalancer) UpdateClientConnState(state balancer.ClientConnState if len(state.ResolverState.Addresses) == 0 && len(state.ResolverState.Endpoints) == 0 { // Cleanup state pertaining to the previous resolver state. // Treat an empty address list like an error by calling b.ResolverError. - b.state = connectivity.TransientFailure b.closeSubConnsLocked() b.addressList.updateAddrs(nil) b.resolverErrorLocked(errors.New("produced zero addresses")) return balancer.ErrBadResolverState } + b.healthCheckingEnabled = state.ResolverState.Attributes.Value(enableHealthListenerKeyType{}) != nil cfg, ok := state.BalancerConfig.(pfConfig) if state.BalancerConfig != nil && !ok { return fmt.Errorf("pickfirst: received illegal BalancerConfig (type %T): %v: %w", state.BalancerConfig, state.BalancerConfig, balancer.ErrBadResolverState) @@ -279,12 +295,15 @@ func (b *pickfirstBalancer) UpdateClientConnState(state balancer.ClientConnState newAddrs = deDupAddresses(newAddrs) newAddrs = interleaveAddresses(newAddrs) - // If the previous ready SubConn exists in new address list, - // keep this connection and don't create new SubConns. prevAddr := b.addressList.currentAddress() + prevSCData, found := b.subConns.Get(prevAddr) prevAddrsCount := b.addressList.size() + isPrevRawConnectivityStateReady := found && prevSCData.(*scData).rawConnectivityState == connectivity.Ready b.addressList.updateAddrs(newAddrs) - if b.state == connectivity.Ready && b.addressList.seekTo(prevAddr) { + + // If the previous ready SubConn exists in new address list, + // keep this connection and don't create new SubConns. + if isPrevRawConnectivityStateReady && b.addressList.seekTo(prevAddr) { return nil } @@ -296,10 +315,9 @@ func (b *pickfirstBalancer) UpdateClientConnState(state balancer.ClientConnState // we should still enter CONNECTING because the sticky TF behaviour // mentioned in A62 applies only when the TRANSIENT_FAILURE is reported // due to connectivity failures. - if b.state == connectivity.Ready || b.state == connectivity.Connecting || prevAddrsCount == 0 { + if isPrevRawConnectivityStateReady || b.state == connectivity.Connecting || prevAddrsCount == 0 { // Start connection attempt at first address. - b.state = connectivity.Connecting - b.cc.UpdateState(balancer.State{ + b.forceUpdateConcludedStateLocked(balancer.State{ ConnectivityState: connectivity.Connecting, Picker: &picker{err: balancer.ErrNoSubConnAvailable}, }) @@ -501,7 +519,7 @@ func (b *pickfirstBalancer) requestConnectionLocked() { } scd := sd.(*scData) - switch scd.state { + switch scd.rawConnectivityState { case connectivity.Idle: scd.subConn.Connect() b.scheduleNextConnectionLocked() @@ -519,7 +537,7 @@ func (b *pickfirstBalancer) requestConnectionLocked() { b.scheduleNextConnectionLocked() return default: - b.logger.Errorf("SubConn with unexpected state %v present in SubConns map.", scd.state) + b.logger.Errorf("SubConn with unexpected state %v present in SubConns map.", scd.rawConnectivityState) return } @@ -562,16 +580,17 @@ func (b *pickfirstBalancer) scheduleNextConnectionLocked() { func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.SubConnState) { b.mu.Lock() defer b.mu.Unlock() - oldState := sd.state - sd.state = newState.ConnectivityState + oldState := sd.rawConnectivityState + sd.rawConnectivityState = newState.ConnectivityState // Previously relevant SubConns can still callback with state updates. // To prevent pickers from returning these obsolete SubConns, this logic // is included to check if the current list of active SubConns includes this // SubConn. - if activeSD, found := b.subConns.Get(sd.addr); !found || activeSD != sd { + if !b.isActiveSCData(sd) { return } if newState.ConnectivityState == connectivity.Shutdown { + sd.effectiveState = connectivity.Shutdown return } @@ -590,10 +609,30 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub b.logger.Errorf("Address %q not found address list in %v", sd.addr, b.addressList.addresses) return } - b.state = connectivity.Ready - b.cc.UpdateState(balancer.State{ - ConnectivityState: connectivity.Ready, - Picker: &picker{result: balancer.PickResult{SubConn: sd.subConn}}, + if !b.healthCheckingEnabled { + if b.logger.V(2) { + b.logger.Infof("SubConn %p reported connectivity state READY and the health listener is disabled. Transitioning SubConn to READY.", sd.subConn) + } + + sd.effectiveState = connectivity.Ready + b.updateBalancerState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &picker{result: balancer.PickResult{SubConn: sd.subConn}}, + }) + return + } + if b.logger.V(2) { + b.logger.Infof("SubConn %p reported connectivity state READY. Registering health listener.", sd.subConn) + } + // Send a CONNECTING update to take the SubConn out of sticky-TF if + // required. + sd.effectiveState = connectivity.Connecting + b.updateBalancerState(balancer.State{ + ConnectivityState: connectivity.Connecting, + Picker: &picker{err: balancer.ErrNoSubConnAvailable}, + }) + sd.subConn.RegisterHealthListener(func(scs balancer.SubConnState) { + b.updateSubConnHealthState(sd, scs) }) return } @@ -604,11 +643,13 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub // a transport is successfully created, but the connection fails // before the SubConn can send the notification for READY. We treat // this as a successful connection and transition to IDLE. - if (b.state == connectivity.Ready && newState.ConnectivityState != connectivity.Ready) || (oldState == connectivity.Connecting && newState.ConnectivityState == connectivity.Idle) { + // TODO: https://github.com/grpc/grpc-go/issues/7862 - Remove the second + // part of the if condition below once the issue is fixed. + if oldState == connectivity.Ready || (oldState == connectivity.Connecting && newState.ConnectivityState == connectivity.Idle) { // Once a transport fails, the balancer enters IDLE and starts from // the first address when the picker is used. b.shutdownRemainingLocked(sd) - b.state = connectivity.Idle + sd.effectiveState = newState.ConnectivityState // READY SubConn interspliced in between CONNECTING and IDLE, need to // account for that. if oldState == connectivity.Connecting { @@ -619,7 +660,7 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub } disconnectionsMetric.Record(b.metricsRecorder, 1, b.target) b.addressList.reset() - b.cc.UpdateState(balancer.State{ + b.updateBalancerState(balancer.State{ ConnectivityState: connectivity.Idle, Picker: &idlePicker{exitIdle: sync.OnceFunc(b.ExitIdle)}, }) @@ -629,19 +670,19 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub if b.firstPass { switch newState.ConnectivityState { case connectivity.Connecting: - // The balancer can be in either IDLE, CONNECTING or - // TRANSIENT_FAILURE. If it's in TRANSIENT_FAILURE, stay in + // The effective state can be in either IDLE, CONNECTING or + // TRANSIENT_FAILURE. If it's TRANSIENT_FAILURE, stay in // TRANSIENT_FAILURE until it's READY. See A62. - // If the balancer is already in CONNECTING, no update is needed. - if b.state == connectivity.Idle { - b.state = connectivity.Connecting - b.cc.UpdateState(balancer.State{ + if sd.effectiveState != connectivity.TransientFailure { + sd.effectiveState = connectivity.Connecting + b.updateBalancerState(balancer.State{ ConnectivityState: connectivity.Connecting, Picker: &picker{err: balancer.ErrNoSubConnAvailable}, }) } case connectivity.TransientFailure: sd.lastErr = newState.ConnectionError + sd.effectiveState = connectivity.TransientFailure // Since we're re-using common SubConns while handling resolver // updates, we could receive an out of turn TRANSIENT_FAILURE from // a pass over the previous address list. Happy Eyeballs will also @@ -668,7 +709,7 @@ func (b *pickfirstBalancer) updateSubConnState(sd *scData, newState balancer.Sub b.numTF = (b.numTF + 1) % b.subConns.Len() sd.lastErr = newState.ConnectionError if b.numTF%b.subConns.Len() == 0 { - b.cc.UpdateState(balancer.State{ + b.updateBalancerState(balancer.State{ ConnectivityState: connectivity.TransientFailure, Picker: &picker{err: newState.ConnectionError}, }) @@ -698,21 +739,79 @@ func (b *pickfirstBalancer) endFirstPassIfPossibleLocked(lastErr error) { } } b.firstPass = false - b.state = connectivity.TransientFailure - - b.cc.UpdateState(balancer.State{ + b.updateBalancerState(balancer.State{ ConnectivityState: connectivity.TransientFailure, Picker: &picker{err: lastErr}, }) // Start re-connecting all the SubConns that are already in IDLE. for _, v := range b.subConns.Values() { sd := v.(*scData) - if sd.state == connectivity.Idle { + if sd.rawConnectivityState == connectivity.Idle { sd.subConn.Connect() } } } +func (b *pickfirstBalancer) isActiveSCData(sd *scData) bool { + activeSD, found := b.subConns.Get(sd.addr) + return found && activeSD == sd +} + +func (b *pickfirstBalancer) updateSubConnHealthState(sd *scData, state balancer.SubConnState) { + b.mu.Lock() + defer b.mu.Unlock() + // Previously relevant SubConns can still callback with state updates. + // To prevent pickers from returning these obsolete SubConns, this logic + // is included to check if the current list of active SubConns includes + // this SubConn. + if !b.isActiveSCData(sd) { + return + } + sd.effectiveState = state.ConnectivityState + switch state.ConnectivityState { + case connectivity.Ready: + b.updateBalancerState(balancer.State{ + ConnectivityState: connectivity.Ready, + Picker: &picker{result: balancer.PickResult{SubConn: sd.subConn}}, + }) + case connectivity.TransientFailure: + b.updateBalancerState(balancer.State{ + ConnectivityState: connectivity.TransientFailure, + Picker: &picker{err: fmt.Errorf("pickfirst: health check failure: %v", state.ConnectionError)}, + }) + case connectivity.Connecting: + b.updateBalancerState(balancer.State{ + ConnectivityState: connectivity.Connecting, + Picker: &picker{err: balancer.ErrNoSubConnAvailable}, + }) + default: + b.logger.Errorf("Got unexpected health update for SubConn %p: %v", state) + } +} + +// updateBalancerState stores the state reported to the channel and calls +// ClientConn.UpdateState(). As an optimization, it avoids sending duplicate +// updates to the channel. +func (b *pickfirstBalancer) updateBalancerState(newState balancer.State) { + // In case of TransientFailures allow the picker to be updated to update + // the connectivity error, in all other cases don't send duplicate state + // updates. + if newState.ConnectivityState == b.state && b.state != connectivity.TransientFailure { + return + } + b.forceUpdateConcludedStateLocked(newState) +} + +// forceUpdateConcludedStateLocked stores the state reported to the channel and +// calls ClientConn.UpdateState(). +// A separate function is defined to force update the ClientConn state since the +// channel doesn't correctly assume that LB policies start in CONNECTING and +// relies on LB policy to send an initial CONNECTING update. +func (b *pickfirstBalancer) forceUpdateConcludedStateLocked(newState balancer.State) { + b.state = newState.ConnectivityState + b.cc.UpdateState(newState) +} + type picker struct { result balancer.PickResult err error diff --git a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go index 9e835a6731b8..9667c2b3db6b 100644 --- a/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go +++ b/balancer/pickfirst/pickfirstleaf/pickfirstleaf_ext_test.go @@ -34,6 +34,7 @@ import ( "google.golang.org/grpc/connectivity" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/balancer/stub" "google.golang.org/grpc/internal/grpcsync" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/stubserver" @@ -1224,6 +1225,253 @@ func (s) TestPickFirstLeaf_InterleavingUnknownPreffered(t *testing.T) { } } +// Test verifies that pickfirst balancer transitions to READY when the health +// listener is enabled. Since client side health checking is not enabled in +// the service config, the health listener will send a health update for READY +// after registering the listener. +func (s) TestPickFirstLeaf_HealthListenerEnabled(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + bf := stub.BalancerFuncs{ + Init: func(bd *stub.BalancerData) { + bd.Data = balancer.Get(pickfirstleaf.Name).Build(bd.ClientConn, bd.BuildOptions) + }, + Close: func(bd *stub.BalancerData) { + bd.Data.(balancer.Balancer).Close() + }, + UpdateClientConnState: func(bd *stub.BalancerData, ccs balancer.ClientConnState) error { + ccs.ResolverState = pickfirstleaf.EnableHealthListener(ccs.ResolverState) + return bd.Data.(balancer.Balancer).UpdateClientConnState(ccs) + }, + } + + stub.Register(t.Name(), bf) + svcCfg := fmt.Sprintf(`{ "loadBalancingConfig": [{%q: {}}] }`, t.Name()) + backend := stubserver.StartTestService(t, nil) + defer backend.Stop() + opts := []grpc.DialOption{ + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithDefaultServiceConfig(svcCfg), + } + cc, err := grpc.NewClient(backend.Address, opts...) + if err != nil { + t.Fatalf("grpc.NewClient(%q) failed: %v", backend.Address, err) + + } + defer cc.Close() + + if err := pickfirst.CheckRPCsToBackend(ctx, cc, resolver.Address{Addr: backend.Address}); err != nil { + t.Fatal(err) + } +} + +// Test verifies that a health listener is not registered when pickfirst is not +// under a petiole policy. +func (s) TestPickFirstLeaf_HealthListenerNotEnabled(t *testing.T) { + // Wrap the clientconn to intercept NewSubConn. + // Capture the health list by wrapping the SC. + // Wrap the picker to unwrap the SC. + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + healthListenerCh := make(chan func(balancer.SubConnState)) + + bf := stub.BalancerFuncs{ + Init: func(bd *stub.BalancerData) { + ccw := &healthListenerCapturingCCWrapper{ + ClientConn: bd.ClientConn, + healthListenerCh: healthListenerCh, + subConnStateCh: make(chan balancer.SubConnState, 5), + } + bd.Data = balancer.Get(pickfirstleaf.Name).Build(ccw, bd.BuildOptions) + }, + Close: func(bd *stub.BalancerData) { + bd.Data.(balancer.Balancer).Close() + }, + UpdateClientConnState: func(bd *stub.BalancerData, ccs balancer.ClientConnState) error { + // Functions like a non-petiole policy by not configuring the use + // of health listeners. + return bd.Data.(balancer.Balancer).UpdateClientConnState(ccs) + }, + } + + stub.Register(t.Name(), bf) + svcCfg := fmt.Sprintf(`{ "loadBalancingConfig": [{%q: {}}] }`, t.Name()) + backend := stubserver.StartTestService(t, nil) + defer backend.Stop() + opts := []grpc.DialOption{ + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithDefaultServiceConfig(svcCfg), + } + cc, err := grpc.NewClient(backend.Address, opts...) + if err != nil { + t.Fatalf("grpc.NewClient(%q) failed: %v", backend.Address, err) + + } + defer cc.Close() + cc.Connect() + + select { + case <-healthListenerCh: + t.Fatal("Health listener registered when not enabled.") + case <-time.After(defaultTestShortTimeout): + } + + testutils.AwaitState(ctx, t, cc, connectivity.Ready) +} + +// Test mocks the updates sent to the health listener and verifies that the +// balancer correctly reports the health state once the SubConn's connectivity +// state becomes READY. +func (s) TestPickFirstLeaf_HealthUpdates(t *testing.T) { + // Wrap the clientconn to intercept NewSubConn. + // Capture the health list by wrapping the SC. + // Wrap the picker to unwrap the SC. + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + healthListenerCh := make(chan func(balancer.SubConnState)) + scConnectivityStateCh := make(chan balancer.SubConnState, 5) + + bf := stub.BalancerFuncs{ + Init: func(bd *stub.BalancerData) { + ccw := &healthListenerCapturingCCWrapper{ + ClientConn: bd.ClientConn, + healthListenerCh: healthListenerCh, + subConnStateCh: scConnectivityStateCh, + } + bd.Data = balancer.Get(pickfirstleaf.Name).Build(ccw, bd.BuildOptions) + }, + Close: func(bd *stub.BalancerData) { + bd.Data.(balancer.Balancer).Close() + }, + UpdateClientConnState: func(bd *stub.BalancerData, ccs balancer.ClientConnState) error { + ccs.ResolverState = pickfirstleaf.EnableHealthListener(ccs.ResolverState) + return bd.Data.(balancer.Balancer).UpdateClientConnState(ccs) + }, + } + + stub.Register(t.Name(), bf) + svcCfg := fmt.Sprintf(`{ "loadBalancingConfig": [{%q: {}}] }`, t.Name()) + backend := stubserver.StartTestService(t, nil) + defer backend.Stop() + opts := []grpc.DialOption{ + grpc.WithTransportCredentials(insecure.NewCredentials()), + grpc.WithDefaultServiceConfig(svcCfg), + } + cc, err := grpc.NewClient(backend.Address, opts...) + if err != nil { + t.Fatalf("grpc.NewClient(%q) failed: %v", backend.Address, err) + + } + defer cc.Close() + cc.Connect() + + var healthListener func(balancer.SubConnState) + select { + case healthListener = <-healthListenerCh: + case <-ctx.Done(): + t.Fatal("Context timed out waiting for health listener to be registered.") + } + + // Wait for the raw connectivity state to become READY. The LB policy should + // wait for the health updates before transitioning the channel to READY. + for { + var scs balancer.SubConnState + select { + case scs = <-scConnectivityStateCh: + case <-ctx.Done(): + t.Fatal("Context timed out waiting for the SubConn connectivity state to become READY.") + } + if scs.ConnectivityState == connectivity.Ready { + break + } + } + + shortCtx, cancel := context.WithTimeout(ctx, defaultTestShortTimeout) + defer cancel() + testutils.AwaitNoStateChange(shortCtx, t, cc, connectivity.Connecting) + + // The LB policy should update the channel state based on the health state. + healthListener(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: fmt.Errorf("test health check failure"), + }) + testutils.AwaitState(ctx, t, cc, connectivity.TransientFailure) + + healthListener(balancer.SubConnState{ + ConnectivityState: connectivity.Connecting, + ConnectionError: balancer.ErrNoSubConnAvailable, + }) + testutils.AwaitState(ctx, t, cc, connectivity.Connecting) + + healthListener(balancer.SubConnState{ + ConnectivityState: connectivity.Ready, + }) + if err := pickfirst.CheckRPCsToBackend(ctx, cc, resolver.Address{Addr: backend.Address}); err != nil { + t.Fatal(err) + } + + // When the health check fails, the channel should transition to TF. + healthListener(balancer.SubConnState{ + ConnectivityState: connectivity.TransientFailure, + ConnectionError: fmt.Errorf("test health check failure"), + }) + testutils.AwaitState(ctx, t, cc, connectivity.TransientFailure) +} + +// healthListenerCapturingCCWrapper is used to capture the health listener so +// that health updates can be mocked for testing. +type healthListenerCapturingCCWrapper struct { + balancer.ClientConn + healthListenerCh chan func(balancer.SubConnState) + subConnStateCh chan balancer.SubConnState +} + +func (ccw *healthListenerCapturingCCWrapper) NewSubConn(addrs []resolver.Address, opts balancer.NewSubConnOptions) (balancer.SubConn, error) { + oldListener := opts.StateListener + opts.StateListener = func(scs balancer.SubConnState) { + ccw.subConnStateCh <- scs + if oldListener != nil { + oldListener(scs) + } + } + sc, err := ccw.ClientConn.NewSubConn(addrs, opts) + if err != nil { + return nil, err + } + return &healthListenerCapturingSCWrapper{ + SubConn: sc, + listenerCh: ccw.healthListenerCh, + }, nil +} + +func (ccw *healthListenerCapturingCCWrapper) UpdateState(state balancer.State) { + state.Picker = &unwrappingPicker{state.Picker} + ccw.ClientConn.UpdateState(state) +} + +type healthListenerCapturingSCWrapper struct { + balancer.SubConn + listenerCh chan func(balancer.SubConnState) +} + +func (scw *healthListenerCapturingSCWrapper) RegisterHealthListener(listener func(balancer.SubConnState)) { + scw.listenerCh <- listener +} + +// unwrappingPicker unwraps SubConns because the channel expects SubConns to be +// addrConns. +type unwrappingPicker struct { + balancer.Picker +} + +func (pw *unwrappingPicker) Pick(info balancer.PickInfo) (balancer.PickResult, error) { + pr, err := pw.Picker.Pick(info) + if pr.SubConn != nil { + pr.SubConn = pr.SubConn.(*healthListenerCapturingSCWrapper).SubConn + } + return pr, err +} + // subConnAddresses makes the pickfirst balancer create the requested number of // SubConns by triggering transient failures. The function returns the // addresses of the created SubConns. From d7286fbc3f8f5d03e232fa3eff21098b76e2277d Mon Sep 17 00:00:00 2001 From: Purnesh Dixit Date: Thu, 5 Dec 2024 14:53:01 +0530 Subject: [PATCH 20/57] Change version to 1.70.0-dev (#7903) --- version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.go b/version.go index a5b038829dfb..271fe23f43e7 100644 --- a/version.go +++ b/version.go @@ -19,4 +19,4 @@ package grpc // Version is the current grpc version. -const Version = "1.69.0-dev" +const Version = "1.70.0-dev" From 645aadf4bdb30a9512659d9e08ea8ae7d2141433 Mon Sep 17 00:00:00 2001 From: Purnesh Dixit Date: Thu, 5 Dec 2024 17:46:58 +0530 Subject: [PATCH 21/57] deps: update dependencies for all modules (#7904) --- .../grpclb/grpc_lb_v1/load_balancer.pb.go | 2 +- binarylog/grpc_binarylog_v1/binarylog.pb.go | 2 +- channelz/grpc_channelz_v1/channelz.pb.go | 2 +- cmd/protoc-gen-go-grpc/go.mod | 2 +- cmd/protoc-gen-go-grpc/go.sum | 4 +- .../internal/proto/grpc_gcp/altscontext.pb.go | 2 +- .../internal/proto/grpc_gcp/handshaker.pb.go | 2 +- .../grpc_gcp/transport_security_common.pb.go | 2 +- examples/features/proto/echo/echo.pb.go | 2 +- examples/go.mod | 34 +++---- examples/go.sum | 95 ++++++++----------- .../helloworld/helloworld/helloworld.pb.go | 2 +- .../route_guide/routeguide/route_guide.pb.go | 2 +- gcp/observability/go.mod | 24 ++--- gcp/observability/go.sum | 88 +++++++---------- go.mod | 34 +++---- go.sum | 68 ++++++------- health/grpc_health_v1/health.pb.go | 2 +- internal/proto/grpc_lookup_v1/rls.pb.go | 2 +- .../proto/grpc_lookup_v1/rls_config.pb.go | 2 +- interop/grpc_testing/benchmark_service.pb.go | 2 +- interop/grpc_testing/control.pb.go | 2 +- interop/grpc_testing/core/stats.pb.go | 2 +- interop/grpc_testing/empty.pb.go | 2 +- interop/grpc_testing/messages.pb.go | 2 +- interop/grpc_testing/payloads.pb.go | 2 +- .../report_qps_scenario_service.pb.go | 2 +- interop/grpc_testing/stats.pb.go | 2 +- interop/grpc_testing/test.pb.go | 2 +- interop/grpc_testing/worker_service.pb.go | 2 +- interop/observability/go.mod | 28 +++--- interop/observability/go.sum | 91 ++++++++---------- interop/stress/grpc_testing/metrics.pb.go | 2 +- interop/xds/go.mod | 32 +++---- interop/xds/go.sum | 64 ++++++------- profiling/proto/service.pb.go | 2 +- .../grpc_reflection_v1/reflection.pb.go | 2 +- .../grpc_reflection_v1alpha/reflection.pb.go | 2 +- reflection/grpc_testing/proto2.pb.go | 2 +- reflection/grpc_testing/proto2_ext.pb.go | 2 +- reflection/grpc_testing/proto2_ext2.pb.go | 2 +- reflection/grpc_testing/test.pb.go | 2 +- security/advancedtls/examples/go.mod | 16 ++-- security/advancedtls/examples/go.sum | 44 ++++----- security/advancedtls/go.mod | 14 +-- security/advancedtls/go.sum | 44 ++++----- stats/opencensus/go.mod | 14 +-- stats/opencensus/go.sum | 83 +++++++--------- test/codec_perf/perf.pb.go | 2 +- test/tools/go.mod | 18 ++-- test/tools/go.sum | 36 +++---- 51 files changed, 412 insertions(+), 483 deletions(-) diff --git a/balancer/grpclb/grpc_lb_v1/load_balancer.pb.go b/balancer/grpclb/grpc_lb_v1/load_balancer.pb.go index 3f274482c74b..86d495bb624f 100644 --- a/balancer/grpclb/grpc_lb_v1/load_balancer.pb.go +++ b/balancer/grpclb/grpc_lb_v1/load_balancer.pb.go @@ -19,7 +19,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/lb/v1/load_balancer.proto diff --git a/binarylog/grpc_binarylog_v1/binarylog.pb.go b/binarylog/grpc_binarylog_v1/binarylog.pb.go index 9e9d0806995c..21dd72969aee 100644 --- a/binarylog/grpc_binarylog_v1/binarylog.pb.go +++ b/binarylog/grpc_binarylog_v1/binarylog.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/binlog/v1/binarylog.proto diff --git a/channelz/grpc_channelz_v1/channelz.pb.go b/channelz/grpc_channelz_v1/channelz.pb.go index 18456d750246..161b2d7704cc 100644 --- a/channelz/grpc_channelz_v1/channelz.pb.go +++ b/channelz/grpc_channelz_v1/channelz.pb.go @@ -21,7 +21,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/channelz/v1/channelz.proto diff --git a/cmd/protoc-gen-go-grpc/go.mod b/cmd/protoc-gen-go-grpc/go.mod index d31ccb766651..e0c1c89856bd 100644 --- a/cmd/protoc-gen-go-grpc/go.mod +++ b/cmd/protoc-gen-go-grpc/go.mod @@ -4,7 +4,7 @@ go 1.22 require ( google.golang.org/grpc v1.65.0 - google.golang.org/protobuf v1.35.1 + google.golang.org/protobuf v1.35.2 ) require ( diff --git a/cmd/protoc-gen-go-grpc/go.sum b/cmd/protoc-gen-go-grpc/go.sum index 5e87ede1d588..797fe278eb0e 100644 --- a/cmd/protoc-gen-go-grpc/go.sum +++ b/cmd/protoc-gen-go-grpc/go.sum @@ -10,5 +10,5 @@ google.golang.org/genproto/googleapis/rpc v0.0.0-20240604185151-ef581f913117 h1: google.golang.org/genproto/googleapis/rpc v0.0.0-20240604185151-ef581f913117/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/grpc v1.65.0 h1:bs/cUb4lp1G5iImFFd3u5ixQzweKizoZJAwBNLR42lc= google.golang.org/grpc v1.65.0/go.mod h1:WgYC2ypjlB0EiQi6wdKixMqukr6lBc0Vo+oOgjrM5ZQ= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= diff --git a/credentials/alts/internal/proto/grpc_gcp/altscontext.pb.go b/credentials/alts/internal/proto/grpc_gcp/altscontext.pb.go index 83d23f65aa54..40e42b6ae582 100644 --- a/credentials/alts/internal/proto/grpc_gcp/altscontext.pb.go +++ b/credentials/alts/internal/proto/grpc_gcp/altscontext.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/gcp/altscontext.proto diff --git a/credentials/alts/internal/proto/grpc_gcp/handshaker.pb.go b/credentials/alts/internal/proto/grpc_gcp/handshaker.pb.go index 915b36df8214..2993bbfab15c 100644 --- a/credentials/alts/internal/proto/grpc_gcp/handshaker.pb.go +++ b/credentials/alts/internal/proto/grpc_gcp/handshaker.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/gcp/handshaker.proto diff --git a/credentials/alts/internal/proto/grpc_gcp/transport_security_common.pb.go b/credentials/alts/internal/proto/grpc_gcp/transport_security_common.pb.go index e9676db4b52a..a8d5c4857b80 100644 --- a/credentials/alts/internal/proto/grpc_gcp/transport_security_common.pb.go +++ b/credentials/alts/internal/proto/grpc_gcp/transport_security_common.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/gcp/transport_security_common.proto diff --git a/examples/features/proto/echo/echo.pb.go b/examples/features/proto/echo/echo.pb.go index 6b23482b348b..0e7f8ea60424 100644 --- a/examples/features/proto/echo/echo.pb.go +++ b/examples/features/proto/echo/echo.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: examples/features/proto/echo/echo.proto diff --git a/examples/go.mod b/examples/go.mod index ed63e04cf9fd..51c2f2b1ff83 100644 --- a/examples/go.mod +++ b/examples/go.mod @@ -6,17 +6,17 @@ require ( github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 github.com/prometheus/client_golang v1.20.5 go.opentelemetry.io/otel/exporters/prometheus v0.53.0 - go.opentelemetry.io/otel/sdk/metric v1.31.0 - golang.org/x/oauth2 v0.23.0 - google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 + go.opentelemetry.io/otel/sdk/metric v1.32.0 + golang.org/x/oauth2 v0.24.0 + google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a google.golang.org/grpc v1.67.1 google.golang.org/grpc/gcp/observability v1.0.1 google.golang.org/grpc/security/advancedtls v1.0.0 - google.golang.org/protobuf v1.35.1 + google.golang.org/protobuf v1.35.2 ) require ( - cel.dev/expr v0.16.2 // indirect + cel.dev/expr v0.19.0 // indirect cloud.google.com/go v0.116.0 // indirect cloud.google.com/go/auth v0.9.8 // indirect cloud.google.com/go/auth/oauth2adapt v0.2.4 // indirect @@ -26,7 +26,7 @@ require ( cloud.google.com/go/monitoring v1.21.1 // indirect cloud.google.com/go/trace v1.11.1 // indirect contrib.go.opencensus.io/exporter/stackdriver v0.13.15-0.20230702191903-2de6d2748484 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.3 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0 // indirect github.com/aws/aws-sdk-go-v2 v1.32.2 // indirect github.com/aws/aws-sdk-go-v2/config v1.28.0 // indirect github.com/aws/aws-sdk-go-v2/credentials v1.17.41 // indirect @@ -61,22 +61,22 @@ require ( github.com/prometheus/common v0.60.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/contrib/detectors/gcp v1.31.0 // indirect + go.opentelemetry.io/contrib/detectors/gcp v1.32.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.56.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.56.0 // indirect - go.opentelemetry.io/otel v1.31.0 // indirect - go.opentelemetry.io/otel/metric v1.31.0 // indirect - go.opentelemetry.io/otel/sdk v1.31.0 // indirect - go.opentelemetry.io/otel/trace v1.31.0 // indirect - golang.org/x/crypto v0.28.0 // indirect - golang.org/x/net v0.30.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.26.0 // indirect - golang.org/x/text v0.19.0 // indirect + go.opentelemetry.io/otel v1.32.0 // indirect + go.opentelemetry.io/otel/metric v1.32.0 // indirect + go.opentelemetry.io/otel/sdk v1.32.0 // indirect + go.opentelemetry.io/otel/trace v1.32.0 // indirect + golang.org/x/crypto v0.30.0 // indirect + golang.org/x/net v0.32.0 // indirect + golang.org/x/sync v0.10.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/text v0.21.0 // indirect golang.org/x/time v0.7.0 // indirect google.golang.org/api v0.201.0 // indirect google.golang.org/genproto v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a // indirect google.golang.org/grpc/stats/opencensus v1.0.0 // indirect ) diff --git a/examples/go.sum b/examples/go.sum index 5bfb0231ed13..9cb58041e4ba 100644 --- a/examples/go.sum +++ b/examples/go.sum @@ -1,6 +1,6 @@ cel.dev/expr v0.15.0/go.mod h1:TRSuuV7DlVCE/uwv5QbAiW/v8l5O8C4eEPHeu7gf7Sg= -cel.dev/expr v0.16.2 h1:RwRhoH17VhAu9U5CMvMhH1PDVgf0tuz9FT+24AfMLfU= -cel.dev/expr v0.16.2/go.mod h1:gXngZQMkWJoSbE8mOzehJlXQyubn/Vg0vR9/F3W7iw8= +cel.dev/expr v0.19.0 h1:lXuo+nDhpyJSpWxpPVi5cPUwzKb+dsdOiw6IreM5yt0= +cel.dev/expr v0.19.0/go.mod h1:MrpN08Q+lEBs+bGYdLxxHkZoUSsCp0nSKTs0nTymJgw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= @@ -220,7 +220,6 @@ cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1h cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= -cloud.google.com/go/compute/metadata v0.5.0/go.mod h1:aHnloV2TPI38yx4s9+wAZhHykWvVCfu7hQbF+9CWoiY= cloud.google.com/go/compute/metadata v0.5.2 h1:UxK4uu/Tn+I3p2dYWTfiX4wva7aYlKixAHn3fyqngqo= cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= @@ -777,9 +776,8 @@ gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zum git.sr.ht/~sbinet/gg v0.3.1/go.mod h1:KGYtlADtqsqANL9ueOFkWymvzUvLMQllU5Ixo+8v3pc= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2/go.mod h1:itPGVDKf9cC/ov4MdvJ2QZ0khw4bfoo9jzwTJlaxy2k= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.3 h1:cb3br57K508pQEFgBxn9GDhPS9HefpyMPK1RzmtMNzk= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.3/go.mod h1:itPGVDKf9cC/ov4MdvJ2QZ0khw4bfoo9jzwTJlaxy2k= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0 h1:3c8yed4lgqTt+oTQ+JNMDo+F4xprBf+O/il4ZC0nRLw= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0/go.mod h1:obipzmGjfSjam60XLwGfqUkJsfiheAl+TUjG+4yzyPM= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= github.com/ajstarks/deck v0.0.0-20200831202436-30c9fc6549a9/go.mod h1:JynElWSGnm/4RlzPXRlREEwqTHAN3T56Bv2ITsFT3gY= github.com/ajstarks/deck/generate v0.0.0-20210309230005-c3f852c02e19/go.mod h1:T13YZdzov6OU0A1+RfKZiZN9ca6VeKdBdyDV+BY97Tk= @@ -817,7 +815,6 @@ github.com/aws/aws-sdk-go-v2/service/sts v1.32.2 h1:CiS7i0+FUe+/YY1GvIBLLrR/XNGZ github.com/aws/aws-sdk-go-v2/service/sts v1.32.2/go.mod h1:HtaiBI8CjYoNVde8arShXb94UbQQi9L4EMr6D+xGBwo= github.com/aws/smithy-go v1.22.0 h1:uunKnWlcoL3zO7q+gG2Pk53joueEOsnNB28QdMsmiMM= github.com/aws/smithy-go v1.22.0/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= -github.com/bazelbuild/rules_go v0.49.0/go.mod h1:Dhcz716Kqg1RHNWos+N6MlXNkjNP2EwZQ0LukRKJfMs= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= @@ -886,12 +883,11 @@ github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre github.com/go-pdf/fpdf v0.5.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/go-pdf/fpdf v0.6.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= github.com/golang/glog v1.1.0/go.mod h1:pfYeQZ3JWZoXTV5sFc986z3HTpwQs9At6P4ImfuP3NQ= -github.com/golang/glog v1.2.2/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= +github.com/golang/glog v1.2.3/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -905,7 +901,6 @@ github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/mock v1.7.0-rc.1/go.mod h1:s42URUywIqd+OcERslBJvOjepvNymP31m3q8d/GkuRs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -1018,7 +1013,6 @@ github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/X github.com/jung-kurt/gofpdf v1.0.0/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51/go.mod h1:CzGEWj7cYgsdH8dAjBGEr58BoE7ScuLd+fwFZ44+/x8= -github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= @@ -1120,24 +1114,24 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/detectors/gcp v1.31.0 h1:G1JQOreVrfhRkner+l4mrGxmfqYCAuy76asTDAo0xsA= -go.opentelemetry.io/contrib/detectors/gcp v1.31.0/go.mod h1:tzQL6E1l+iV44YFTkcAeNQqzXUiekSYP9jjJjXwEd00= +go.opentelemetry.io/contrib/detectors/gcp v1.32.0 h1:P78qWqkLSShicHmAzfECaTgvslqHxblNE9j62Ws1NK8= +go.opentelemetry.io/contrib/detectors/gcp v1.32.0/go.mod h1:TVqo0Sda4Cv8gCIixd7LuLwW4EylumVWfhjZJjDD4DU= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.56.0 h1:yMkBS9yViCc7U7yeLzJPM2XizlfdVvBRSmsQDWu6qc0= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.56.0/go.mod h1:n8MR6/liuGB5EmTETUBeU5ZgqMOlqKRxUaqPQBOANZ8= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.56.0 h1:UP6IpuHFkUgOQL9FFQFrZ+5LiwhhYRbi7VZSIx6Nj5s= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.56.0/go.mod h1:qxuZLtbq5QDtdeSHsS7bcf6EH6uO6jUAgk764zd3rhM= -go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= -go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= go.opentelemetry.io/otel/exporters/prometheus v0.53.0 h1:QXobPHrwiGLM4ufrY3EOmDPJpo2P90UuFau4CDPJA/I= go.opentelemetry.io/otel/exporters/prometheus v0.53.0/go.mod h1:WOAXGr3D00CfzmFxtTV1eR0GpoHuPEu+HJT8UWW2SIU= -go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= -go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= -go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= -go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= -go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= -go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= -go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= -go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= @@ -1159,15 +1153,14 @@ golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= -golang.org/x/crypto v0.15.0/go.mod h1:4ChreQoLWfG3xLDer1WdlH5NdlQ3+mwnQq1YTKY+72g= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= -golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= -golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= +golang.org/x/crypto v0.30.0 h1:RwoQn3GkWiMkzlX562cLB7OxWvjH1L8xutO2WoJcRoY= +golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1226,7 +1219,6 @@ golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.14.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.15.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1289,15 +1281,14 @@ golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= -golang.org/x/net v0.18.0/go.mod h1:/czyP5RqHAH4odGYxBJ1qz0+CE5WZ+2j1YgoEo8F2jQ= golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1326,8 +1317,8 @@ golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= -golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= -golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.24.0 h1:KTBBxWqUa0ykRPLtV69rRto9TLXcqYkeswu48x/gvNE= +golang.org/x/oauth2 v0.24.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1345,11 +1336,11 @@ golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= -golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1429,19 +1420,17 @@ golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.23.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1454,14 +1443,13 @@ golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= -golang.org/x/term v0.14.0/go.mod h1:TySc+nGkYR6qt8km8wUhuFRTVSMIX3XPR58y2lC8vww= golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= -golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= +golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1483,8 +1471,8 @@ golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1532,7 +1520,6 @@ golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= @@ -1542,7 +1529,6 @@ golang.org/x/tools v0.0.0-20201124115921-2c860bdd6e78/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= @@ -1550,7 +1536,6 @@ golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.8/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= @@ -1558,7 +1543,6 @@ golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= -golang.org/x/tools v0.15.0/go.mod h1:hpksKq4dtpQWS1uQ61JkdqWM3LscIS6Slf+VVkm+wQk= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1796,9 +1780,8 @@ google.golang.org/genproto/googleapis/api v0.0.0-20230706204954-ccb25ca9f130/go. google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU= -google.golang.org/genproto/googleapis/api v0.0.0-20240826202546-f6391c0de4c7/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 h1:fVoAXEKA4+yufmbdVYv+SE73+cPZbbbe8paLsHfkK+U= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53/go.mod h1:riSXTwQ4+nqmPGtobMFyW5FqVAmIs0St6VPp4Ug7CE4= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a h1:OAiGFfOiA0v9MRYsSidp3ubZaBnteRUyn3xB2ZQ5G/E= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a/go.mod h1:jehYqy3+AhJU9ve55aNOaSml7wUXjF9x6z2LcCfpAhY= google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:ylj+BE99M198VPbBh6A8d9n3w8fChvyLK3wwBOjXBFA= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234015-3fc162c6f38a/go.mod h1:xURIpW9ES5+/GZhnV6beoEtxQrnkRGIfP5VQG2tCBLc= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= @@ -1812,13 +1795,11 @@ google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go. google.golang.org/genproto/googleapis/rpc v0.0.0-20240318140521-94a12d6c2237/go.mod h1:WtryC6hu0hhx87FDGxWCDptyssuo68sk10vYjF+T9fY= google.golang.org/genproto/googleapis/rpc v0.0.0-20240521202816-d264139d666e/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240823204242-4ba0660f739c/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/genproto/googleapis/rpc v0.0.0-20240826202546-f6391c0de4c7/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241007155032-5fefd90f89a9/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a h1:hgh8P4EuoxpsuKMXX/To36nOFD7vixReXgn8lPGnt+o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= -google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0/go.mod h1:Dk1tviKTvMCz5tvh7t+fh94dhmQVHuCt2OzJB3CTW9Y= google.golang.org/grpc/gcp/observability v1.0.1 h1:2IQ7szW1gobfZaS/sDSAu2uxO0V/aTryMZvlcyqKqQA= google.golang.org/grpc/gcp/observability v1.0.1/go.mod h1:yM0UcrYRMe/B+Nu0mDXeTJNDyIMJRJnzuxqnJMz7Ewk= google.golang.org/grpc/security/advancedtls v1.0.0 h1:/KQ7VP/1bs53/aopk9QhuPyFAp9Dm9Ejix3lzYkCrDA= @@ -1847,8 +1828,8 @@ google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHh google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= diff --git a/examples/helloworld/helloworld/helloworld.pb.go b/examples/helloworld/helloworld/helloworld.pb.go index b8351f257707..fbbf4c216a45 100644 --- a/examples/helloworld/helloworld/helloworld.pb.go +++ b/examples/helloworld/helloworld/helloworld.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: examples/helloworld/helloworld/helloworld.proto diff --git a/examples/route_guide/routeguide/route_guide.pb.go b/examples/route_guide/routeguide/route_guide.pb.go index 5a8babf15aaa..759f38398d0c 100644 --- a/examples/route_guide/routeguide/route_guide.pb.go +++ b/examples/route_guide/routeguide/route_guide.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: examples/route_guide/routeguide/route_guide.proto diff --git a/gcp/observability/go.mod b/gcp/observability/go.mod index f11b9c23569d..d85b4b73eed7 100644 --- a/gcp/observability/go.mod +++ b/gcp/observability/go.mod @@ -8,7 +8,7 @@ require ( github.com/google/go-cmp v0.6.0 github.com/google/uuid v1.6.0 go.opencensus.io v0.24.0 - golang.org/x/oauth2 v0.23.0 + golang.org/x/oauth2 v0.24.0 google.golang.org/api v0.201.0 google.golang.org/grpc v1.67.1 google.golang.org/grpc/stats/opencensus v1.0.0 @@ -46,19 +46,19 @@ require ( github.com/googleapis/gax-go/v2 v2.13.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.56.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.56.0 // indirect - go.opentelemetry.io/otel v1.31.0 // indirect - go.opentelemetry.io/otel/metric v1.31.0 // indirect - go.opentelemetry.io/otel/trace v1.31.0 // indirect - golang.org/x/crypto v0.28.0 // indirect - golang.org/x/net v0.30.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.26.0 // indirect - golang.org/x/text v0.19.0 // indirect + go.opentelemetry.io/otel v1.32.0 // indirect + go.opentelemetry.io/otel/metric v1.32.0 // indirect + go.opentelemetry.io/otel/trace v1.32.0 // indirect + golang.org/x/crypto v0.30.0 // indirect + golang.org/x/net v0.32.0 // indirect + golang.org/x/sync v0.10.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/text v0.21.0 // indirect golang.org/x/time v0.7.0 // indirect google.golang.org/genproto v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/protobuf v1.35.1 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a // indirect + google.golang.org/protobuf v1.35.2 // indirect ) replace google.golang.org/grpc => ../.. diff --git a/gcp/observability/go.sum b/gcp/observability/go.sum index 472ac41d57ad..72588140a0e7 100644 --- a/gcp/observability/go.sum +++ b/gcp/observability/go.sum @@ -1,5 +1,5 @@ cel.dev/expr v0.15.0/go.mod h1:TRSuuV7DlVCE/uwv5QbAiW/v8l5O8C4eEPHeu7gf7Sg= -cel.dev/expr v0.16.2/go.mod h1:gXngZQMkWJoSbE8mOzehJlXQyubn/Vg0vR9/F3W7iw8= +cel.dev/expr v0.19.0/go.mod h1:MrpN08Q+lEBs+bGYdLxxHkZoUSsCp0nSKTs0nTymJgw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= @@ -219,7 +219,6 @@ cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1h cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= -cloud.google.com/go/compute/metadata v0.5.0/go.mod h1:aHnloV2TPI38yx4s9+wAZhHykWvVCfu7hQbF+9CWoiY= cloud.google.com/go/compute/metadata v0.5.2 h1:UxK4uu/Tn+I3p2dYWTfiX4wva7aYlKixAHn3fyqngqo= cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= @@ -776,7 +775,7 @@ gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zum git.sr.ht/~sbinet/gg v0.3.1/go.mod h1:KGYtlADtqsqANL9ueOFkWymvzUvLMQllU5Ixo+8v3pc= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2/go.mod h1:itPGVDKf9cC/ov4MdvJ2QZ0khw4bfoo9jzwTJlaxy2k= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0/go.mod h1:obipzmGjfSjam60XLwGfqUkJsfiheAl+TUjG+4yzyPM= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= github.com/ajstarks/deck v0.0.0-20200831202436-30c9fc6549a9/go.mod h1:JynElWSGnm/4RlzPXRlREEwqTHAN3T56Bv2ITsFT3gY= github.com/ajstarks/deck/generate v0.0.0-20210309230005-c3f852c02e19/go.mod h1:T13YZdzov6OU0A1+RfKZiZN9ca6VeKdBdyDV+BY97Tk= @@ -814,7 +813,6 @@ github.com/aws/aws-sdk-go-v2/service/sts v1.32.2 h1:CiS7i0+FUe+/YY1GvIBLLrR/XNGZ github.com/aws/aws-sdk-go-v2/service/sts v1.32.2/go.mod h1:HtaiBI8CjYoNVde8arShXb94UbQQi9L4EMr6D+xGBwo= github.com/aws/smithy-go v1.22.0 h1:uunKnWlcoL3zO7q+gG2Pk53joueEOsnNB28QdMsmiMM= github.com/aws/smithy-go v1.22.0/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= -github.com/bazelbuild/rules_go v0.49.0/go.mod h1:Dhcz716Kqg1RHNWos+N6MlXNkjNP2EwZQ0LukRKJfMs= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/boombuler/barcode v1.0.1/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= @@ -877,12 +875,11 @@ github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre github.com/go-pdf/fpdf v0.5.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/go-pdf/fpdf v0.6.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= github.com/golang/glog v1.1.0/go.mod h1:pfYeQZ3JWZoXTV5sFc986z3HTpwQs9At6P4ImfuP3NQ= -github.com/golang/glog v1.2.2/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= +github.com/golang/glog v1.2.3/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= @@ -896,7 +893,6 @@ github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/mock v1.7.0-rc.1/go.mod h1:s42URUywIqd+OcERslBJvOjepvNymP31m3q8d/GkuRs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -1009,7 +1005,6 @@ github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/X github.com/jung-kurt/gofpdf v1.0.0/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51/go.mod h1:CzGEWj7cYgsdH8dAjBGEr58BoE7ScuLd+fwFZ44+/x8= -github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= @@ -1096,21 +1091,21 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/detectors/gcp v1.31.0/go.mod h1:tzQL6E1l+iV44YFTkcAeNQqzXUiekSYP9jjJjXwEd00= +go.opentelemetry.io/contrib/detectors/gcp v1.32.0/go.mod h1:TVqo0Sda4Cv8gCIixd7LuLwW4EylumVWfhjZJjDD4DU= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.56.0 h1:yMkBS9yViCc7U7yeLzJPM2XizlfdVvBRSmsQDWu6qc0= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.56.0/go.mod h1:n8MR6/liuGB5EmTETUBeU5ZgqMOlqKRxUaqPQBOANZ8= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.56.0 h1:UP6IpuHFkUgOQL9FFQFrZ+5LiwhhYRbi7VZSIx6Nj5s= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.56.0/go.mod h1:qxuZLtbq5QDtdeSHsS7bcf6EH6uO6jUAgk764zd3rhM= -go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= -go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= -go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= -go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= -go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= -go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= -go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= -go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= -go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= -go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= @@ -1132,15 +1127,14 @@ golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= -golang.org/x/crypto v0.15.0/go.mod h1:4ChreQoLWfG3xLDer1WdlH5NdlQ3+mwnQq1YTKY+72g= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= -golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= -golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= +golang.org/x/crypto v0.30.0 h1:RwoQn3GkWiMkzlX562cLB7OxWvjH1L8xutO2WoJcRoY= +golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1199,7 +1193,6 @@ golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.14.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.15.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1262,15 +1255,14 @@ golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= -golang.org/x/net v0.18.0/go.mod h1:/czyP5RqHAH4odGYxBJ1qz0+CE5WZ+2j1YgoEo8F2jQ= golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1299,8 +1291,8 @@ golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= -golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= -golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.24.0 h1:KTBBxWqUa0ykRPLtV69rRto9TLXcqYkeswu48x/gvNE= +golang.org/x/oauth2 v0.24.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1318,11 +1310,11 @@ golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= -golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1402,19 +1394,17 @@ golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.23.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1427,14 +1417,13 @@ golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= -golang.org/x/term v0.14.0/go.mod h1:TySc+nGkYR6qt8km8wUhuFRTVSMIX3XPR58y2lC8vww= golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= -golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= +golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1456,8 +1445,8 @@ golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1505,7 +1494,6 @@ golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= @@ -1515,7 +1503,6 @@ golang.org/x/tools v0.0.0-20201124115921-2c860bdd6e78/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= @@ -1523,7 +1510,6 @@ golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.8/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= @@ -1531,7 +1517,6 @@ golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= -golang.org/x/tools v0.15.0/go.mod h1:hpksKq4dtpQWS1uQ61JkdqWM3LscIS6Slf+VVkm+wQk= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1769,9 +1754,8 @@ google.golang.org/genproto/googleapis/api v0.0.0-20230706204954-ccb25ca9f130/go. google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU= -google.golang.org/genproto/googleapis/api v0.0.0-20240826202546-f6391c0de4c7/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 h1:fVoAXEKA4+yufmbdVYv+SE73+cPZbbbe8paLsHfkK+U= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53/go.mod h1:riSXTwQ4+nqmPGtobMFyW5FqVAmIs0St6VPp4Ug7CE4= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a h1:OAiGFfOiA0v9MRYsSidp3ubZaBnteRUyn3xB2ZQ5G/E= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a/go.mod h1:jehYqy3+AhJU9ve55aNOaSml7wUXjF9x6z2LcCfpAhY= google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:ylj+BE99M198VPbBh6A8d9n3w8fChvyLK3wwBOjXBFA= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234015-3fc162c6f38a/go.mod h1:xURIpW9ES5+/GZhnV6beoEtxQrnkRGIfP5VQG2tCBLc= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= @@ -1785,13 +1769,11 @@ google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go. google.golang.org/genproto/googleapis/rpc v0.0.0-20240318140521-94a12d6c2237/go.mod h1:WtryC6hu0hhx87FDGxWCDptyssuo68sk10vYjF+T9fY= google.golang.org/genproto/googleapis/rpc v0.0.0-20240521202816-d264139d666e/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240823204242-4ba0660f739c/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/genproto/googleapis/rpc v0.0.0-20240826202546-f6391c0de4c7/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241007155032-5fefd90f89a9/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a h1:hgh8P4EuoxpsuKMXX/To36nOFD7vixReXgn8lPGnt+o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= -google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0/go.mod h1:Dk1tviKTvMCz5tvh7t+fh94dhmQVHuCt2OzJB3CTW9Y= google.golang.org/grpc/stats/opencensus v1.0.0 h1:evSYcRZaSToQp+borzWE52+03joezZeXcKJvZDfkUJA= google.golang.org/grpc/stats/opencensus v1.0.0/go.mod h1:FhdkeYvN43wLYUnapVuRJJ9JXkNwe403iLUW2LKSnjs= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= @@ -1816,8 +1798,8 @@ google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHh google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= diff --git a/go.mod b/go.mod index 9b3d296cc882..2696f8ac7991 100644 --- a/go.mod +++ b/go.mod @@ -6,33 +6,33 @@ require ( github.com/cespare/xxhash/v2 v2.3.0 github.com/cncf/xds/go v0.0.0-20240905190251-b4127c9b8d78 github.com/envoyproxy/go-control-plane v0.13.1 - github.com/golang/glog v1.2.2 + github.com/golang/glog v1.2.3 github.com/golang/protobuf v1.5.4 github.com/google/go-cmp v0.6.0 github.com/google/uuid v1.6.0 - go.opentelemetry.io/contrib/detectors/gcp v1.31.0 - go.opentelemetry.io/otel v1.31.0 - go.opentelemetry.io/otel/metric v1.31.0 - go.opentelemetry.io/otel/sdk v1.31.0 - go.opentelemetry.io/otel/sdk/metric v1.31.0 - go.opentelemetry.io/otel/trace v1.31.0 - golang.org/x/net v0.30.0 - golang.org/x/oauth2 v0.23.0 - golang.org/x/sync v0.8.0 - golang.org/x/sys v0.26.0 - google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 - google.golang.org/protobuf v1.35.1 + go.opentelemetry.io/contrib/detectors/gcp v1.32.0 + go.opentelemetry.io/otel v1.32.0 + go.opentelemetry.io/otel/metric v1.32.0 + go.opentelemetry.io/otel/sdk v1.32.0 + go.opentelemetry.io/otel/sdk/metric v1.32.0 + go.opentelemetry.io/otel/trace v1.32.0 + golang.org/x/net v0.32.0 + golang.org/x/oauth2 v0.24.0 + golang.org/x/sync v0.10.0 + golang.org/x/sys v0.28.0 + google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a + google.golang.org/protobuf v1.35.2 ) require ( - cel.dev/expr v0.16.2 // indirect + cel.dev/expr v0.19.0 // indirect cloud.google.com/go/compute/metadata v0.5.2 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/envoyproxy/protoc-gen-validate v1.1.0 // indirect github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/planetscale/vtprotobuf v0.6.1-0.20240319094008-0393e58bdf10 // indirect - golang.org/x/text v0.19.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 // indirect + golang.org/x/text v0.21.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a // indirect ) diff --git a/go.sum b/go.sum index fe263da6d774..577b6aaeed98 100644 --- a/go.sum +++ b/go.sum @@ -1,9 +1,9 @@ -cel.dev/expr v0.16.2 h1:RwRhoH17VhAu9U5CMvMhH1PDVgf0tuz9FT+24AfMLfU= -cel.dev/expr v0.16.2/go.mod h1:gXngZQMkWJoSbE8mOzehJlXQyubn/Vg0vR9/F3W7iw8= +cel.dev/expr v0.19.0 h1:lXuo+nDhpyJSpWxpPVi5cPUwzKb+dsdOiw6IreM5yt0= +cel.dev/expr v0.19.0/go.mod h1:MrpN08Q+lEBs+bGYdLxxHkZoUSsCp0nSKTs0nTymJgw= cloud.google.com/go/compute/metadata v0.5.2 h1:UxK4uu/Tn+I3p2dYWTfiX4wva7aYlKixAHn3fyqngqo= cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2 h1:cZpsGsWTIFKymTA0je7IIvi1O7Es7apb9CF3EQlOcfE= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2/go.mod h1:itPGVDKf9cC/ov4MdvJ2QZ0khw4bfoo9jzwTJlaxy2k= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0 h1:3c8yed4lgqTt+oTQ+JNMDo+F4xprBf+O/il4ZC0nRLw= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0/go.mod h1:obipzmGjfSjam60XLwGfqUkJsfiheAl+TUjG+4yzyPM= github.com/census-instrumentation/opencensus-proto v0.4.1 h1:iKLQ0xPNFxR/2hzXZMrBo8f1j86j5WHzznCCQxV/b8g= github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw= github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= @@ -21,8 +21,8 @@ github.com/go-logr/logr v1.4.2 h1:6pFjapn8bFcIbiKo3XT4j/BhANplGihG6tvd+8rYgrY= github.com/go-logr/logr v1.4.2/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= -github.com/golang/glog v1.2.2 h1:1+mZ9upx1Dh6FmUTFR1naJ77miKiXgALjWOZ3NVFPmY= -github.com/golang/glog v1.2.2/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= +github.com/golang/glog v1.2.3 h1:oDTdz9f5VGVVNGu/Q7UXKWYsD0873HXLHdJUNBsSEKM= +github.com/golang/glog v1.2.3/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/protobuf v1.5.4 h1:i7eJL8qZTpSEXOPTxNKhASYpMn+8e5Q6AdndVa1dWek= github.com/golang/protobuf v1.5.4/go.mod h1:lnTiLA8Wa4RWRcIUkrtSVa5nRhsEGBg48fD6rSs7xps= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= @@ -35,35 +35,35 @@ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZb github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= -go.opentelemetry.io/contrib/detectors/gcp v1.31.0 h1:G1JQOreVrfhRkner+l4mrGxmfqYCAuy76asTDAo0xsA= -go.opentelemetry.io/contrib/detectors/gcp v1.31.0/go.mod h1:tzQL6E1l+iV44YFTkcAeNQqzXUiekSYP9jjJjXwEd00= -go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= -go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= -go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= -go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= -go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= -go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= -go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= -go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= -go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= -go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= +go.opentelemetry.io/contrib/detectors/gcp v1.32.0 h1:P78qWqkLSShicHmAzfECaTgvslqHxblNE9j62Ws1NK8= +go.opentelemetry.io/contrib/detectors/gcp v1.32.0/go.mod h1:TVqo0Sda4Cv8gCIixd7LuLwW4EylumVWfhjZJjDD4DU= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= -golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= -golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 h1:fVoAXEKA4+yufmbdVYv+SE73+cPZbbbe8paLsHfkK+U= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53/go.mod h1:riSXTwQ4+nqmPGtobMFyW5FqVAmIs0St6VPp4Ug7CE4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= +golang.org/x/oauth2 v0.24.0 h1:KTBBxWqUa0ykRPLtV69rRto9TLXcqYkeswu48x/gvNE= +golang.org/x/oauth2 v0.24.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a h1:OAiGFfOiA0v9MRYsSidp3ubZaBnteRUyn3xB2ZQ5G/E= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a/go.mod h1:jehYqy3+AhJU9ve55aNOaSml7wUXjF9x6z2LcCfpAhY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a h1:hgh8P4EuoxpsuKMXX/To36nOFD7vixReXgn8lPGnt+o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/health/grpc_health_v1/health.pb.go b/health/grpc_health_v1/health.pb.go index 26e16d91924f..467de16bdbcd 100644 --- a/health/grpc_health_v1/health.pb.go +++ b/health/grpc_health_v1/health.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/health/v1/health.proto diff --git a/internal/proto/grpc_lookup_v1/rls.pb.go b/internal/proto/grpc_lookup_v1/rls.pb.go index 14185ca35a0c..22731029f5f3 100644 --- a/internal/proto/grpc_lookup_v1/rls.pb.go +++ b/internal/proto/grpc_lookup_v1/rls.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/lookup/v1/rls.proto diff --git a/internal/proto/grpc_lookup_v1/rls_config.pb.go b/internal/proto/grpc_lookup_v1/rls_config.pb.go index 1549a7aa13a3..73b70c25ea39 100644 --- a/internal/proto/grpc_lookup_v1/rls_config.pb.go +++ b/internal/proto/grpc_lookup_v1/rls_config.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/lookup/v1/rls_config.proto diff --git a/interop/grpc_testing/benchmark_service.pb.go b/interop/grpc_testing/benchmark_service.pb.go index 365bca71c141..807fa8ef5840 100644 --- a/interop/grpc_testing/benchmark_service.pb.go +++ b/interop/grpc_testing/benchmark_service.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/testing/benchmark_service.proto diff --git a/interop/grpc_testing/control.pb.go b/interop/grpc_testing/control.pb.go index 96f1252a10c5..2380b188ac55 100644 --- a/interop/grpc_testing/control.pb.go +++ b/interop/grpc_testing/control.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/testing/control.proto diff --git a/interop/grpc_testing/core/stats.pb.go b/interop/grpc_testing/core/stats.pb.go index 853ddbb9617e..a689f158f7ee 100644 --- a/interop/grpc_testing/core/stats.pb.go +++ b/interop/grpc_testing/core/stats.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/core/stats.proto diff --git a/interop/grpc_testing/empty.pb.go b/interop/grpc_testing/empty.pb.go index 5af8fd67d0b1..c54dc619498d 100644 --- a/interop/grpc_testing/empty.pb.go +++ b/interop/grpc_testing/empty.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/testing/empty.proto diff --git a/interop/grpc_testing/messages.pb.go b/interop/grpc_testing/messages.pb.go index 9b44bd03c0ab..7ae05d83cae8 100644 --- a/interop/grpc_testing/messages.pb.go +++ b/interop/grpc_testing/messages.pb.go @@ -16,7 +16,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/testing/messages.proto diff --git a/interop/grpc_testing/payloads.pb.go b/interop/grpc_testing/payloads.pb.go index 2380728d293e..52ea4484f2f5 100644 --- a/interop/grpc_testing/payloads.pb.go +++ b/interop/grpc_testing/payloads.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/testing/payloads.proto diff --git a/interop/grpc_testing/report_qps_scenario_service.pb.go b/interop/grpc_testing/report_qps_scenario_service.pb.go index 8e74c093f284..630d7aaa2d49 100644 --- a/interop/grpc_testing/report_qps_scenario_service.pb.go +++ b/interop/grpc_testing/report_qps_scenario_service.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/testing/report_qps_scenario_service.proto diff --git a/interop/grpc_testing/stats.pb.go b/interop/grpc_testing/stats.pb.go index b0aed877755a..36cb78dfeb24 100644 --- a/interop/grpc_testing/stats.pb.go +++ b/interop/grpc_testing/stats.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/testing/stats.proto diff --git a/interop/grpc_testing/test.pb.go b/interop/grpc_testing/test.pb.go index 148a6beb54f0..7a244bfa7841 100644 --- a/interop/grpc_testing/test.pb.go +++ b/interop/grpc_testing/test.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/testing/test.proto diff --git a/interop/grpc_testing/worker_service.pb.go b/interop/grpc_testing/worker_service.pb.go index d074cb5e3e6d..a45cab37c6d6 100644 --- a/interop/grpc_testing/worker_service.pb.go +++ b/interop/grpc_testing/worker_service.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/testing/worker_service.proto diff --git a/interop/observability/go.mod b/interop/observability/go.mod index 149abb2e900f..07bcfcee15bc 100644 --- a/interop/observability/go.mod +++ b/interop/observability/go.mod @@ -3,7 +3,7 @@ module google.golang.org/grpc/interop/observability go 1.22 require ( - google.golang.org/grpc v1.67.1 + google.golang.org/grpc v1.68.1 google.golang.org/grpc/gcp/observability v1.0.1 ) @@ -36,7 +36,7 @@ require ( github.com/felixge/httpsnoop v1.0.4 // indirect github.com/go-logr/logr v1.4.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect - github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect + github.com/golang/groupcache v0.0.0-20241129210726-2c02b8208cf8 // indirect github.com/golang/protobuf v1.5.4 // indirect github.com/google/s2a-go v0.1.8 // indirect github.com/google/uuid v1.6.0 // indirect @@ -45,22 +45,22 @@ require ( go.opencensus.io v0.24.0 // indirect go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.56.0 // indirect go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.56.0 // indirect - go.opentelemetry.io/otel v1.31.0 // indirect - go.opentelemetry.io/otel/metric v1.31.0 // indirect - go.opentelemetry.io/otel/trace v1.31.0 // indirect - golang.org/x/crypto v0.28.0 // indirect - golang.org/x/net v0.30.0 // indirect - golang.org/x/oauth2 v0.23.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.26.0 // indirect - golang.org/x/text v0.19.0 // indirect + go.opentelemetry.io/otel v1.32.0 // indirect + go.opentelemetry.io/otel/metric v1.32.0 // indirect + go.opentelemetry.io/otel/trace v1.32.0 // indirect + golang.org/x/crypto v0.30.0 // indirect + golang.org/x/net v0.32.0 // indirect + golang.org/x/oauth2 v0.24.0 // indirect + golang.org/x/sync v0.10.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/text v0.21.0 // indirect golang.org/x/time v0.7.0 // indirect google.golang.org/api v0.201.0 // indirect google.golang.org/genproto v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a // indirect google.golang.org/grpc/stats/opencensus v1.0.0 // indirect - google.golang.org/protobuf v1.35.1 // indirect + google.golang.org/protobuf v1.35.2 // indirect ) replace google.golang.org/grpc => ../.. diff --git a/interop/observability/go.sum b/interop/observability/go.sum index 4cdbd27b0fc9..054a3377d46c 100644 --- a/interop/observability/go.sum +++ b/interop/observability/go.sum @@ -1,5 +1,5 @@ cel.dev/expr v0.15.0/go.mod h1:TRSuuV7DlVCE/uwv5QbAiW/v8l5O8C4eEPHeu7gf7Sg= -cel.dev/expr v0.16.2/go.mod h1:gXngZQMkWJoSbE8mOzehJlXQyubn/Vg0vR9/F3W7iw8= +cel.dev/expr v0.19.0/go.mod h1:MrpN08Q+lEBs+bGYdLxxHkZoUSsCp0nSKTs0nTymJgw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= @@ -219,7 +219,6 @@ cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1h cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= -cloud.google.com/go/compute/metadata v0.5.0/go.mod h1:aHnloV2TPI38yx4s9+wAZhHykWvVCfu7hQbF+9CWoiY= cloud.google.com/go/compute/metadata v0.5.2 h1:UxK4uu/Tn+I3p2dYWTfiX4wva7aYlKixAHn3fyqngqo= cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= @@ -776,7 +775,7 @@ gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zum git.sr.ht/~sbinet/gg v0.3.1/go.mod h1:KGYtlADtqsqANL9ueOFkWymvzUvLMQllU5Ixo+8v3pc= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2/go.mod h1:itPGVDKf9cC/ov4MdvJ2QZ0khw4bfoo9jzwTJlaxy2k= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0/go.mod h1:obipzmGjfSjam60XLwGfqUkJsfiheAl+TUjG+4yzyPM= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= github.com/ajstarks/deck v0.0.0-20200831202436-30c9fc6549a9/go.mod h1:JynElWSGnm/4RlzPXRlREEwqTHAN3T56Bv2ITsFT3gY= github.com/ajstarks/deck/generate v0.0.0-20210309230005-c3f852c02e19/go.mod h1:T13YZdzov6OU0A1+RfKZiZN9ca6VeKdBdyDV+BY97Tk= @@ -814,7 +813,6 @@ github.com/aws/aws-sdk-go-v2/service/sts v1.32.2 h1:CiS7i0+FUe+/YY1GvIBLLrR/XNGZ github.com/aws/aws-sdk-go-v2/service/sts v1.32.2/go.mod h1:HtaiBI8CjYoNVde8arShXb94UbQQi9L4EMr6D+xGBwo= github.com/aws/smithy-go v1.22.0 h1:uunKnWlcoL3zO7q+gG2Pk53joueEOsnNB28QdMsmiMM= github.com/aws/smithy-go v1.22.0/go.mod h1:irrKGvNn1InZwb2d7fkIRNucdfwR8R+Ts3wxYa/cJHg= -github.com/bazelbuild/rules_go v0.49.0/go.mod h1:Dhcz716Kqg1RHNWos+N6MlXNkjNP2EwZQ0LukRKJfMs= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/boombuler/barcode v1.0.1/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= @@ -879,17 +877,17 @@ github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre github.com/go-pdf/fpdf v0.5.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/go-pdf/fpdf v0.6.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= github.com/golang/glog v1.1.0/go.mod h1:pfYeQZ3JWZoXTV5sFc986z3HTpwQs9At6P4ImfuP3NQ= -github.com/golang/glog v1.2.2/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= +github.com/golang/glog v1.2.3/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20241129210726-2c02b8208cf8 h1:f+oWsMOmNPc8JmEHVZIycC7hBoQxHH9pNKQORJNozsQ= +github.com/golang/groupcache v0.0.0-20241129210726-2c02b8208cf8/go.mod h1:wcDNUvekVysuuOpQKo3191zZyTpiI6se1N1ULghS0sw= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= @@ -898,7 +896,6 @@ github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/mock v1.7.0-rc.1/go.mod h1:s42URUywIqd+OcERslBJvOjepvNymP31m3q8d/GkuRs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -1011,7 +1008,6 @@ github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/X github.com/jung-kurt/gofpdf v1.0.0/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51/go.mod h1:CzGEWj7cYgsdH8dAjBGEr58BoE7ScuLd+fwFZ44+/x8= -github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= @@ -1098,21 +1094,21 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/detectors/gcp v1.31.0/go.mod h1:tzQL6E1l+iV44YFTkcAeNQqzXUiekSYP9jjJjXwEd00= +go.opentelemetry.io/contrib/detectors/gcp v1.32.0/go.mod h1:TVqo0Sda4Cv8gCIixd7LuLwW4EylumVWfhjZJjDD4DU= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.56.0 h1:yMkBS9yViCc7U7yeLzJPM2XizlfdVvBRSmsQDWu6qc0= go.opentelemetry.io/contrib/instrumentation/google.golang.org/grpc/otelgrpc v0.56.0/go.mod h1:n8MR6/liuGB5EmTETUBeU5ZgqMOlqKRxUaqPQBOANZ8= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.56.0 h1:UP6IpuHFkUgOQL9FFQFrZ+5LiwhhYRbi7VZSIx6Nj5s= go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.56.0/go.mod h1:qxuZLtbq5QDtdeSHsS7bcf6EH6uO6jUAgk764zd3rhM= -go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= -go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= -go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= -go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= -go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= -go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= -go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= -go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= -go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= -go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= @@ -1134,15 +1130,14 @@ golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= -golang.org/x/crypto v0.15.0/go.mod h1:4ChreQoLWfG3xLDer1WdlH5NdlQ3+mwnQq1YTKY+72g= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= -golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= -golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= +golang.org/x/crypto v0.30.0 h1:RwoQn3GkWiMkzlX562cLB7OxWvjH1L8xutO2WoJcRoY= +golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1201,7 +1196,6 @@ golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.14.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.15.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1264,15 +1258,14 @@ golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= -golang.org/x/net v0.18.0/go.mod h1:/czyP5RqHAH4odGYxBJ1qz0+CE5WZ+2j1YgoEo8F2jQ= golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1301,8 +1294,8 @@ golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= -golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= -golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.24.0 h1:KTBBxWqUa0ykRPLtV69rRto9TLXcqYkeswu48x/gvNE= +golang.org/x/oauth2 v0.24.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1320,11 +1313,11 @@ golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= -golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1404,19 +1397,17 @@ golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.23.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1429,14 +1420,13 @@ golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= -golang.org/x/term v0.14.0/go.mod h1:TySc+nGkYR6qt8km8wUhuFRTVSMIX3XPR58y2lC8vww= golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= -golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= +golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1458,8 +1448,8 @@ golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1507,7 +1497,6 @@ golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= @@ -1517,7 +1506,6 @@ golang.org/x/tools v0.0.0-20201124115921-2c860bdd6e78/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= @@ -1525,7 +1513,6 @@ golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.8/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= @@ -1533,7 +1520,6 @@ golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= -golang.org/x/tools v0.15.0/go.mod h1:hpksKq4dtpQWS1uQ61JkdqWM3LscIS6Slf+VVkm+wQk= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1771,9 +1757,8 @@ google.golang.org/genproto/googleapis/api v0.0.0-20230706204954-ccb25ca9f130/go. google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU= -google.golang.org/genproto/googleapis/api v0.0.0-20240826202546-f6391c0de4c7/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 h1:fVoAXEKA4+yufmbdVYv+SE73+cPZbbbe8paLsHfkK+U= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53/go.mod h1:riSXTwQ4+nqmPGtobMFyW5FqVAmIs0St6VPp4Ug7CE4= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a h1:OAiGFfOiA0v9MRYsSidp3ubZaBnteRUyn3xB2ZQ5G/E= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a/go.mod h1:jehYqy3+AhJU9ve55aNOaSml7wUXjF9x6z2LcCfpAhY= google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:ylj+BE99M198VPbBh6A8d9n3w8fChvyLK3wwBOjXBFA= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234015-3fc162c6f38a/go.mod h1:xURIpW9ES5+/GZhnV6beoEtxQrnkRGIfP5VQG2tCBLc= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= @@ -1787,13 +1772,11 @@ google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go. google.golang.org/genproto/googleapis/rpc v0.0.0-20240318140521-94a12d6c2237/go.mod h1:WtryC6hu0hhx87FDGxWCDptyssuo68sk10vYjF+T9fY= google.golang.org/genproto/googleapis/rpc v0.0.0-20240521202816-d264139d666e/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240823204242-4ba0660f739c/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/genproto/googleapis/rpc v0.0.0-20240826202546-f6391c0de4c7/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241007155032-5fefd90f89a9/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a h1:hgh8P4EuoxpsuKMXX/To36nOFD7vixReXgn8lPGnt+o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= -google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0/go.mod h1:Dk1tviKTvMCz5tvh7t+fh94dhmQVHuCt2OzJB3CTW9Y= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -1816,8 +1799,8 @@ google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHh google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= diff --git a/interop/stress/grpc_testing/metrics.pb.go b/interop/stress/grpc_testing/metrics.pb.go index 98298d0ead68..87ab0bf40723 100644 --- a/interop/stress/grpc_testing/metrics.pb.go +++ b/interop/stress/grpc_testing/metrics.pb.go @@ -21,7 +21,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: interop/stress/grpc_testing/metrics.proto diff --git a/interop/xds/go.mod b/interop/xds/go.mod index 5d37a33bce40..2cb3c2389040 100644 --- a/interop/xds/go.mod +++ b/interop/xds/go.mod @@ -7,14 +7,14 @@ replace google.golang.org/grpc => ../.. require ( github.com/prometheus/client_golang v1.20.5 go.opentelemetry.io/otel/exporters/prometheus v0.53.0 - go.opentelemetry.io/otel/sdk/metric v1.31.0 + go.opentelemetry.io/otel/sdk/metric v1.32.0 google.golang.org/grpc v1.67.1 ) require ( - cel.dev/expr v0.16.2 // indirect + cel.dev/expr v0.19.0 // indirect cloud.google.com/go/compute/metadata v0.5.2 // indirect - github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.3 // indirect + github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0 // indirect github.com/beorn7/perks v1.0.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.3.0 // indirect @@ -30,17 +30,17 @@ require ( github.com/prometheus/client_model v0.6.1 // indirect github.com/prometheus/common v0.60.0 // indirect github.com/prometheus/procfs v0.15.1 // indirect - go.opentelemetry.io/contrib/detectors/gcp v1.31.0 // indirect - go.opentelemetry.io/otel v1.31.0 // indirect - go.opentelemetry.io/otel/metric v1.31.0 // indirect - go.opentelemetry.io/otel/sdk v1.31.0 // indirect - go.opentelemetry.io/otel/trace v1.31.0 // indirect - golang.org/x/net v0.30.0 // indirect - golang.org/x/oauth2 v0.23.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.26.0 // indirect - golang.org/x/text v0.19.0 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/protobuf v1.35.1 // indirect + go.opentelemetry.io/contrib/detectors/gcp v1.32.0 // indirect + go.opentelemetry.io/otel v1.32.0 // indirect + go.opentelemetry.io/otel/metric v1.32.0 // indirect + go.opentelemetry.io/otel/sdk v1.32.0 // indirect + go.opentelemetry.io/otel/trace v1.32.0 // indirect + golang.org/x/net v0.32.0 // indirect + golang.org/x/oauth2 v0.24.0 // indirect + golang.org/x/sync v0.10.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/text v0.21.0 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a // indirect + google.golang.org/protobuf v1.35.2 // indirect ) diff --git a/interop/xds/go.sum b/interop/xds/go.sum index 16f77351e6eb..f6dac8d48bea 100644 --- a/interop/xds/go.sum +++ b/interop/xds/go.sum @@ -1,9 +1,9 @@ -cel.dev/expr v0.16.2 h1:RwRhoH17VhAu9U5CMvMhH1PDVgf0tuz9FT+24AfMLfU= -cel.dev/expr v0.16.2/go.mod h1:gXngZQMkWJoSbE8mOzehJlXQyubn/Vg0vR9/F3W7iw8= +cel.dev/expr v0.19.0 h1:lXuo+nDhpyJSpWxpPVi5cPUwzKb+dsdOiw6IreM5yt0= +cel.dev/expr v0.19.0/go.mod h1:MrpN08Q+lEBs+bGYdLxxHkZoUSsCp0nSKTs0nTymJgw= cloud.google.com/go/compute/metadata v0.5.2 h1:UxK4uu/Tn+I3p2dYWTfiX4wva7aYlKixAHn3fyqngqo= cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.3 h1:cb3br57K508pQEFgBxn9GDhPS9HefpyMPK1RzmtMNzk= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.3/go.mod h1:itPGVDKf9cC/ov4MdvJ2QZ0khw4bfoo9jzwTJlaxy2k= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0 h1:3c8yed4lgqTt+oTQ+JNMDo+F4xprBf+O/il4ZC0nRLw= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0/go.mod h1:obipzmGjfSjam60XLwGfqUkJsfiheAl+TUjG+4yzyPM= github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/census-instrumentation/opencensus-proto v0.4.1 h1:iKLQ0xPNFxR/2hzXZMrBo8f1j86j5WHzznCCQxV/b8g= @@ -49,35 +49,35 @@ github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0leargg github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= -go.opentelemetry.io/contrib/detectors/gcp v1.31.0 h1:G1JQOreVrfhRkner+l4mrGxmfqYCAuy76asTDAo0xsA= -go.opentelemetry.io/contrib/detectors/gcp v1.31.0/go.mod h1:tzQL6E1l+iV44YFTkcAeNQqzXUiekSYP9jjJjXwEd00= -go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= -go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= +go.opentelemetry.io/contrib/detectors/gcp v1.32.0 h1:P78qWqkLSShicHmAzfECaTgvslqHxblNE9j62Ws1NK8= +go.opentelemetry.io/contrib/detectors/gcp v1.32.0/go.mod h1:TVqo0Sda4Cv8gCIixd7LuLwW4EylumVWfhjZJjDD4DU= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= go.opentelemetry.io/otel/exporters/prometheus v0.53.0 h1:QXobPHrwiGLM4ufrY3EOmDPJpo2P90UuFau4CDPJA/I= go.opentelemetry.io/otel/exporters/prometheus v0.53.0/go.mod h1:WOAXGr3D00CfzmFxtTV1eR0GpoHuPEu+HJT8UWW2SIU= -go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= -go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= -go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= -go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= -go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= -go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= -go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= -go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= -golang.org/x/oauth2 v0.23.0 h1:PbgcYx2W7i4LvjJWEbf0ngHV6qJYr86PkAV3bXdLEbs= -golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53 h1:fVoAXEKA4+yufmbdVYv+SE73+cPZbbbe8paLsHfkK+U= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53/go.mod h1:riSXTwQ4+nqmPGtobMFyW5FqVAmIs0St6VPp4Ug7CE4= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= +golang.org/x/oauth2 v0.24.0 h1:KTBBxWqUa0ykRPLtV69rRto9TLXcqYkeswu48x/gvNE= +golang.org/x/oauth2 v0.24.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a h1:OAiGFfOiA0v9MRYsSidp3ubZaBnteRUyn3xB2ZQ5G/E= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a/go.mod h1:jehYqy3+AhJU9ve55aNOaSml7wUXjF9x6z2LcCfpAhY= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a h1:hgh8P4EuoxpsuKMXX/To36nOFD7vixReXgn8lPGnt+o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= diff --git a/profiling/proto/service.pb.go b/profiling/proto/service.pb.go index e63964fb8d9c..179b7d7c0d5f 100644 --- a/profiling/proto/service.pb.go +++ b/profiling/proto/service.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: profiling/proto/service.proto diff --git a/reflection/grpc_reflection_v1/reflection.pb.go b/reflection/grpc_reflection_v1/reflection.pb.go index 58019722d01b..30cb61c65e19 100644 --- a/reflection/grpc_reflection_v1/reflection.pb.go +++ b/reflection/grpc_reflection_v1/reflection.pb.go @@ -21,7 +21,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: grpc/reflection/v1/reflection.proto diff --git a/reflection/grpc_reflection_v1alpha/reflection.pb.go b/reflection/grpc_reflection_v1alpha/reflection.pb.go index 4d7b654ef544..3747f6d5334f 100644 --- a/reflection/grpc_reflection_v1alpha/reflection.pb.go +++ b/reflection/grpc_reflection_v1alpha/reflection.pb.go @@ -18,7 +18,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // grpc/reflection/v1alpha/reflection.proto is a deprecated file. diff --git a/reflection/grpc_testing/proto2.pb.go b/reflection/grpc_testing/proto2.pb.go index 3b12b738d1a9..038f334c247f 100644 --- a/reflection/grpc_testing/proto2.pb.go +++ b/reflection/grpc_testing/proto2.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: reflection/grpc_testing/proto2.proto diff --git a/reflection/grpc_testing/proto2_ext.pb.go b/reflection/grpc_testing/proto2_ext.pb.go index d72a25d03e11..dd958358d032 100644 --- a/reflection/grpc_testing/proto2_ext.pb.go +++ b/reflection/grpc_testing/proto2_ext.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: reflection/grpc_testing/proto2_ext.proto diff --git a/reflection/grpc_testing/proto2_ext2.pb.go b/reflection/grpc_testing/proto2_ext2.pb.go index 2be507e1cb37..9b80d3afb6d7 100644 --- a/reflection/grpc_testing/proto2_ext2.pb.go +++ b/reflection/grpc_testing/proto2_ext2.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: reflection/grpc_testing/proto2_ext2.proto diff --git a/reflection/grpc_testing/test.pb.go b/reflection/grpc_testing/test.pb.go index 1e411dc3e519..dec5f57665dc 100644 --- a/reflection/grpc_testing/test.pb.go +++ b/reflection/grpc_testing/test.pb.go @@ -14,7 +14,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: reflection/grpc_testing/test.proto diff --git a/security/advancedtls/examples/go.mod b/security/advancedtls/examples/go.mod index c7dbca3f301b..43a16752b17c 100644 --- a/security/advancedtls/examples/go.mod +++ b/security/advancedtls/examples/go.mod @@ -3,18 +3,18 @@ module google.golang.org/grpc/security/advancedtls/examples go 1.22 require ( - google.golang.org/grpc v1.67.1 - google.golang.org/grpc/examples v0.0.0-20241017035653-830135e6c5a3 + google.golang.org/grpc v1.68.1 + google.golang.org/grpc/examples v0.0.0-20241205092301-d7286fbc3f8f google.golang.org/grpc/security/advancedtls v1.0.0 ) require ( - golang.org/x/crypto v0.28.0 // indirect - golang.org/x/net v0.30.0 // indirect - golang.org/x/sys v0.26.0 // indirect - golang.org/x/text v0.19.0 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/protobuf v1.35.1 // indirect + golang.org/x/crypto v0.30.0 // indirect + golang.org/x/net v0.32.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/text v0.21.0 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a // indirect + google.golang.org/protobuf v1.35.2 // indirect ) replace google.golang.org/grpc => ../../.. diff --git a/security/advancedtls/examples/go.sum b/security/advancedtls/examples/go.sum index 2192e85919d7..ec9f8e8372e7 100644 --- a/security/advancedtls/examples/go.sum +++ b/security/advancedtls/examples/go.sum @@ -8,25 +8,25 @@ github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= -go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= -go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= -go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= -go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= -go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= -go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= -go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= -go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= -go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= -golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= -golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= +golang.org/x/crypto v0.30.0 h1:RwoQn3GkWiMkzlX562cLB7OxWvjH1L8xutO2WoJcRoY= +golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a h1:hgh8P4EuoxpsuKMXX/To36nOFD7vixReXgn8lPGnt+o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= diff --git a/security/advancedtls/go.mod b/security/advancedtls/go.mod index a010a986ef3f..5cc89cbb74d7 100644 --- a/security/advancedtls/go.mod +++ b/security/advancedtls/go.mod @@ -4,17 +4,17 @@ go 1.22 require ( github.com/google/go-cmp v0.6.0 - golang.org/x/crypto v0.28.0 - google.golang.org/grpc v1.67.1 + golang.org/x/crypto v0.30.0 + google.golang.org/grpc v1.68.1 google.golang.org/grpc/examples v0.0.0-20201112215255-90f1b3ee835b ) require ( - golang.org/x/net v0.30.0 // indirect - golang.org/x/sys v0.26.0 // indirect - golang.org/x/text v0.19.0 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/protobuf v1.35.1 // indirect + golang.org/x/net v0.32.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/text v0.21.0 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a // indirect + google.golang.org/protobuf v1.35.2 // indirect ) replace google.golang.org/grpc => ../../ diff --git a/security/advancedtls/go.sum b/security/advancedtls/go.sum index 2192e85919d7..ec9f8e8372e7 100644 --- a/security/advancedtls/go.sum +++ b/security/advancedtls/go.sum @@ -8,25 +8,25 @@ github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/uuid v1.6.0 h1:NIvaJDMOsjHA8n1jAhLSgzrAzy1Hgr+hNrb57e+94F0= github.com/google/uuid v1.6.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= -go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= -go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= -go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= -go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= -go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= -go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= -go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= -go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= -go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= -go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= -golang.org/x/crypto v0.28.0 h1:GBDwsMXVQi34v5CCYUm2jkJvu4cbtru2U4TN2PSyQnw= -golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= +golang.org/x/crypto v0.30.0 h1:RwoQn3GkWiMkzlX562cLB7OxWvjH1L8xutO2WoJcRoY= +golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a h1:hgh8P4EuoxpsuKMXX/To36nOFD7vixReXgn8lPGnt+o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= diff --git a/stats/opencensus/go.mod b/stats/opencensus/go.mod index 919333509fc3..1464e824133b 100644 --- a/stats/opencensus/go.mod +++ b/stats/opencensus/go.mod @@ -5,16 +5,16 @@ go 1.22 require ( github.com/google/go-cmp v0.6.0 go.opencensus.io v0.24.0 - google.golang.org/grpc v1.67.1 + google.golang.org/grpc v1.68.1 ) require ( - github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect - golang.org/x/net v0.30.0 // indirect - golang.org/x/sys v0.26.0 // indirect - golang.org/x/text v0.19.0 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 // indirect - google.golang.org/protobuf v1.35.1 // indirect + github.com/golang/groupcache v0.0.0-20241129210726-2c02b8208cf8 // indirect + golang.org/x/net v0.32.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/text v0.21.0 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a // indirect + google.golang.org/protobuf v1.35.2 // indirect ) replace google.golang.org/grpc => ../.. diff --git a/stats/opencensus/go.sum b/stats/opencensus/go.sum index 2e88e8bf1877..df09d09f399e 100644 --- a/stats/opencensus/go.sum +++ b/stats/opencensus/go.sum @@ -1,5 +1,5 @@ cel.dev/expr v0.15.0/go.mod h1:TRSuuV7DlVCE/uwv5QbAiW/v8l5O8C4eEPHeu7gf7Sg= -cel.dev/expr v0.16.2/go.mod h1:gXngZQMkWJoSbE8mOzehJlXQyubn/Vg0vR9/F3W7iw8= +cel.dev/expr v0.19.0/go.mod h1:MrpN08Q+lEBs+bGYdLxxHkZoUSsCp0nSKTs0nTymJgw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.38.0/go.mod h1:990N+gfupTy94rShfmMCWGDn0LpTmnzTp2qbd1dvSRU= cloud.google.com/go v0.44.1/go.mod h1:iSa0KzasP4Uvy3f1mN/7PiObzGgflwredwwASm/v6AU= @@ -213,7 +213,6 @@ cloud.google.com/go/compute/metadata v0.2.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1h cloud.google.com/go/compute/metadata v0.2.1/go.mod h1:jgHgmJd2RKBGzXqF5LR2EZMGxBkeanZ9wwa75XHJgOM= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/compute/metadata v0.3.0/go.mod h1:zFmK7XCadkQkj6TtorcaGlCW1hT1fIilQDwofLpJ20k= -cloud.google.com/go/compute/metadata v0.5.0/go.mod h1:aHnloV2TPI38yx4s9+wAZhHykWvVCfu7hQbF+9CWoiY= cloud.google.com/go/compute/metadata v0.5.2/go.mod h1:C66sj2AluDcIqakBq/M8lw8/ybHgOZqin2obFxa/E5k= cloud.google.com/go/contactcenterinsights v1.3.0/go.mod h1:Eu2oemoePuEFc/xKFPjbTuPSj0fYJcPls9TFlPNnHHY= cloud.google.com/go/contactcenterinsights v1.4.0/go.mod h1:L2YzkGbPsv+vMQMCADxJoT9YiTTnSEd6fEvCeHTYVck= @@ -757,7 +756,7 @@ gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zum git.sr.ht/~sbinet/gg v0.3.1/go.mod h1:KGYtlADtqsqANL9ueOFkWymvzUvLMQllU5Ixo+8v3pc= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.24.2/go.mod h1:itPGVDKf9cC/ov4MdvJ2QZ0khw4bfoo9jzwTJlaxy2k= +github.com/GoogleCloudPlatform/opentelemetry-operations-go/detectors/gcp v1.25.0/go.mod h1:obipzmGjfSjam60XLwGfqUkJsfiheAl+TUjG+4yzyPM= github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c/go.mod h1:X0CRv0ky0k6m906ixxpzmDRLvX58TFUKS2eePweuyxk= github.com/ajstarks/deck v0.0.0-20200831202436-30c9fc6549a9/go.mod h1:JynElWSGnm/4RlzPXRlREEwqTHAN3T56Bv2ITsFT3gY= github.com/ajstarks/deck/generate v0.0.0-20210309230005-c3f852c02e19/go.mod h1:T13YZdzov6OU0A1+RfKZiZN9ca6VeKdBdyDV+BY97Tk= @@ -769,7 +768,6 @@ github.com/apache/arrow/go/v10 v10.0.1/go.mod h1:YvhnlEePVnBS4+0z3fhPfUy7W1Ikj0I github.com/apache/arrow/go/v11 v11.0.0/go.mod h1:Eg5OsL5H+e299f7u5ssuXsuHQVEGC4xei5aX110hRiI= github.com/apache/arrow/go/v12 v12.0.0/go.mod h1:d+tV/eHZZ7Dz7RPrFKtPK02tpr+c9/PEd/zm8mDS9Vg= github.com/apache/thrift v0.16.0/go.mod h1:PHK3hniurgQaNMZYaCLEqXKsYK8upmhPbmdP2FXSqgU= -github.com/bazelbuild/rules_go v0.49.0/go.mod h1:Dhcz716Kqg1RHNWos+N6MlXNkjNP2EwZQ0LukRKJfMs= github.com/boombuler/barcode v1.0.0/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/boombuler/barcode v1.0.1/go.mod h1:paBWMcWSl3LHKBqUq+rly7CNSldXjb2rDl3JlRe0mD8= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= @@ -828,17 +826,17 @@ github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre github.com/go-pdf/fpdf v0.5.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/go-pdf/fpdf v0.6.0/go.mod h1:HzcnA+A23uwogo0tp9yU+l3V+KXhiESpt1PMayhOh5M= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= -github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/freetype v0.0.0-20170609003504-e2365dfdc4a0/go.mod h1:E/TSTwGwJL78qG/PmXZO1EjYhfJinVAhrmmHX6Z8B9k= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= github.com/golang/glog v1.1.0/go.mod h1:pfYeQZ3JWZoXTV5sFc986z3HTpwQs9At6P4ImfuP3NQ= -github.com/golang/glog v1.2.2/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= +github.com/golang/glog v1.2.3/go.mod h1:6AhwSGph0fcJtXVM/PEHPqZlFeoLxhs7/t5UDAwmO+w= github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= -github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da h1:oI5xCqsCo564l8iNU+DwB5epxmsaqB+rhGL0m5jtYqE= github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= +github.com/golang/groupcache v0.0.0-20241129210726-2c02b8208cf8 h1:f+oWsMOmNPc8JmEHVZIycC7hBoQxHH9pNKQORJNozsQ= +github.com/golang/groupcache v0.0.0-20241129210726-2c02b8208cf8/go.mod h1:wcDNUvekVysuuOpQKo3191zZyTpiI6se1N1ULghS0sw= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= @@ -847,7 +845,6 @@ github.com/golang/mock v1.4.3/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/mock v1.5.0/go.mod h1:CWnOUgYIOo4TcNZ0wHX3YZCqsaM1I1Jvs6v3mP3KVu8= github.com/golang/mock v1.6.0/go.mod h1:p6yTPP+5HYm5mzsMV8JkE6ZKdX+/wYM6Hr+LicevLPs= -github.com/golang/mock v1.7.0-rc.1/go.mod h1:s42URUywIqd+OcERslBJvOjepvNymP31m3q8d/GkuRs= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -954,7 +951,6 @@ github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/X github.com/jung-kurt/gofpdf v1.0.0/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/jung-kurt/gofpdf v1.0.3-0.20190309125859-24315acbbda5/go.mod h1:7Id9E/uU8ce6rXgefFLlgrJj/GYY22cpxn+r32jIOes= github.com/kballard/go-shellquote v0.0.0-20180428030007-95032a82bc51/go.mod h1:CzGEWj7cYgsdH8dAjBGEr58BoE7ScuLd+fwFZ44+/x8= -github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j0HLHbNSE= github.com/klauspost/compress v1.15.9/go.mod h1:PhcZ0MbTNciWF3rruxRgKxI5NkcHHrHUDtV4Yw2GlzU= @@ -1039,17 +1035,17 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/detectors/gcp v1.31.0/go.mod h1:tzQL6E1l+iV44YFTkcAeNQqzXUiekSYP9jjJjXwEd00= -go.opentelemetry.io/otel v1.31.0 h1:NsJcKPIW0D0H3NgzPDHmo0WW6SptzPdqg/L1zsIm2hY= -go.opentelemetry.io/otel v1.31.0/go.mod h1:O0C14Yl9FgkjqcCZAsE053C13OaddMYr/hz6clDkEJE= -go.opentelemetry.io/otel/metric v1.31.0 h1:FSErL0ATQAmYHUIzSezZibnyVlft1ybhy4ozRPcF2fE= -go.opentelemetry.io/otel/metric v1.31.0/go.mod h1:C3dEloVbLuYoX41KpmAhOqNriGbA+qqH6PQ5E5mUfnY= -go.opentelemetry.io/otel/sdk v1.31.0 h1:xLY3abVHYZ5HSfOg3l2E5LUj2Cwva5Y7yGxnSW9H5Gk= -go.opentelemetry.io/otel/sdk v1.31.0/go.mod h1:TfRbMdhvxIIr/B2N2LQW2S5v9m3gOQ/08KsbbO5BPT0= -go.opentelemetry.io/otel/sdk/metric v1.31.0 h1:i9hxxLJF/9kkvfHppyLL55aW7iIJz4JjxTeYusH7zMc= -go.opentelemetry.io/otel/sdk/metric v1.31.0/go.mod h1:CRInTMVvNhUKgSAMbKyTMxqOBC0zgyxzW55lZzX43Y8= -go.opentelemetry.io/otel/trace v1.31.0 h1:ffjsj1aRouKewfr85U2aGagJ46+MvodynlQ1HYdmJys= -go.opentelemetry.io/otel/trace v1.31.0/go.mod h1:TXZkRk7SM2ZQLtR6eoAWQFIHPvzQ06FJAsO1tJg480A= +go.opentelemetry.io/contrib/detectors/gcp v1.32.0/go.mod h1:TVqo0Sda4Cv8gCIixd7LuLwW4EylumVWfhjZJjDD4DU= +go.opentelemetry.io/otel v1.32.0 h1:WnBN+Xjcteh0zdk01SVqV55d/m62NJLJdIyb4y/WO5U= +go.opentelemetry.io/otel v1.32.0/go.mod h1:00DCVSB0RQcnzlwyTfqtxSm+DRr9hpYrHjNGiBHVQIg= +go.opentelemetry.io/otel/metric v1.32.0 h1:xV2umtmNcThh2/a/aCP+h64Xx5wsj8qqnkYZktzNa0M= +go.opentelemetry.io/otel/metric v1.32.0/go.mod h1:jH7CIbbK6SH2V2wE16W05BHCtIDzauciCRLoc/SyMv8= +go.opentelemetry.io/otel/sdk v1.32.0 h1:RNxepc9vK59A8XsgZQouW8ue8Gkb4jpWtJm9ge5lEG4= +go.opentelemetry.io/otel/sdk v1.32.0/go.mod h1:LqgegDBjKMmb2GC6/PrTnteJG39I8/vJCAP9LlJXEjU= +go.opentelemetry.io/otel/sdk/metric v1.32.0 h1:rZvFnvmvawYb0alrYkjraqJq0Z4ZUJAiyYCU9snn1CU= +go.opentelemetry.io/otel/sdk/metric v1.32.0/go.mod h1:PWeZlq0zt9YkYAp3gjKZ0eicRYvOh1Gd+X99x6GHpCQ= +go.opentelemetry.io/otel/trace v1.32.0 h1:WIC9mYrXf8TmY/EXuULKc8hR17vE+Hjv2cssQDe03fM= +go.opentelemetry.io/otel/trace v1.32.0/go.mod h1:+i4rkvCraA+tG6AzwloGaCtkx53Fa+L+V8e9a7YvhT8= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= @@ -1071,14 +1067,13 @@ golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU golang.org/x/crypto v0.9.0/go.mod h1:yrmDGqONDYtNj3tH8X9dzUun2m2lzPa9ngI6/RUPGR0= golang.org/x/crypto v0.12.0/go.mod h1:NF0Gs7EO5K4qLn+Ylc+fih8BSTeIjAP05siRnAh98yw= golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= -golang.org/x/crypto v0.15.0/go.mod h1:4ChreQoLWfG3xLDer1WdlH5NdlQ3+mwnQq1YTKY+72g= golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/crypto v0.21.0/go.mod h1:0BP7YvVV9gBbVKyeTG0Gyn+gZm94bibOW5BjDEYAOMs= golang.org/x/crypto v0.23.0/go.mod h1:CKFgDieR+mRhux2Lsu27y0fO304Db0wZe70UKqHu0v8= golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= -golang.org/x/crypto v0.28.0/go.mod h1:rmgy+3RHxRZMyY0jjAJShp2zgEdOqj2AO7U0pYmeQ7U= +golang.org/x/crypto v0.30.0/go.mod h1:kDsLvtWBEx7MV9tJOj9bnXsPbxwJQ6csT/x4KIN4Ssk= golang.org/x/exp v0.0.0-20180321215751-8460e604b9de/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20180807140117-3d87b88a115f/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= @@ -1137,7 +1132,6 @@ golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.12.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= -golang.org/x/mod v0.14.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.15.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/mod v0.17.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1200,15 +1194,14 @@ golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= golang.org/x/net v0.10.0/go.mod h1:0qNGK6F8kojg2nk9dLZ2mShWaEBan6FAoqfSigmmuDg= golang.org/x/net v0.14.0/go.mod h1:PpSgVXXLK0OxS0F31C1/tv6XNguvCrnXIDrFMspZIUI= golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= -golang.org/x/net v0.18.0/go.mod h1:/czyP5RqHAH4odGYxBJ1qz0+CE5WZ+2j1YgoEo8F2jQ= golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= golang.org/x/net v0.21.0/go.mod h1:bIjVDfnllIU7BJ2DNgfnXvpSvtn8VRwhlsaeUTyUS44= golang.org/x/net v0.22.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= golang.org/x/net v0.25.0/go.mod h1:JkAGAh7GEvH74S6FOH42FLoXpXbE/aqXSrIQjXgsiwM= golang.org/x/net v0.26.0/go.mod h1:5YKkiSynbBIh3p6iOc/vibscux0x38BZDkn8sCUPxHE= golang.org/x/net v0.28.0/go.mod h1:yqtgsTWOOnlGLG9GFRrK3++bGOUEkNBoHZc8MEDWPNg= -golang.org/x/net v0.30.0 h1:AcW1SDZMkb8IpzCdQUaIq2sP4sZ4zw+55h6ynffypl4= -golang.org/x/net v0.30.0/go.mod h1:2wGyMJ5iFasEhkwi13ChkO/t1ECNC4X4eBKkVFyYFlU= +golang.org/x/net v0.32.0 h1:ZqPmj8Kzc+Y6e0+skZsuACbx+wzMgo5MQsJh9Qd6aYI= +golang.org/x/net v0.32.0/go.mod h1:CwU0IoeOlnQQWJ6ioyFrfRuomB8GKF6KbYXZVyeXNfs= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1237,7 +1230,7 @@ golang.org/x/oauth2 v0.5.0/go.mod h1:9/XBHVqLaWO3/BRHs5jbpYCnOZVjj5V0ndyaAM7KB4I golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= golang.org/x/oauth2 v0.7.0/go.mod h1:hPLQkd9LyjfXTiRohC/41GhcFqxisoUQ99sCUOHO9x4= golang.org/x/oauth2 v0.8.0/go.mod h1:yr7u4HXZRm1R1kBWqr/xKNqewf0plRYoB7sla+BCIXE= -golang.org/x/oauth2 v0.23.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= +golang.org/x/oauth2 v0.24.0/go.mod h1:XYTD2NtWslqkgxebSiOHnXEap4TF09sJSc7H1sXbhtI= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1255,10 +1248,10 @@ golang.org/x/sync v0.0.0-20220929204114-8fcdb60fdcc0/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.1.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.2.0/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= -golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1338,19 +1331,17 @@ golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.14.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.18.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.20.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.22.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.23.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/sys v0.25.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.27.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/telemetry v0.0.0-20240228155512-f48c80bd79b2/go.mod h1:TeRTkGYfJXctD9OcfyVLyj2J3IxLnKwHJR8f4D8a3YE= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= @@ -1363,14 +1354,13 @@ golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= golang.org/x/term v0.8.0/go.mod h1:xPskH00ivmX89bAKVGSKKtLOWNx2+17Eiy94tnKShWo= golang.org/x/term v0.11.0/go.mod h1:zC9APTIj3jG3FdV/Ons+XE1riIZXG4aZ4GTHiPZJPIU= golang.org/x/term v0.12.0/go.mod h1:owVbMEjm3cBLCHdkQu9b1opXd4ETQWc3BhuQGKgXgvU= -golang.org/x/term v0.14.0/go.mod h1:TySc+nGkYR6qt8km8wUhuFRTVSMIX3XPR58y2lC8vww= golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/term v0.18.0/go.mod h1:ILwASektA3OnRv7amZ1xhE/KTR+u50pbXfZ03+6Nx58= golang.org/x/term v0.20.0/go.mod h1:8UkIAJTvZgivsXaD6/pH6U9ecQzZ45awqEOzuCvwpFY= golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= -golang.org/x/term v0.25.0/go.mod h1:RPyXicDX+6vLxogjjRxjgD2TKtmAO6NZBsBRfrOLu7M= +golang.org/x/term v0.27.0/go.mod h1:iMsnZpn0cago0GOrHO2+Y7u7JPn5AylBrcoWkElMTSM= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1392,8 +1382,8 @@ golang.org/x/text v0.14.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.15.0/go.mod h1:18ZOQIKpY8NJVqYksKHtTdi31H5itFRjB5/qKTNYzSU= golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1439,7 +1429,6 @@ golang.org/x/tools v0.0.0-20200501065659-ab2804fb9c9d/go.mod h1:EkVYQZoAsY45+roY golang.org/x/tools v0.0.0-20200512131952-2bc93b1c0c88/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200515010526-7d3b6ebf133d/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200618134242-20370b0cb4b2/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/tools v0.0.0-20200729194436-6467de6f59a7/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200804011535-6c149bb5ef0d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= golang.org/x/tools v0.0.0-20200825202427-b303f430e36d/go.mod h1:njjCfa9FT2d7l9Bc6FUM5FLjQPp3cFF28FI3qnDFljA= @@ -1449,7 +1438,6 @@ golang.org/x/tools v0.0.0-20201124115921-2c860bdd6e78/go.mod h1:emZCQorbCU4vsT4f golang.org/x/tools v0.0.0-20201201161351-ac6f37ff4c2a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20201208233053-a543418bbed2/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210105154028-b0ab187a4818/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.0.0-20210108195828-e2f9c7f1fc8e/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/tools v0.1.1/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= @@ -1457,7 +1445,6 @@ golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.8/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.9/go.mod h1:nABZi5QlRsZVlzPpHl034qft6wpY4eDcsTt5AaioBiU= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= @@ -1465,7 +1452,6 @@ golang.org/x/tools v0.6.0/go.mod h1:Xwgl3UAJ/d3gWutnCtw505GrjyAbvKui8lOU390QaIU= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/tools v0.9.1/go.mod h1:owI94Op576fPu3cIGQeHs3joujW/2Oc6MtlxbF5dfNc= golang.org/x/tools v0.13.0/go.mod h1:HvlwmtVNQAhOuCjW7xxvovg8wbNq7LwfXh/k7wXUl58= -golang.org/x/tools v0.15.0/go.mod h1:hpksKq4dtpQWS1uQ61JkdqWM3LscIS6Slf+VVkm+wQk= golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d/go.mod h1:aiJjzUbINMkxbQROHiO6hDPo2LHcIPhhQsa9DLh0yGk= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1699,8 +1685,7 @@ google.golang.org/genproto/googleapis/api v0.0.0-20230706204954-ccb25ca9f130/go. google.golang.org/genproto/googleapis/api v0.0.0-20230726155614-23370e0ffb3e/go.mod h1:rsr7RhLuwsDKL7RmgDDCUc6yaGr1iqceVb5Wv6f6YvQ= google.golang.org/genproto/googleapis/api v0.0.0-20230822172742-b8732ec3820d/go.mod h1:KjSP20unUpOx5kyQUFa7k4OJg0qeJ7DEZflGDu2p6Bk= google.golang.org/genproto/googleapis/api v0.0.0-20240528184218-531527333157/go.mod h1:99sLkeliLXfdj2J75X3Ho+rrVCaJze0uwN7zDDkjPVU= -google.golang.org/genproto/googleapis/api v0.0.0-20240826202546-f6391c0de4c7/go.mod h1:OCdP9MfskevB/rbYvHTsXTtKC+3bHWajPdoKgjcYkfo= -google.golang.org/genproto/googleapis/api v0.0.0-20241015192408-796eee8c2d53/go.mod h1:riSXTwQ4+nqmPGtobMFyW5FqVAmIs0St6VPp4Ug7CE4= +google.golang.org/genproto/googleapis/api v0.0.0-20241202173237-19429a94021a/go.mod h1:jehYqy3+AhJU9ve55aNOaSml7wUXjF9x6z2LcCfpAhY= google.golang.org/genproto/googleapis/bytestream v0.0.0-20230530153820-e85fd2cbaebc/go.mod h1:ylj+BE99M198VPbBh6A8d9n3w8fChvyLK3wwBOjXBFA= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234015-3fc162c6f38a/go.mod h1:xURIpW9ES5+/GZhnV6beoEtxQrnkRGIfP5VQG2tCBLc= google.golang.org/genproto/googleapis/rpc v0.0.0-20230525234030-28d5490b6b19/go.mod h1:66JfowdXAEgad5O9NnYcsNPLCPZJD++2L9X0PCMODrA= @@ -1714,13 +1699,11 @@ google.golang.org/genproto/googleapis/rpc v0.0.0-20230822172742-b8732ec3820d/go. google.golang.org/genproto/googleapis/rpc v0.0.0-20240318140521-94a12d6c2237/go.mod h1:WtryC6hu0hhx87FDGxWCDptyssuo68sk10vYjF+T9fY= google.golang.org/genproto/googleapis/rpc v0.0.0-20240521202816-d264139d666e/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= google.golang.org/genproto/googleapis/rpc v0.0.0-20240528184218-531527333157/go.mod h1:EfXuqaE1J41VCDicxHzUDm+8rk+7ZdXzHV0IhO/I6s0= -google.golang.org/genproto/googleapis/rpc v0.0.0-20240823204242-4ba0660f739c/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= google.golang.org/genproto/googleapis/rpc v0.0.0-20240826202546-f6391c0de4c7/go.mod h1:UqMtugtsSgubUsoxbuAoiCXvqvErP7Gf0so0mK9tHxU= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241007155032-5fefd90f89a9/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53 h1:X58yt85/IXCx0Y3ZwN6sEIKZzQtDEYaBWrDvErdXrRE= -google.golang.org/genproto/googleapis/rpc v0.0.0-20241015192408-796eee8c2d53/go.mod h1:GX3210XPVPUjJbTUbvwI8f2IpZDMZuPJWDzDuebbviI= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241118233622-e639e219e697/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a h1:hgh8P4EuoxpsuKMXX/To36nOFD7vixReXgn8lPGnt+o= +google.golang.org/genproto/googleapis/rpc v0.0.0-20241202173237-19429a94021a/go.mod h1:5uTbfoYQed2U9p3KIj2/Zzm02PYhndfdmML0qC3q3FU= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= -google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.3.0/go.mod h1:Dk1tviKTvMCz5tvh7t+fh94dhmQVHuCt2OzJB3CTW9Y= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= @@ -1743,8 +1726,8 @@ google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHh google.golang.org/protobuf v1.33.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.1/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= google.golang.org/protobuf v1.34.2/go.mod h1:qYOHts0dSfpeUzUFpOMr/WGzszTmLH+DiWniOlNbLDw= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= diff --git a/test/codec_perf/perf.pb.go b/test/codec_perf/perf.pb.go index dbe6cb5700ca..e13c493d5f49 100644 --- a/test/codec_perf/perf.pb.go +++ b/test/codec_perf/perf.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: -// protoc-gen-go v1.35.1 +// protoc-gen-go v1.35.2 // protoc v5.27.1 // source: test/codec_perf/perf.proto diff --git a/test/tools/go.mod b/test/tools/go.mod index a647da3f1a46..e86e8a3548e6 100644 --- a/test/tools/go.mod +++ b/test/tools/go.mod @@ -4,16 +4,16 @@ go 1.22.1 require ( github.com/client9/misspell v0.3.4 - github.com/mgechev/revive v1.4.0 - golang.org/x/tools v0.26.0 - google.golang.org/protobuf v1.35.1 + github.com/mgechev/revive v1.5.1 + golang.org/x/tools v0.28.0 + google.golang.org/protobuf v1.35.2 honnef.co/go/tools v0.5.1 ) require ( github.com/BurntSushi/toml v1.4.1-0.20240526193622-a339e1f7089c // indirect github.com/chavacava/garif v0.1.0 // indirect - github.com/fatih/color v1.17.0 // indirect + github.com/fatih/color v1.18.0 // indirect github.com/fatih/structtag v1.2.0 // indirect github.com/hashicorp/go-version v1.7.0 // indirect github.com/mattn/go-colorable v0.1.13 // indirect @@ -24,9 +24,9 @@ require ( github.com/olekukonko/tablewriter v0.0.5 // indirect github.com/rivo/uniseg v0.4.7 // indirect github.com/spf13/afero v1.11.0 // indirect - golang.org/x/exp/typeparams v0.0.0-20241009180824-f66d83c29e7c // indirect - golang.org/x/mod v0.21.0 // indirect - golang.org/x/sync v0.8.0 // indirect - golang.org/x/sys v0.26.0 // indirect - golang.org/x/text v0.19.0 // indirect + golang.org/x/exp/typeparams v0.0.0-20241204233417-43b7b7cde48d // indirect + golang.org/x/mod v0.22.0 // indirect + golang.org/x/sync v0.10.0 // indirect + golang.org/x/sys v0.28.0 // indirect + golang.org/x/text v0.21.0 // indirect ) diff --git a/test/tools/go.sum b/test/tools/go.sum index 9e262e028bd8..815fa1face72 100644 --- a/test/tools/go.sum +++ b/test/tools/go.sum @@ -7,8 +7,8 @@ github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDk github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= -github.com/fatih/color v1.17.0 h1:GlRw1BRJxkpqUCBKzKOw098ed57fEsKeNjpTe3cSjK4= -github.com/fatih/color v1.17.0/go.mod h1:YZ7TlrGPkiz6ku9fK3TLD/pl3CpsiFyu8N92HLgmosI= +github.com/fatih/color v1.18.0 h1:S8gINlzdQ840/4pfAwic/ZE0djQEH3wM94VfqLTZcOM= +github.com/fatih/color v1.18.0/go.mod h1:4FelSpRwEGDpQ12mAdzqdOukCy4u8WUtOY6lkT/6HfU= github.com/fatih/structtag v1.2.0 h1:/OdNE99OxoI/PqaW/SuSK9uxxT3f/tcSZgon/ssNSx4= github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= @@ -25,8 +25,8 @@ github.com/mattn/go-runewidth v0.0.16 h1:E5ScNMtiwvlvB5paMFdw9p4kSQzbXFikJ5SQO6T github.com/mattn/go-runewidth v0.0.16/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= github.com/mgechev/dots v0.0.0-20210922191527-e955255bf517 h1:zpIH83+oKzcpryru8ceC6BxnoG8TBrhgAvRg8obzup0= github.com/mgechev/dots v0.0.0-20210922191527-e955255bf517/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= -github.com/mgechev/revive v1.4.0 h1:+6LDNE1XKsUCkpuDOMrzjOsXqiQOZ/jPlscLyA6mMXw= -github.com/mgechev/revive v1.4.0/go.mod h1:uzGR6feiCiJi4oND58/KMt/lEnR5vmjzRYPZiR0sQRQ= +github.com/mgechev/revive v1.5.1 h1:hE+QPeq0/wIzJwOphdVyUJ82njdd8Khp4fUIHGZHW3M= +github.com/mgechev/revive v1.5.1/go.mod h1:lC9AhkJIBs5zwx8wkudyHrU+IJkrEKmpCmGMnIJPk4o= github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= @@ -46,22 +46,22 @@ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/stretchr/testify v1.9.0 h1:HtqpIVDClZ4nwg75+f6Lvsy/wHu+3BoSGCbBAcpTsTg= github.com/stretchr/testify v1.9.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= -golang.org/x/exp/typeparams v0.0.0-20241009180824-f66d83c29e7c h1:F/15/6p7LyGUSoP0GE5CB/U9+TNEER1foNOP5sWLLnI= -golang.org/x/exp/typeparams v0.0.0-20241009180824-f66d83c29e7c/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= -golang.org/x/mod v0.21.0 h1:vvrHzRwRfVKSiLrG+d4FMl/Qi4ukBCE6kZlTUkDYRT0= -golang.org/x/mod v0.21.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= -golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= -golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/exp/typeparams v0.0.0-20241204233417-43b7b7cde48d h1:WQHXGzzI2u+AAlupPRpQbdG4WVvVZ7d2dg/CkpEu1hI= +golang.org/x/exp/typeparams v0.0.0-20241204233417-43b7b7cde48d/go.mod h1:AbB0pIl9nAr9wVwH+Z2ZpaocVmF5I4GyWCDIsVjR0bk= +golang.org/x/mod v0.22.0 h1:D4nJWe9zXqHOmWqj4VMOJhvzj7bEZg4wEYa759z1pH4= +golang.org/x/mod v0.22.0/go.mod h1:6SkKJ3Xj0I0BrPOZoBy3bdMptDDU9oJrpohJ3eWZ1fY= +golang.org/x/sync v0.10.0 h1:3NQrjDixjgGwUOCaF8w2+VYHv0Ve/vGYSbdkTa98gmQ= +golang.org/x/sync v0.10.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.26.0 h1:KHjCJyddX0LoSTb3J+vWpupP9p0oznkqVk/IfjymZbo= -golang.org/x/sys v0.26.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/text v0.19.0 h1:kTxAhCbGbxhK0IwgSKiMO5awPoDQ0RpfiVYBfK860YM= -golang.org/x/text v0.19.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= -golang.org/x/tools v0.26.0 h1:v/60pFQmzmT9ExmjDv2gGIfi3OqfKoEP6I5+umXlbnQ= -golang.org/x/tools v0.26.0/go.mod h1:TPVVj70c7JJ3WCazhD8OdXcZg/og+b9+tH/KxylGwH0= -google.golang.org/protobuf v1.35.1 h1:m3LfL6/Ca+fqnjnlqQXNpFPABW1UD7mjh8KO2mKFytA= -google.golang.org/protobuf v1.35.1/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +golang.org/x/sys v0.28.0 h1:Fksou7UEQUWlKvIdsqzJmUmCX3cZuD2+P3XyyzwMhlA= +golang.org/x/sys v0.28.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/text v0.21.0 h1:zyQAAkrwaneQ066sspRyJaG9VNi/YJ1NfzcGB3hZ/qo= +golang.org/x/text v0.21.0/go.mod h1:4IBbMaMmOPCJ8SecivzSH54+73PCFmPWxNTLm+vZkEQ= +golang.org/x/tools v0.28.0 h1:WuB6qZ4RPCQo5aP3WdKZS7i595EdWqWR8vqJTlwTVK8= +golang.org/x/tools v0.28.0/go.mod h1:dcIOrVd3mfQKTgrDVQHqCPMWy6lnhfhtX3hLXYVLfRw= +google.golang.org/protobuf v1.35.2 h1:8Ar7bF+apOIoThw1EdZl0p1oWvMqTHmpA2fRTyZO8io= +google.golang.org/protobuf v1.35.2/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= From f53724da14727f0cdf09b3ff0b66cf21ab4a4e47 Mon Sep 17 00:00:00 2001 From: Doug Fawley Date: Thu, 5 Dec 2024 15:16:45 -0800 Subject: [PATCH 22/57] serviceconfig: Return errors instead of skipping invalid retry policy config (#7905) --- service_config.go | 17 ++-- service_config_test.go | 226 +++++++++++++++++++++++++++++++---------- 2 files changed, 185 insertions(+), 58 deletions(-) diff --git a/service_config.go b/service_config.go index 7e83027d1994..8d451e07c7cc 100644 --- a/service_config.go +++ b/service_config.go @@ -268,18 +268,21 @@ func parseServiceConfig(js string, maxAttempts int) *serviceconfig.ParseResult { return &serviceconfig.ParseResult{Config: &sc} } +func isValidRetryPolicy(jrp *jsonRetryPolicy) bool { + return jrp.MaxAttempts > 1 && + jrp.InitialBackoff > 0 && + jrp.MaxBackoff > 0 && + jrp.BackoffMultiplier > 0 && + len(jrp.RetryableStatusCodes) > 0 +} + func convertRetryPolicy(jrp *jsonRetryPolicy, maxAttempts int) (p *internalserviceconfig.RetryPolicy, err error) { if jrp == nil { return nil, nil } - if jrp.MaxAttempts <= 1 || - jrp.InitialBackoff <= 0 || - jrp.MaxBackoff <= 0 || - jrp.BackoffMultiplier <= 0 || - len(jrp.RetryableStatusCodes) == 0 { - logger.Warningf("grpc: ignoring retry policy %v due to illegal configuration", jrp) - return nil, nil + if !isValidRetryPolicy(jrp) { + return nil, fmt.Errorf("invalid retry policy (%+v): ", jrp) } if jrp.MaxAttempts < maxAttempts { diff --git a/service_config_test.go b/service_config_test.go index 3d671c3e9a3c..5dc4e6d98bb8 100644 --- a/service_config_test.go +++ b/service_config_test.go @@ -26,11 +26,15 @@ import ( "time" "google.golang.org/grpc/balancer" + "google.golang.org/grpc/codes" "google.golang.org/grpc/internal/balancer/gracefulswitch" "google.golang.org/grpc/serviceconfig" + + internalserviceconfig "google.golang.org/grpc/internal/serviceconfig" ) type parseTestCase struct { + name string scjs string wantSC *ServiceConfig wantErr bool @@ -59,7 +63,11 @@ func lbConfigFor(t *testing.T, name string, cfg serviceconfig.LoadBalancingConfi func runParseTests(t *testing.T, testCases []parseTestCase) { t.Helper() for i, c := range testCases { - t.Run(fmt.Sprint(i), func(t *testing.T) { + name := c.name + if name == "" { + name = fmt.Sprint(i) + } + t.Run(name, func(t *testing.T) { scpr := parseServiceConfig(c.scjs, defaultMaxCallAttempts) var sc *ServiceConfig sc, _ = scpr.Config.(*ServiceConfig) @@ -104,14 +112,14 @@ func init() { func (s) TestParseLBConfig(t *testing.T) { testcases := []parseTestCase{ { - `{ + scjs: `{ "loadBalancingConfig": [{"pbb": { "foo": "hi" } }] }`, - &ServiceConfig{ + wantSC: &ServiceConfig{ Methods: make(map[string]MethodConfig), lbConfig: lbConfigFor(t, "pbb", pbbData{Foo: "hi"}), }, - false, + wantErr: false, }, } runParseTests(t, testcases) @@ -137,7 +145,7 @@ func (s) TestParseNoLBConfigSupported(t *testing.T) { func (s) TestParseLoadBalancer(t *testing.T) { testcases := []parseTestCase{ { - `{ + scjs: `{ "loadBalancingPolicy": "round_robin", "methodConfig": [ { @@ -151,7 +159,7 @@ func (s) TestParseLoadBalancer(t *testing.T) { } ] }`, - &ServiceConfig{ + wantSC: &ServiceConfig{ Methods: map[string]MethodConfig{ "/foo/Bar": { WaitForReady: newBool(true), @@ -159,10 +167,10 @@ func (s) TestParseLoadBalancer(t *testing.T) { }, lbConfig: lbConfigFor(t, "round_robin", nil), }, - false, + wantErr: false, }, { - `{ + scjs: `{ "loadBalancingPolicy": 1, "methodConfig": [ { @@ -176,8 +184,7 @@ func (s) TestParseLoadBalancer(t *testing.T) { } ] }`, - nil, - true, + wantErr: true, }, } runParseTests(t, testcases) @@ -186,7 +193,7 @@ func (s) TestParseLoadBalancer(t *testing.T) { func (s) TestParseWaitForReady(t *testing.T) { testcases := []parseTestCase{ { - `{ + scjs: `{ "methodConfig": [ { "name": [ @@ -199,7 +206,7 @@ func (s) TestParseWaitForReady(t *testing.T) { } ] }`, - &ServiceConfig{ + wantSC: &ServiceConfig{ Methods: map[string]MethodConfig{ "/foo/Bar": { WaitForReady: newBool(true), @@ -207,10 +214,9 @@ func (s) TestParseWaitForReady(t *testing.T) { }, lbConfig: lbConfigFor(t, "", nil), }, - false, }, { - `{ + scjs: `{ "methodConfig": [ { "name": [ @@ -223,7 +229,7 @@ func (s) TestParseWaitForReady(t *testing.T) { } ] }`, - &ServiceConfig{ + wantSC: &ServiceConfig{ Methods: map[string]MethodConfig{ "/foo/Bar": { WaitForReady: newBool(false), @@ -231,10 +237,9 @@ func (s) TestParseWaitForReady(t *testing.T) { }, lbConfig: lbConfigFor(t, "", nil), }, - false, }, { - `{ + scjs: `{ "methodConfig": [ { "name": [ @@ -256,8 +261,7 @@ func (s) TestParseWaitForReady(t *testing.T) { } ] }`, - nil, - true, + wantErr: true, }, } @@ -267,7 +271,7 @@ func (s) TestParseWaitForReady(t *testing.T) { func (s) TestParseTimeOut(t *testing.T) { testcases := []parseTestCase{ { - `{ + scjs: `{ "methodConfig": [ { "name": [ @@ -280,7 +284,7 @@ func (s) TestParseTimeOut(t *testing.T) { } ] }`, - &ServiceConfig{ + wantSC: &ServiceConfig{ Methods: map[string]MethodConfig{ "/foo/Bar": { Timeout: newDuration(time.Second), @@ -288,10 +292,9 @@ func (s) TestParseTimeOut(t *testing.T) { }, lbConfig: lbConfigFor(t, "", nil), }, - false, }, { - `{ + scjs: `{ "methodConfig": [ { "name": [ @@ -304,11 +307,10 @@ func (s) TestParseTimeOut(t *testing.T) { } ] }`, - nil, - true, + wantErr: true, }, { - `{ + scjs: `{ "methodConfig": [ { "name": [ @@ -330,8 +332,7 @@ func (s) TestParseTimeOut(t *testing.T) { } ] }`, - nil, - true, + wantErr: true, }, } @@ -341,7 +342,7 @@ func (s) TestParseTimeOut(t *testing.T) { func (s) TestParseMsgSize(t *testing.T) { testcases := []parseTestCase{ { - `{ + scjs: `{ "methodConfig": [ { "name": [ @@ -355,7 +356,7 @@ func (s) TestParseMsgSize(t *testing.T) { } ] }`, - &ServiceConfig{ + wantSC: &ServiceConfig{ Methods: map[string]MethodConfig{ "/foo/Bar": { MaxReqSize: newInt(1024), @@ -364,10 +365,9 @@ func (s) TestParseMsgSize(t *testing.T) { }, lbConfig: lbConfigFor(t, "", nil), }, - false, }, { - `{ + scjs: `{ "methodConfig": [ { "name": [ @@ -391,8 +391,7 @@ func (s) TestParseMsgSize(t *testing.T) { } ] }`, - nil, - true, + wantErr: true, }, } @@ -408,54 +407,49 @@ func (s) TestParseDefaultMethodConfig(t *testing.T) { runParseTests(t, []parseTestCase{ { - `{ + scjs: `{ "methodConfig": [{ "name": [{}], "waitForReady": true }] }`, - dc, - false, + wantSC: dc, }, { - `{ + scjs: `{ "methodConfig": [{ "name": [{"service": null}], "waitForReady": true }] }`, - dc, - false, + wantSC: dc, }, { - `{ + scjs: `{ "methodConfig": [{ "name": [{"service": ""}], "waitForReady": true }] }`, - dc, - false, + wantSC: dc, }, { - `{ + scjs: `{ "methodConfig": [{ "name": [{"method": "Bar"}], "waitForReady": true }] }`, - nil, - true, + wantErr: true, }, { - `{ + scjs: `{ "methodConfig": [{ "name": [{"service": "", "method": "Bar"}], "waitForReady": true }] }`, - nil, - true, + wantErr: true, }, }) } @@ -463,7 +457,7 @@ func (s) TestParseDefaultMethodConfig(t *testing.T) { func (s) TestParseMethodConfigDuplicatedName(t *testing.T) { runParseTests(t, []parseTestCase{ { - `{ + scjs: `{ "methodConfig": [{ "name": [ {"service": "foo"}, @@ -471,7 +465,137 @@ func (s) TestParseMethodConfigDuplicatedName(t *testing.T) { ], "waitForReady": true }] -}`, nil, true, +}`, + wantErr: true, + }, + }) +} + +func (s) TestParseRetryPolicy(t *testing.T) { + runParseTests(t, []parseTestCase{ + { + name: "valid", + scjs: `{ + "methodConfig": [{ + "name": [{"service": "foo"}], + "retryPolicy": { + "maxAttempts": 2, + "initialBackoff": "2s", + "maxBackoff": "10s", + "backoffMultiplier": 2, + "retryableStatusCodes": ["UNAVAILABLE"] + } + }] + }`, + wantSC: &ServiceConfig{ + Methods: map[string]MethodConfig{ + "/foo/": { + RetryPolicy: &internalserviceconfig.RetryPolicy{ + MaxAttempts: 2, + InitialBackoff: 2 * time.Second, + MaxBackoff: 10 * time.Second, + BackoffMultiplier: 2, + RetryableStatusCodes: map[codes.Code]bool{codes.Unavailable: true}, + }, + }, + }, + lbConfig: lbConfigFor(t, "", nil), + }, + }, + { + name: "negative maxAttempts", + scjs: `{ + "methodConfig": [{ + "name": [{"service": "foo"}], + "retryPolicy": { + "maxAttempts": -1, + "initialBackoff": "2s", + "maxBackoff": "10s", + "backoffMultiplier": 2, + "retryableStatusCodes": ["UNAVAILABLE"] + } + }] + }`, + wantErr: true, + }, + { + name: "missing maxAttempts", + scjs: `{ + "methodConfig": [{ + "name": [{"service": "foo"}], + "retryPolicy": { + "initialBackoff": "2s", + "maxBackoff": "10s", + "backoffMultiplier": 2, + "retryableStatusCodes": ["UNAVAILABLE"] + } + }] + }`, + wantErr: true, + }, + { + name: "zero initialBackoff", + scjs: `{ + "methodConfig": [{ + "name": [{"service": "foo"}], + "retryPolicy": { + "maxAttempts": 2, + "initialBackoff": "0s", + "maxBackoff": "10s", + "backoffMultiplier": 2, + "retryableStatusCodes": ["UNAVAILABLE"] + } + }] + }`, + wantErr: true, + }, + { + name: "zero maxBackoff", + scjs: `{ + "methodConfig": [{ + "name": [{"service": "foo"}], + "retryPolicy": { + "maxAttempts": 2, + "initialBackoff": "2s", + "maxBackoff": "0s", + "backoffMultiplier": 2, + "retryableStatusCodes": ["UNAVAILABLE"] + } + }] + }`, + wantErr: true, + }, + { + name: "zero backoffMultiplier", + scjs: `{ + "methodConfig": [{ + "name": [{"service": "foo"}], + "retryPolicy": { + "maxAttempts": 2, + "initialBackoff": "2s", + "maxBackoff": "10s", + "backoffMultiplier": 0, + "retryableStatusCodes": ["UNAVAILABLE"] + } + }] + }`, + wantErr: true, + }, + { + name: "no retryable codes", + scjs: `{ + "methodConfig": [{ + "name": [{"service": "foo"}], + "retryPolicy": { + "maxAttempts": 2, + "initialBackoff": "2s", + "maxBackoff": "10s", + "backoffMultiplier": 2, + "retryableStatusCodes": [] + } + }] + }`, + wantErr: true, }, }) } From adad26df1826bf2fb66ad56ff32a62b98bf5cb3a Mon Sep 17 00:00:00 2001 From: Easwar Swaminathan Date: Thu, 5 Dec 2024 15:50:47 -0800 Subject: [PATCH 23/57] test/kokoro: Add psm-fallback build config (#7899) --- test/kokoro/psm-fallback.cfg | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) create mode 100644 test/kokoro/psm-fallback.cfg diff --git a/test/kokoro/psm-fallback.cfg b/test/kokoro/psm-fallback.cfg new file mode 100644 index 000000000000..92516cf3b3a5 --- /dev/null +++ b/test/kokoro/psm-fallback.cfg @@ -0,0 +1,18 @@ +# Config file for internal CI + +# Location of the continuous shell script in repository. +build_file: "grpc-go/test/kokoro/psm-interop-test-go.sh" +timeout_mins: 30 + +action { + define_artifacts { + regex: "artifacts/**/*sponge_log.xml" + regex: "artifacts/**/*.log" + strip_prefix: "artifacts" + } +} +env_vars { + key: "PSM_TEST_SUITE" + value: "fallback" +} + From 66ba4b264d26808cb7af3c86eee66e843472915e Mon Sep 17 00:00:00 2001 From: Purnesh Dixit Date: Fri, 6 Dec 2024 10:30:55 +0530 Subject: [PATCH 24/57] examples/features/gracefulstop: add example to demonstrate server graceful stop (#7865) --- examples/examples_test.sh | 3 + examples/features/gracefulstop/README.md | 45 ++++++++ examples/features/gracefulstop/client/main.go | 80 +++++++++++++ examples/features/gracefulstop/server/main.go | 105 ++++++++++++++++++ 4 files changed, 233 insertions(+) create mode 100644 examples/features/gracefulstop/README.md create mode 100644 examples/features/gracefulstop/client/main.go create mode 100644 examples/features/gracefulstop/server/main.go diff --git a/examples/examples_test.sh b/examples/examples_test.sh index ef0d34769482..28f050c0a0e8 100755 --- a/examples/examples_test.sh +++ b/examples/examples_test.sh @@ -71,6 +71,7 @@ EXAMPLES=( "features/orca" "features/retry" "features/unix_abstract" + "features/gracefulstop" ) declare -A SERVER_ARGS=( @@ -129,6 +130,7 @@ declare -A EXPECTED_SERVER_OUTPUT=( ["features/retry"]="request succeeded count: 4" ["features/unix_abstract"]="serving on @abstract-unix-socket" ["features/advancedtls"]="" + ["features/gracefulstop"]="Server stopped gracefully." ) declare -A EXPECTED_CLIENT_OUTPUT=( @@ -154,6 +156,7 @@ declare -A EXPECTED_CLIENT_OUTPUT=( ["features/retry"]="UnaryEcho reply: message:\"Try and Success\"" ["features/unix_abstract"]="calling echo.Echo/UnaryEcho to unix-abstract:abstract-unix-socket" ["features/advancedtls"]="" + ["features/gracefulstop"]="Successful unary requests processed by server and made by client are same." ) cd ./examples diff --git a/examples/features/gracefulstop/README.md b/examples/features/gracefulstop/README.md new file mode 100644 index 000000000000..b9a7eff07ec2 --- /dev/null +++ b/examples/features/gracefulstop/README.md @@ -0,0 +1,45 @@ +# Graceful Stop + +This example demonstrates how to gracefully stop a gRPC server using +`Server.GracefulStop()`. The graceful shutdown process involves two key steps: + +- Initiate `Server.GracefulStop()`. This function blocks until all currently + running RPCs have completed. This ensures that in-flight requests are + allowed to finish processing. + +- It's crucial to call `Server.Stop()` with a timeout before calling + `GracefulStop()`. This acts as a safety net, ensuring that the server + eventually shuts down even if some in-flight RPCs don't complete within a + reasonable timeframe. This prevents indefinite blocking. + +## Try it + +``` +go run server/main.go +``` + +``` +go run client/main.go +``` + +## Explanation + +The server starts with a client streaming and unary request handler. When +client streaming is started, client streaming handler signals the server to +initiate graceful stop and waits for the stream to be closed or aborted. Until +the`Server.GracefulStop()` is initiated, server will continue to accept unary +requests. Once `Server.GracefulStop()` is initiated, server will not accept +new unary requests. + +Client will start the client stream to the server and starts making unary +requests until receiving an error. Error will indicate that the server graceful +shutdown is initiated so client will stop making further unary requests and +closes the client stream. + +Server and client will keep track of number of unary requests processed on +their side. Once the client has successfully closed the stream, server returns +the total number of unary requests processed as response. The number from +stream response should be equal to the number of unary requests tracked by +client. This indicates that server has processed all in-flight requests before +shutting down. + diff --git a/examples/features/gracefulstop/client/main.go b/examples/features/gracefulstop/client/main.go new file mode 100644 index 000000000000..61f5874b3940 --- /dev/null +++ b/examples/features/gracefulstop/client/main.go @@ -0,0 +1,80 @@ +/* + * + * Copyright 2024 gRPC 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. + */ + +// Binary client demonstrates sending multiple requests to server and observe +// graceful stop. +package main + +import ( + "context" + "flag" + "fmt" + "log" + "time" + + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + pb "google.golang.org/grpc/examples/features/proto/echo" +) + +var addr = flag.String("addr", "localhost:50052", "the address to connect to") + +func main() { + flag.Parse() + + conn, err := grpc.NewClient(*addr, grpc.WithTransportCredentials(insecure.NewCredentials())) + if err != nil { + log.Fatalf("Failed to create new client: %v", err) + } + defer conn.Close() + c := pb.NewEchoClient(conn) + + ctx, cancel := context.WithTimeout(context.Background(), 20*time.Second) + defer cancel() + + // Start a client stream and keep calling the `c.UnaryEcho` until receiving + // an error. Error will indicate that server graceful stop is initiated and + // it won't accept any new requests. + stream, err := c.ClientStreamingEcho(ctx) + if err != nil { + log.Fatalf("Error starting stream: %v", err) + } + + // Keep track of successful unary requests which can be compared later to + // the successful unary requests reported by the server. + unaryRequests := 0 + for { + r, err := c.UnaryEcho(ctx, &pb.EchoRequest{Message: "Hello"}) + if err != nil { + log.Printf("Error calling `UnaryEcho`. Server graceful stop initiated: %v", err) + break + } + unaryRequests++ + time.Sleep(200 * time.Millisecond) + log.Printf(r.Message) + } + log.Printf("Successful unary requests made by client: %d", unaryRequests) + + r, err := stream.CloseAndRecv() + if err != nil { + log.Fatalf("Error closing stream: %v", err) + } + if fmt.Sprintf("%d", unaryRequests) != r.Message { + log.Fatalf("Got %s successful unary requests processed from server, want: %d", r.Message, unaryRequests) + } + log.Printf("Successful unary requests processed by server and made by client are same.") +} diff --git a/examples/features/gracefulstop/server/main.go b/examples/features/gracefulstop/server/main.go new file mode 100644 index 000000000000..7b9233f82fe3 --- /dev/null +++ b/examples/features/gracefulstop/server/main.go @@ -0,0 +1,105 @@ +/* + * + * Copyright 2024 gRPC 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. + */ + +// Binary server demonstrates how to gracefully stop a gRPC server. +package main + +import ( + "context" + "errors" + "flag" + "fmt" + "io" + "log" + "net" + "sync/atomic" + "time" + + "google.golang.org/grpc" + pb "google.golang.org/grpc/examples/features/proto/echo" +) + +var ( + port = flag.Int("port", 50052, "port number") +) + +type server struct { + pb.UnimplementedEchoServer + + unaryRequests atomic.Int32 // to track number of unary RPCs processed + streamStart chan struct{} // to signal if server streaming started +} + +// ClientStreamingEcho implements the EchoService.ClientStreamingEcho method. +// It signals the server that streaming has started and waits for the stream to +// be done or aborted. If `io.EOF` is received on stream that means client +// has successfully closed the stream using `stream.CloseAndRecv()`, so it +// returns an `EchoResponse` with the total number of unary RPCs processed +// otherwise, it returns the error indicating stream is aborted. +func (s *server) ClientStreamingEcho(stream pb.Echo_ClientStreamingEchoServer) error { + // Signal streaming start to initiate graceful stop which should wait until + // server streaming finishes. + s.streamStart <- struct{}{} + + if err := stream.RecvMsg(&pb.EchoResponse{}); err != nil { + if errors.Is(err, io.EOF) { + stream.SendAndClose(&pb.EchoResponse{Message: fmt.Sprintf("%d", s.unaryRequests.Load())}) + return nil + } + return err + } + + return nil +} + +// UnaryEcho implements the EchoService.UnaryEcho method. It increments +// `s.unaryRequests` on every call and returns it as part of `EchoResponse`. +func (s *server) UnaryEcho(_ context.Context, req *pb.EchoRequest) (*pb.EchoResponse, error) { + s.unaryRequests.Add(1) + return &pb.EchoResponse{Message: req.Message}, nil +} + +func main() { + flag.Parse() + + address := fmt.Sprintf(":%v", *port) + lis, err := net.Listen("tcp", address) + if err != nil { + log.Fatalf("failed to listen: %v", err) + } + + s := grpc.NewServer() + ss := &server{streamStart: make(chan struct{})} + pb.RegisterEchoServer(s, ss) + + go func() { + <-ss.streamStart // wait until server streaming starts + time.Sleep(1 * time.Second) + log.Println("Initiating graceful shutdown...") + timer := time.AfterFunc(10*time.Second, func() { + log.Println("Server couldn't stop gracefully in time. Doing force stop.") + s.Stop() + }) + defer timer.Stop() + s.GracefulStop() // gracefully stop server after in-flight server streaming rpc finishes + log.Println("Server stopped gracefully.") + }() + + if err := s.Serve(lis); err != nil { + log.Fatalf("failed to serve: %v", err) + } +} From 0027558c5d484e48e1d9b71b69532775562b46c7 Mon Sep 17 00:00:00 2001 From: jovial <41611371+JovialYip@users.noreply.github.com> Date: Tue, 10 Dec 2024 17:53:27 +0800 Subject: [PATCH 25/57] internal/transport: replace integer status codes with http constants (#7910) --- internal/transport/http2_server.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/internal/transport/http2_server.go b/internal/transport/http2_server.go index 0055fddd7ecf..997b0a59b586 100644 --- a/internal/transport/http2_server.go +++ b/internal/transport/http2_server.go @@ -564,7 +564,7 @@ func (t *http2Server) operateHeaders(ctx context.Context, frame *http2.MetaHeade t.logger.Infof("Aborting the stream early: %v", errMsg) } t.controlBuf.put(&earlyAbortStream{ - httpStatus: 405, + httpStatus: http.StatusMethodNotAllowed, streamID: streamID, contentSubtype: s.contentSubtype, status: status.New(codes.Internal, errMsg), @@ -585,7 +585,7 @@ func (t *http2Server) operateHeaders(ctx context.Context, frame *http2.MetaHeade stat = status.New(codes.PermissionDenied, err.Error()) } t.controlBuf.put(&earlyAbortStream{ - httpStatus: 200, + httpStatus: http.StatusOK, streamID: s.id, contentSubtype: s.contentSubtype, status: stat, From b1f70ce0555fffe2a384969eacf8beaa73454695 Mon Sep 17 00:00:00 2001 From: janardhanvissa <47281167+janardhanvissa@users.noreply.github.com> Date: Tue, 10 Dec 2024 16:18:57 +0530 Subject: [PATCH 26/57] test: replace grpc.Dial with grpc.NewClient --- test/balancer_switching_test.go | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/test/balancer_switching_test.go b/test/balancer_switching_test.go index e5da19d30d0f..2e46fcdb4a76 100644 --- a/test/balancer_switching_test.go +++ b/test/balancer_switching_test.go @@ -128,22 +128,24 @@ func (s) TestBalancerSwitch_Basic(t *testing.T) { addrs := stubBackendsToResolverAddrs(backends) r := manual.NewBuilderWithScheme("whatever") - cc, err := grpc.Dial(r.Scheme()+":///test.server", grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(r)) + + r.InitialState(resolver.State{Addresses: addrs}) + + cc, err := grpc.NewClient(r.Scheme()+":///test.server", grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(r)) if err != nil { - t.Fatalf("grpc.Dial() failed: %v", err) + t.Fatalf("grpc.NewClient() failed: %v", err) } defer cc.Close() // Push a resolver update without an LB policy in the service config. The // channel should pick the default LB policy, which is pick_first. - r.UpdateState(resolver.State{Addresses: addrs}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() if err := pfutil.CheckRPCsToBackend(ctx, cc, addrs[0]); err != nil { t.Fatal(err) } - // Push a resolver update with the service config specifying "round_robin". + // Push a resolver update with a service config specifying "round_robin". r.UpdateState(resolver.State{ Addresses: addrs, ServiceConfig: parseServiceConfig(t, r, rrServiceConfig), @@ -153,7 +155,7 @@ func (s) TestBalancerSwitch_Basic(t *testing.T) { t.Fatal(err) } - // Push a resolver update with the service config specifying "pick_first". + // Push another resolver update with a service config specifying "pick_first". r.UpdateState(resolver.State{ Addresses: addrs, ServiceConfig: parseServiceConfig(t, r, pickFirstServiceConfig), From e4d084a6ece3eae1f1e9a62d3093537dc6040e31 Mon Sep 17 00:00:00 2001 From: eshitachandwani <59800922+eshitachandwani@users.noreply.github.com> Date: Tue, 10 Dec 2024 23:10:08 +0530 Subject: [PATCH 27/57] examples: replace printf with print for log message in gracefulstop (#7917) --- examples/features/gracefulstop/client/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/features/gracefulstop/client/main.go b/examples/features/gracefulstop/client/main.go index 61f5874b3940..772d128ec2b9 100644 --- a/examples/features/gracefulstop/client/main.go +++ b/examples/features/gracefulstop/client/main.go @@ -65,7 +65,7 @@ func main() { } unaryRequests++ time.Sleep(200 * time.Millisecond) - log.Printf(r.Message) + log.Print(r.Message) } log.Printf("Successful unary requests made by client: %d", unaryRequests) From c1b6b3744a35d7cb7eadb3418042262d17efb8f4 Mon Sep 17 00:00:00 2001 From: Ashu Pednekar Date: Wed, 11 Dec 2024 16:12:42 +0530 Subject: [PATCH 28/57] Update README.md (#7921) --- examples/features/xds/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/features/xds/README.md b/examples/features/xds/README.md index 8cf8c4413957..a6a78b0f168a 100644 --- a/examples/features/xds/README.md +++ b/examples/features/xds/README.md @@ -1,7 +1,7 @@ # gRPC xDS example xDS is the protocol initially used by Envoy, that is evolving into a universal -data plan API for service mesh. +data plane API for service mesh. The xDS example is a Hello World client/server capable of being configured with the XDS management protocol. Out-of-the-box it behaves the same as [our other From 38a8b9a7057218c210a599d79f3e1d07f84d18c5 Mon Sep 17 00:00:00 2001 From: Arjan Singh Bal <46515553+arjan-bal@users.noreply.github.com> Date: Thu, 12 Dec 2024 11:50:25 +0530 Subject: [PATCH 29/57] health, grpc: Deliver health service updates through the health listener (#7900) --- balancer_wrapper.go | 73 +++++++++- health/producer.go | 106 +++++++++++++++ internal/internal.go | 4 + test/healthcheck_test.go | 284 ++++++++++++++++++++++++++++++--------- 4 files changed, 397 insertions(+), 70 deletions(-) create mode 100644 health/producer.go diff --git a/balancer_wrapper.go b/balancer_wrapper.go index 905817b5fc7b..c2688376ae74 100644 --- a/balancer_wrapper.go +++ b/balancer_wrapper.go @@ -34,7 +34,15 @@ import ( "google.golang.org/grpc/status" ) -var setConnectedAddress = internal.SetConnectedAddress.(func(*balancer.SubConnState, resolver.Address)) +var ( + setConnectedAddress = internal.SetConnectedAddress.(func(*balancer.SubConnState, resolver.Address)) + // noOpRegisterHealthListenerFn is used when client side health checking is + // disabled. It sends a single READY update on the registered listener. + noOpRegisterHealthListenerFn = func(_ context.Context, listener func(balancer.SubConnState)) func() { + listener(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + return func() {} + } +) // ccBalancerWrapper sits between the ClientConn and the Balancer. // @@ -277,10 +285,17 @@ type healthData struct { // to the LB policy. This is stored to avoid sending updates when the // SubConn has already exited connectivity state READY. connectivityState connectivity.State + // closeHealthProducer stores function to close the ref counted health + // producer. The health producer is automatically closed when the SubConn + // state changes. + closeHealthProducer func() } func newHealthData(s connectivity.State) *healthData { - return &healthData{connectivityState: s} + return &healthData{ + connectivityState: s, + closeHealthProducer: func() {}, + } } // updateState is invoked by grpc to push a subConn state update to the @@ -413,6 +428,37 @@ func (acbw *acBalancerWrapper) closeProducers() { } } +// healthProducerRegisterFn is a type alias for the health producer's function +// for registering listeners. +type healthProducerRegisterFn = func(context.Context, balancer.SubConn, string, func(balancer.SubConnState)) func() + +// healthListenerRegFn returns a function to register a listener for health +// updates. If client side health checks are disabled, the registered listener +// will get a single READY (raw connectivity state) update. +// +// Client side health checking is enabled when all the following +// conditions are satisfied: +// 1. Health checking is not disabled using the dial option. +// 2. The health package is imported. +// 3. The health check config is present in the service config. +func (acbw *acBalancerWrapper) healthListenerRegFn() func(context.Context, func(balancer.SubConnState)) func() { + if acbw.ccb.cc.dopts.disableHealthCheck { + return noOpRegisterHealthListenerFn + } + regHealthLisFn := internal.RegisterClientHealthCheckListener + if regHealthLisFn == nil { + // The health package is not imported. + return noOpRegisterHealthListenerFn + } + cfg := acbw.ac.cc.healthCheckConfig() + if cfg == nil { + return noOpRegisterHealthListenerFn + } + return func(ctx context.Context, listener func(balancer.SubConnState)) func() { + return regHealthLisFn.(healthProducerRegisterFn)(ctx, acbw, cfg.ServiceName, listener) + } +} + // RegisterHealthListener accepts a health listener from the LB policy. It sends // updates to the health listener as long as the SubConn's connectivity state // doesn't change and a new health listener is not registered. To invalidate @@ -421,6 +467,7 @@ func (acbw *acBalancerWrapper) closeProducers() { func (acbw *acBalancerWrapper) RegisterHealthListener(listener func(balancer.SubConnState)) { acbw.healthMu.Lock() defer acbw.healthMu.Unlock() + acbw.healthData.closeHealthProducer() // listeners should not be registered when the connectivity state // isn't Ready. This may happen when the balancer registers a listener // after the connectivityState is updated, but before it is notified @@ -436,6 +483,7 @@ func (acbw *acBalancerWrapper) RegisterHealthListener(listener func(balancer.Sub return } + registerFn := acbw.healthListenerRegFn() acbw.ccb.serializer.TrySchedule(func(ctx context.Context) { if ctx.Err() != nil || acbw.ccb.balancer == nil { return @@ -443,10 +491,25 @@ func (acbw *acBalancerWrapper) RegisterHealthListener(listener func(balancer.Sub // Don't send updates if a new listener is registered. acbw.healthMu.Lock() defer acbw.healthMu.Unlock() - curHD := acbw.healthData - if curHD != hd { + if acbw.healthData != hd { return } - listener(balancer.SubConnState{ConnectivityState: connectivity.Ready}) + // Serialize the health updates from the health producer with + // other calls into the LB policy. + listenerWrapper := func(scs balancer.SubConnState) { + acbw.ccb.serializer.TrySchedule(func(ctx context.Context) { + if ctx.Err() != nil || acbw.ccb.balancer == nil { + return + } + acbw.healthMu.Lock() + defer acbw.healthMu.Unlock() + if acbw.healthData != hd { + return + } + listener(scs) + }) + } + + hd.closeHealthProducer = registerFn(ctx, listenerWrapper) }) } diff --git a/health/producer.go b/health/producer.go new file mode 100644 index 000000000000..f938e5790c7b --- /dev/null +++ b/health/producer.go @@ -0,0 +1,106 @@ +/* + * + * Copyright 2024 gRPC 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 health + +import ( + "context" + "sync" + + "google.golang.org/grpc" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/codes" + "google.golang.org/grpc/connectivity" + "google.golang.org/grpc/internal" + "google.golang.org/grpc/status" +) + +func init() { + producerBuilderSingleton = &producerBuilder{} + internal.RegisterClientHealthCheckListener = registerClientSideHealthCheckListener +} + +type producerBuilder struct{} + +var producerBuilderSingleton *producerBuilder + +// Build constructs and returns a producer and its cleanup function. +func (*producerBuilder) Build(cci any) (balancer.Producer, func()) { + p := &healthServiceProducer{ + cc: cci.(grpc.ClientConnInterface), + cancel: func() {}, + } + return p, func() { + p.mu.Lock() + defer p.mu.Unlock() + p.cancel() + } +} + +type healthServiceProducer struct { + // The following fields are initialized at build time and read-only after + // that and therefore do not need to be guarded by a mutex. + cc grpc.ClientConnInterface + + mu sync.Mutex + cancel func() +} + +// registerClientSideHealthCheckListener accepts a listener to provide server +// health state via the health service. +func registerClientSideHealthCheckListener(ctx context.Context, sc balancer.SubConn, serviceName string, listener func(balancer.SubConnState)) func() { + pr, closeFn := sc.GetOrBuildProducer(producerBuilderSingleton) + p := pr.(*healthServiceProducer) + p.mu.Lock() + defer p.mu.Unlock() + p.cancel() + if listener == nil { + return closeFn + } + + ctx, cancel := context.WithCancel(ctx) + p.cancel = cancel + + go p.startHealthCheck(ctx, sc, serviceName, listener) + return closeFn +} + +func (p *healthServiceProducer) startHealthCheck(ctx context.Context, sc balancer.SubConn, serviceName string, listener func(balancer.SubConnState)) { + newStream := func(method string) (any, error) { + return p.cc.NewStream(ctx, &grpc.StreamDesc{ServerStreams: true}, method) + } + + setConnectivityState := func(state connectivity.State, err error) { + listener(balancer.SubConnState{ + ConnectivityState: state, + ConnectionError: err, + }) + } + + // Call the function through the internal variable as tests use it for + // mocking. + err := internal.HealthCheckFunc(ctx, newStream, setConnectivityState, serviceName) + if err == nil { + return + } + if status.Code(err) == codes.Unimplemented { + logger.Errorf("Subchannel health check is unimplemented at server side, thus health check is disabled for SubConn %p", sc) + } else { + logger.Errorf("Health checking failed for SubConn %p: %v", sc, err) + } +} diff --git a/internal/internal.go b/internal/internal.go index 3afc1813440e..c17b98194b3c 100644 --- a/internal/internal.go +++ b/internal/internal.go @@ -31,6 +31,10 @@ import ( var ( // HealthCheckFunc is used to provide client-side LB channel health checking HealthCheckFunc HealthChecker + // RegisterClientHealthCheckListener is used to provide a listener for + // updates from the client-side health checking service. It returns a + // function that can be called to stop the health producer. + RegisterClientHealthCheckListener any // func(ctx context.Context, sc balancer.SubConn, serviceName string, listener func(balancer.SubConnState)) func() // BalancerUnregister is exported by package balancer to unregister a balancer. BalancerUnregister func(name string) // KeepaliveMinPingTime is the minimum ping interval. This must be 10s by diff --git a/test/healthcheck_test.go b/test/healthcheck_test.go index 424682d09625..fac565240ab7 100644 --- a/test/healthcheck_test.go +++ b/test/healthcheck_test.go @@ -28,12 +28,17 @@ import ( "time" "google.golang.org/grpc" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/balancer/pickfirst" + "google.golang.org/grpc/balancer/pickfirst/pickfirstleaf" "google.golang.org/grpc/codes" "google.golang.org/grpc/connectivity" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/health" "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/balancer/stub" "google.golang.org/grpc/internal/channelz" + "google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/testutils" "google.golang.org/grpc/resolver" @@ -46,6 +51,47 @@ import ( testpb "google.golang.org/grpc/interop/grpc_testing" ) +const healthCheckingPetiolePolicyName = "health_checking_petiole_policy" + +var ( + // healthCheckTestPolicyName is the LB policy used for testing the health check + // service. + healthCheckTestPolicyName = "round_robin" +) + +func init() { + balancer.Register(&healthCheckingPetiolePolicyBuilder{}) + // Till dualstack changes are not implemented and round_robin doesn't + // delegate to pickfirst, test a fake petiole policy that delegates to + // the new pickfirst balancer. + // TODO: https://github.com/grpc/grpc-go/issues/7906 - Remove the fake + // petiole policy one round robin starts delegating to pickfirst. + if envconfig.NewPickFirstEnabled { + healthCheckTestPolicyName = healthCheckingPetiolePolicyName + } +} + +type healthCheckingPetiolePolicyBuilder struct{} + +func (bb *healthCheckingPetiolePolicyBuilder) Build(cc balancer.ClientConn, opts balancer.BuildOptions) balancer.Balancer { + return &healthCheckingPetiolePolicy{ + Balancer: balancer.Get(pickfirstleaf.Name).Build(cc, opts), + } +} + +func (bb *healthCheckingPetiolePolicyBuilder) Name() string { + return healthCheckingPetiolePolicyName +} + +func (b *healthCheckingPetiolePolicy) UpdateClientConnState(state balancer.ClientConnState) error { + state.ResolverState = pickfirstleaf.EnableHealthListener(state.ResolverState) + return b.Balancer.UpdateClientConnState(state) +} + +type healthCheckingPetiolePolicy struct { + balancer.Balancer +} + func newTestHealthServer() *testHealthServer { return newTestHealthServerWithWatchFunc(defaultWatchFunc) } @@ -261,12 +307,12 @@ func (s) TestHealthCheckHealthServerNotRegistered(t *testing.T) { cc, r := setupClient(t, nil) r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, - ServiceConfig: parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "foo" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`)}) + ServiceConfig: parseServiceConfig(t, r, fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "foo" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName))}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -288,12 +334,12 @@ func (s) TestHealthCheckWithGoAway(t *testing.T) { tc := testgrpc.NewTestServiceClient(cc) r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, - ServiceConfig: parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "foo" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`)}) + ServiceConfig: parseServiceConfig(t, r, fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "foo" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName))}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -366,12 +412,12 @@ func (s) TestHealthCheckWithConnClose(t *testing.T) { tc := testgrpc.NewTestServiceClient(cc) r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, - ServiceConfig: parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "foo" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`)}) + ServiceConfig: parseServiceConfig(t, r, fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "foo" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName))}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -414,12 +460,12 @@ func (s) TestHealthCheckWithAddrConnDrain(t *testing.T) { hcEnterChan, hcExitChan := setupHealthCheckWrapper(t) cc, r := setupClient(t, &clientConfig{}) tc := testgrpc.NewTestServiceClient(cc) - sc := parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "foo" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`) + sc := parseServiceConfig(t, r, fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "foo" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName)) r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, ServiceConfig: sc, @@ -496,12 +542,12 @@ func (s) TestHealthCheckWithClientConnClose(t *testing.T) { tc := testgrpc.NewTestServiceClient(cc) r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, - ServiceConfig: parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "foo" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`)}) + ServiceConfig: parseServiceConfig(t, r, (fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "foo" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName)))}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -563,12 +609,12 @@ func (s) TestHealthCheckWithoutSetConnectivityStateCalledAddrConnShutDown(t *tes // The serviceName "delay" is specially handled at server side, where response will not be sent // back to client immediately upon receiving the request (client should receive no response until // test ends). - sc := parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "delay" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`) + sc := parseServiceConfig(t, r, fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "delay" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName)) r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, ServiceConfig: sc, @@ -628,12 +674,12 @@ func (s) TestHealthCheckWithoutSetConnectivityStateCalled(t *testing.T) { // test ends). r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, - ServiceConfig: parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "delay" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`)}) + ServiceConfig: parseServiceConfig(t, r, fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "delay" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName))}) select { case <-hcExitChan: @@ -666,12 +712,12 @@ func testHealthCheckDisableWithDialOption(t *testing.T, addr string) { tc := testgrpc.NewTestServiceClient(cc) r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: addr}}, - ServiceConfig: parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "foo" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`)}) + ServiceConfig: parseServiceConfig(t, r, fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "foo" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName))}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -772,12 +818,12 @@ func (s) TestHealthCheckChannelzCountingCallSuccess(t *testing.T) { _, r := setupClient(t, nil) r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, - ServiceConfig: parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "channelzSuccess" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`)}) + ServiceConfig: parseServiceConfig(t, r, fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "channelzSuccess" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName))}) if err := verifyResultWithDelay(func() (bool, error) { cm, _ := channelz.GetTopChannels(0, 0) @@ -821,12 +867,12 @@ func (s) TestHealthCheckChannelzCountingCallFailure(t *testing.T) { _, r := setupClient(t, nil) r.UpdateState(resolver.State{ Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, - ServiceConfig: parseServiceConfig(t, r, `{ - "healthCheckConfig": { - "serviceName": "channelzFailure" - }, - "loadBalancingConfig": [{"round_robin":{}}] -}`)}) + ServiceConfig: parseServiceConfig(t, r, fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "channelzFailure" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, healthCheckTestPolicyName))}) if err := verifyResultWithDelay(func() (bool, error) { cm, _ := channelz.GetTopChannels(0, 0) @@ -935,12 +981,12 @@ func testHealthCheckSuccess(t *testing.T, e env) { // TestHealthCheckFailure invokes the unary Check() RPC on the health server // with an expired context and expects the RPC to fail. func (s) TestHealthCheckFailure(t *testing.T) { - for _, e := range listTestEnv() { - testHealthCheckFailure(t, e) + e := env{ + name: "tcp-tls", + network: "tcp", + security: "tls", + balancer: healthCheckTestPolicyName, } -} - -func testHealthCheckFailure(t *testing.T, e env) { te := newTest(t, e) te.declareLogNoise( "Failed to dial ", @@ -1166,3 +1212,111 @@ func testHealthCheckServingStatus(t *testing.T, e env) { te.setHealthServingStatus(defaultHealthService, healthpb.HealthCheckResponse_NOT_SERVING) verifyHealthCheckStatus(t, 1*time.Second, cc, defaultHealthService, healthpb.HealthCheckResponse_NOT_SERVING) } + +// Test verifies that registering a nil health listener closes the health +// client. +func (s) TestHealthCheckUnregisterHealthListener(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + hcEnterChan, hcExitChan := setupHealthCheckWrapper(t) + scChan := make(chan balancer.SubConn, 1) + readyUpdateReceivedCh := make(chan struct{}) + bf := stub.BalancerFuncs{ + Init: func(bd *stub.BalancerData) { + cc := bd.ClientConn + ccw := &subConnStoringCCWrapper{ + ClientConn: cc, + scChan: scChan, + stateListener: func(scs balancer.SubConnState) { + if scs.ConnectivityState != connectivity.Ready { + return + } + close(readyUpdateReceivedCh) + }, + } + bd.Data = balancer.Get(pickfirst.Name).Build(ccw, bd.BuildOptions) + }, + Close: func(bd *stub.BalancerData) { + bd.Data.(balancer.Balancer).Close() + }, + UpdateClientConnState: func(bd *stub.BalancerData, ccs balancer.ClientConnState) error { + return bd.Data.(balancer.Balancer).UpdateClientConnState(ccs) + }, + } + + stub.Register(t.Name(), bf) + _, lis, ts := setupServer(t, nil) + ts.SetServingStatus("foo", healthpb.HealthCheckResponse_SERVING) + + _, r := setupClient(t, nil) + svcCfg := fmt.Sprintf(`{ + "healthCheckConfig": { + "serviceName": "foo" + }, + "loadBalancingConfig": [{"%s":{}}] + }`, t.Name()) + r.UpdateState(resolver.State{ + Addresses: []resolver.Address{{Addr: lis.Addr().String()}}, + ServiceConfig: parseServiceConfig(t, r, svcCfg)}) + + var sc balancer.SubConn + select { + case sc = <-scChan: + case <-ctx.Done(): + t.Fatal("Context timed out waiting for SubConn creation") + } + + // Wait for the SubConn to enter READY. + select { + case <-readyUpdateReceivedCh: + case <-ctx.Done(): + t.Fatalf("Context timed out waiting for SubConn to enter READY") + } + + // Health check should start only after a health listener is registered. + select { + case <-hcEnterChan: + t.Fatalf("Health service client created prematurely.") + case <-time.After(defaultTestShortTimeout): + } + + // Register a health listener and verify it receives updates. + healthChan := make(chan balancer.SubConnState, 1) + sc.RegisterHealthListener(func(scs balancer.SubConnState) { + healthChan <- scs + }) + + select { + case <-hcEnterChan: + case <-ctx.Done(): + t.Fatalf("Context timed out waiting for health check to begin.") + } + + for readyReceived := false; !readyReceived; { + select { + case scs := <-healthChan: + t.Logf("Received health update: %v", scs) + readyReceived = scs.ConnectivityState == connectivity.Ready + case <-ctx.Done(): + t.Fatalf("Context timed out waiting for healthy backend.") + } + } + + // Registering a nil listener should invalidate the previously registered + // listener and close the health service client. + sc.RegisterHealthListener(nil) + select { + case <-hcExitChan: + case <-ctx.Done(): + t.Fatalf("Context timed out waiting for the health client to close.") + } + + ts.SetServingStatus("foo", healthpb.HealthCheckResponse_NOT_SERVING) + + // No updates should be received on the listener. + select { + case scs := <-healthChan: + t.Fatalf("Received unexpected health update on the listener: %v", scs) + case <-time.After(defaultTestShortTimeout): + } +} From 7ee073d32520b60644f73054239a4c1ef8f09a3c Mon Sep 17 00:00:00 2001 From: Doug Fawley Date: Fri, 13 Dec 2024 11:24:39 -0800 Subject: [PATCH 30/57] experimental/stats: re-add type aliases for migration (#7929) --- experimental/stats/metrics.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/experimental/stats/metrics.go b/experimental/stats/metrics.go index bf9e7f987b04..ee1423605ab4 100644 --- a/experimental/stats/metrics.go +++ b/experimental/stats/metrics.go @@ -19,6 +19,8 @@ // Package stats contains experimental metrics/stats API's. package stats +import "google.golang.org/grpc/stats" + // MetricsRecorder records on metrics derived from metric registry. type MetricsRecorder interface { // RecordInt64Count records the measurement alongside labels on the int @@ -37,3 +39,16 @@ type MetricsRecorder interface { // gauge associated with the provided handle. RecordInt64Gauge(handle *Int64GaugeHandle, incr int64, labels ...string) } + +// Metrics is an experimental legacy alias of the now-stable stats.MetricSet. +// Metrics will be deleted in a future release. +type Metrics = stats.MetricSet + +// Metric was replaced by direct usage of strings. +type Metric = string + +// NewMetrics is an experimental legacy alias of the now-stable +// stats.NewMetricSet. NewMetrics will be deleted in a future release. +func NewMetrics(metrics ...Metric) *Metrics { + return stats.NewMetricSet(metrics...) +} From 3f762759a72850e4bcd1e8244937849ee5ddde67 Mon Sep 17 00:00:00 2001 From: Easwar Swaminathan Date: Fri, 13 Dec 2024 14:14:05 -0800 Subject: [PATCH 31/57] xdsclient: stop caching xdsChannels for potential reuse, after all references are released (#7924) --- internal/testutils/channel.go | 11 ++ .../balancer/cdsbalancer/cdsbalancer_test.go | 40 ++++-- xds/internal/xdsclient/authority.go | 13 +- xds/internal/xdsclient/client_new.go | 15 +-- xds/internal/xdsclient/client_refcounted.go | 9 +- xds/internal/xdsclient/clientimpl.go | 77 ++--------- .../tests/ads_stream_ack_nack_test.go | 56 ++++---- .../tests/ads_stream_restart_test.go | 49 +------ .../xdsclient/tests/authority_test.go | 127 ++---------------- xds/internal/xdsclient/tests/fallback_test.go | 27 ++-- xds/internal/xdsclient/tests/helpers_test.go | 7 +- .../xdsclient/tests/lds_watchers_test.go | 52 +------ 12 files changed, 128 insertions(+), 355 deletions(-) diff --git a/internal/testutils/channel.go b/internal/testutils/channel.go index 720d8537e666..d1bb0458eb13 100644 --- a/internal/testutils/channel.go +++ b/internal/testutils/channel.go @@ -70,6 +70,17 @@ func (c *Channel) ReceiveOrFail() (any, bool) { } } +// Drain drains the channel by repeatedly reading from it until it is empty. +func (c *Channel) Drain() { + for { + select { + case <-c.C: + default: + return + } + } +} + // Receive returns the value received on the underlying channel, or the error // returned by ctx if it is closed or cancelled. func (c *Channel) Receive(ctx context.Context) (any, error) { diff --git a/xds/internal/balancer/cdsbalancer/cdsbalancer_test.go b/xds/internal/balancer/cdsbalancer/cdsbalancer_test.go index bd9cd5573805..9c17cee62cdf 100644 --- a/xds/internal/balancer/cdsbalancer/cdsbalancer_test.go +++ b/xds/internal/balancer/cdsbalancer/cdsbalancer_test.go @@ -21,6 +21,7 @@ import ( "encoding/json" "errors" "fmt" + "net" "strings" "testing" "time" @@ -202,11 +203,18 @@ func registerWrappedCDSPolicy(t *testing.T) chan balancer.Balancer { // - a channel used to signal that previously requested cluster resources are // no longer requested func setupWithManagementServer(t *testing.T) (*e2e.ManagementServer, string, *grpc.ClientConn, *manual.Resolver, xdsclient.XDSClient, chan []string, chan struct{}) { + return setupWithManagementServerAndListener(t, nil) +} + +// Same as setupWithManagementServer, but also allows the caller to specify +// a listener to be used by the management server. +func setupWithManagementServerAndListener(t *testing.T, lis net.Listener) (*e2e.ManagementServer, string, *grpc.ClientConn, *manual.Resolver, xdsclient.XDSClient, chan []string, chan struct{}) { t.Helper() cdsResourceRequestedCh := make(chan []string, 1) cdsResourceCanceledCh := make(chan struct{}, 1) mgmtServer := e2e.StartManagementServer(t, e2e.ManagementServerOptions{ + Listener: lis, OnStreamRequest: func(_ int64, req *v3discoverypb.DiscoveryRequest) error { if req.GetTypeUrl() == version.V3ClusterURL { switch len(req.GetResourceNames()) { @@ -807,11 +815,20 @@ func (s) TestClusterUpdate_Failure(t *testing.T) { // TRANSIENT_FAILURE. It is also expected to cancel the CDS watch. func (s) TestResolverError(t *testing.T) { _, resolverErrCh, _, _ := registerWrappedClusterResolverPolicy(t) - mgmtServer, nodeID, cc, r, _, cdsResourceRequestedCh, cdsResourceCanceledCh := setupWithManagementServer(t) + lis := testutils.NewListenerWrapper(t, nil) + mgmtServer, nodeID, cc, r, _, cdsResourceRequestedCh, cdsResourceCanceledCh := setupWithManagementServerAndListener(t, lis) - // Verify that the specified cluster resource is requested. + // Grab the wrapped connection from the listener wrapper. This will be used + // to verify the connection is closed. ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() + val, err := lis.NewConnCh.Receive(ctx) + if err != nil { + t.Fatalf("Failed to receive new connection from wrapped listener: %v", err) + } + conn := val.(*testutils.ConnWrapper) + + // Verify that the specified cluster resource is requested. wantNames := []string{clusterName} if err := waitForResourceNames(ctx, cdsResourceRequestedCh, wantNames); err != nil { t.Fatal(err) @@ -831,7 +848,7 @@ func (s) TestResolverError(t *testing.T) { // Ensure that the resolver error is propagated to the RPC caller. client := testgrpc.NewTestServiceClient(cc) - _, err := client.EmptyCall(ctx, &testpb.Empty{}) + _, err = client.EmptyCall(ctx, &testpb.Empty{}) if code := status.Code(err); code != codes.Unavailable { t.Fatalf("EmptyCall() failed with code: %v, want %v", code, codes.Unavailable) } @@ -901,11 +918,14 @@ func (s) TestResolverError(t *testing.T) { resolverErr = xdsresource.NewErrorf(xdsresource.ErrorTypeResourceNotFound, "xds resource not found error") r.ReportError(resolverErr) - // Wait for the CDS resource to be not requested anymore. + // Wait for the CDS resource to be not requested anymore, or the connection + // to the management server to be closed (which happens as part of the last + // resource watch being canceled). select { case <-ctx.Done(): t.Fatal("Timeout when waiting for CDS resource to be not requested") case <-cdsResourceCanceledCh: + case <-conn.CloseCh.C: } // Verify that the resolver error is pushed to the child policy. @@ -928,12 +948,12 @@ func (s) TestResolverError(t *testing.T) { } } -// Tests that closing the cds LB policy results in the cluster resource watch -// being cancelled and the child policy being closed. +// Tests that closing the cds LB policy results in the the child policy being +// closed. func (s) TestClose(t *testing.T) { cdsBalancerCh := registerWrappedCDSPolicy(t) _, _, _, childPolicyCloseCh := registerWrappedClusterResolverPolicy(t) - mgmtServer, nodeID, cc, _, _, _, cdsResourceCanceledCh := setupWithManagementServer(t) + mgmtServer, nodeID, cc, _, _, _, _ := setupWithManagementServer(t) // Start a test service backend. server := stubserver.StartTestService(t, nil) @@ -967,12 +987,6 @@ func (s) TestClose(t *testing.T) { } cdsBal.Close() - // Wait for the CDS resource to be not requested anymore. - select { - case <-ctx.Done(): - t.Fatal("Timeout when waiting for CDS resource to be not requested") - case <-cdsResourceCanceledCh: - } // Wait for the child policy to be closed. select { case <-ctx.Done(): diff --git a/xds/internal/xdsclient/authority.go b/xds/internal/xdsclient/authority.go index 24673a8d9077..f81685a45e69 100644 --- a/xds/internal/xdsclient/authority.go +++ b/xds/internal/xdsclient/authority.go @@ -639,6 +639,9 @@ func (a *authority) watchResource(rType xdsresource.Type, resourceName string, w if a.logger.V(2) { a.logger.Infof("Resource type %q with resource name %q found in cache: %s", rType.TypeName(), resourceName, state.cache.ToJSON()) } + // state can only be accessed in the context of an + // xdsClientSerializer callback. Hence making a copy of the cached + // resource here for watchCallbackSerializer. resource := state.cache a.watcherCallbackSerializer.TrySchedule(func(context.Context) { watcher.OnUpdate(resource, func() {}) }) } @@ -646,9 +649,13 @@ func (a *authority) watchResource(rType xdsresource.Type, resourceName string, w // immediately as well. if state.md.Status == xdsresource.ServiceStatusNACKed { if a.logger.V(2) { - a.logger.Infof("Resource type %q with resource name %q was NACKed: %s", rType.TypeName(), resourceName, state.cache.ToJSON()) + a.logger.Infof("Resource type %q with resource name %q was NACKed", rType.TypeName(), resourceName) } - a.watcherCallbackSerializer.TrySchedule(func(context.Context) { watcher.OnError(state.md.ErrState.Err, func() {}) }) + // state can only be accessed in the context of an + // xdsClientSerializer callback. Hence making a copy of the error + // here for watchCallbackSerializer. + err := state.md.ErrState.Err + a.watcherCallbackSerializer.TrySchedule(func(context.Context) { watcher.OnError(err, func() {}) }) } // If the metadata field is updated to indicate that the management // server does not have this resource, notify the new watcher. @@ -687,7 +694,7 @@ func (a *authority) unwatchResource(rType xdsresource.Type, resourceName string, delete(state.watchers, watcher) if len(state.watchers) > 0 { if a.logger.V(2) { - a.logger.Infof("%d more watchers exist for type %q, resource name %q", rType.TypeName(), resourceName) + a.logger.Infof("Other watchers exist for type %q, resource name %q", rType.TypeName(), resourceName) } return } diff --git a/xds/internal/xdsclient/client_new.go b/xds/internal/xdsclient/client_new.go index d8f9d6c9417b..55299c457b25 100644 --- a/xds/internal/xdsclient/client_new.go +++ b/xds/internal/xdsclient/client_new.go @@ -26,7 +26,6 @@ import ( "google.golang.org/grpc/internal" "google.golang.org/grpc/internal/backoff" - "google.golang.org/grpc/internal/cache" "google.golang.org/grpc/internal/grpcsync" "google.golang.org/grpc/internal/xds/bootstrap" xdsclientinternal "google.golang.org/grpc/xds/internal/xdsclient/internal" @@ -61,11 +60,11 @@ func New(name string) (XDSClient, func(), error) { if err != nil { return nil, nil, fmt.Errorf("xds: failed to get xDS bootstrap config: %v", err) } - return newRefCounted(name, config, defaultWatchExpiryTimeout, defaultIdleChannelExpiryTimeout, backoff.DefaultExponential.Backoff) + return newRefCounted(name, config, defaultWatchExpiryTimeout, backoff.DefaultExponential.Backoff) } // newClientImpl returns a new xdsClient with the given config. -func newClientImpl(config *bootstrap.Config, watchExpiryTimeout, idleChannelExpiryTimeout time.Duration, streamBackoff func(int) time.Duration) (*clientImpl, error) { +func newClientImpl(config *bootstrap.Config, watchExpiryTimeout time.Duration, streamBackoff func(int) time.Duration) (*clientImpl, error) { ctx, cancel := context.WithCancel(context.Background()) c := &clientImpl{ done: grpcsync.NewEvent(), @@ -78,7 +77,6 @@ func newClientImpl(config *bootstrap.Config, watchExpiryTimeout, idleChannelExpi transportBuilder: &grpctransport.Builder{}, resourceTypes: newResourceTypeRegistry(), xdsActiveChannels: make(map[string]*channelState), - xdsIdleChannels: cache.NewTimeoutCache(idleChannelExpiryTimeout), } for name, cfg := range config.Authorities() { @@ -121,10 +119,6 @@ type OptionsForTesting struct { // unspecified, uses the default value used in non-test code. WatchExpiryTimeout time.Duration - // IdleChannelExpiryTimeout is the timeout before idle xdsChannels are - // deleted. If unspecified, uses the default value used in non-test code. - IdleChannelExpiryTimeout time.Duration - // StreamBackoffAfterFailure is the backoff function used to determine the // backoff duration after stream failures. // If unspecified, uses the default value used in non-test code. @@ -147,9 +141,6 @@ func NewForTesting(opts OptionsForTesting) (XDSClient, func(), error) { if opts.WatchExpiryTimeout == 0 { opts.WatchExpiryTimeout = defaultWatchExpiryTimeout } - if opts.IdleChannelExpiryTimeout == 0 { - opts.IdleChannelExpiryTimeout = defaultIdleChannelExpiryTimeout - } if opts.StreamBackoffAfterFailure == nil { opts.StreamBackoffAfterFailure = defaultStreamBackoffFunc } @@ -158,7 +149,7 @@ func NewForTesting(opts OptionsForTesting) (XDSClient, func(), error) { if err != nil { return nil, nil, err } - return newRefCounted(opts.Name, config, opts.WatchExpiryTimeout, opts.IdleChannelExpiryTimeout, opts.StreamBackoffAfterFailure) + return newRefCounted(opts.Name, config, opts.WatchExpiryTimeout, opts.StreamBackoffAfterFailure) } // GetForTesting returns an xDS client created earlier using the given name. diff --git a/xds/internal/xdsclient/client_refcounted.go b/xds/internal/xdsclient/client_refcounted.go index 1c105ac4e061..5a256c1bfada 100644 --- a/xds/internal/xdsclient/client_refcounted.go +++ b/xds/internal/xdsclient/client_refcounted.go @@ -27,10 +27,7 @@ import ( "google.golang.org/grpc/internal/xds/bootstrap" ) -const ( - defaultWatchExpiryTimeout = 15 * time.Second - defaultIdleChannelExpiryTimeout = 5 * time.Minute -) +const defaultWatchExpiryTimeout = 15 * time.Second var ( // The following functions are no-ops in the actual code, but can be @@ -62,7 +59,7 @@ func clientRefCountedClose(name string) { // newRefCounted creates a new reference counted xDS client implementation for // name, if one does not exist already. If an xDS client for the given name // exists, it gets a reference to it and returns it. -func newRefCounted(name string, config *bootstrap.Config, watchExpiryTimeout, idleChannelExpiryTimeout time.Duration, streamBackoff func(int) time.Duration) (XDSClient, func(), error) { +func newRefCounted(name string, config *bootstrap.Config, watchExpiryTimeout time.Duration, streamBackoff func(int) time.Duration) (XDSClient, func(), error) { clientsMu.Lock() defer clientsMu.Unlock() @@ -72,7 +69,7 @@ func newRefCounted(name string, config *bootstrap.Config, watchExpiryTimeout, id } // Create the new client implementation. - c, err := newClientImpl(config, watchExpiryTimeout, idleChannelExpiryTimeout, streamBackoff) + c, err := newClientImpl(config, watchExpiryTimeout, streamBackoff) if err != nil { return nil, nil, err } diff --git a/xds/internal/xdsclient/clientimpl.go b/xds/internal/xdsclient/clientimpl.go index df0949e23cc7..bb8d9040022f 100644 --- a/xds/internal/xdsclient/clientimpl.go +++ b/xds/internal/xdsclient/clientimpl.go @@ -25,7 +25,6 @@ import ( "sync/atomic" "time" - "google.golang.org/grpc/internal/cache" "google.golang.org/grpc/internal/grpclog" "google.golang.org/grpc/internal/grpcsync" "google.golang.org/grpc/internal/xds/bootstrap" @@ -63,14 +62,9 @@ type clientImpl struct { // these channels, and forwards updates from the channels to each of these // authorities. // - // Once all references to a channel are dropped, the channel is moved to the - // idle cache where it lives for a configured duration before being closed. - // If the channel is required before the idle timeout fires, it is revived - // from the idle cache and used. + // Once all references to a channel are dropped, the channel is closed. channelsMu sync.Mutex xdsActiveChannels map[string]*channelState // Map from server config to in-use xdsChannels. - xdsIdleChannels *cache.TimeoutCache // Map from server config to idle xdsChannels. - closeCond *sync.Cond } // channelState represents the state of an xDS channel. It tracks the number of @@ -173,21 +167,6 @@ func (c *clientImpl) close() { c.close() } - // Similarly, closing idle channels cannot be done with the lock held, for - // the same reason as described above. So, we clear the idle cache in a - // goroutine and use a condition variable to wait on the condition that the - // idle cache has zero entries. The Wait() method on the condition variable - // releases the lock and blocks the goroutine until signaled (which happens - // when an idle channel is removed from the cache and closed), and grabs the - // lock before returning. - c.channelsMu.Lock() - c.closeCond = sync.NewCond(&c.channelsMu) - go c.xdsIdleChannels.Clear(true) - for c.xdsIdleChannels.Len() > 0 { - c.closeCond.Wait() - } - c.channelsMu.Unlock() - c.serializerClose() <-c.serializer.Done() @@ -289,27 +268,15 @@ func (c *clientImpl) getOrCreateChannel(serverConfig *bootstrap.ServerConfig, in c.logger.Infof("Received request for a reference to an xdsChannel for server config %q", serverConfig) } - // Use an active channel, if one exists for this server config. + // Use an existing channel, if one exists for this server config. if state, ok := c.xdsActiveChannels[serverConfig.String()]; ok { if c.logger.V(2) { - c.logger.Infof("Reusing an active xdsChannel for server config %q", serverConfig) + c.logger.Infof("Reusing an existing xdsChannel for server config %q", serverConfig) } initLocked(state) return state.channel, c.releaseChannel(serverConfig, state, deInitLocked), nil } - // If an idle channel exists for this server config, remove it from the - // idle cache and add it to the map of active channels, and return it. - if s, ok := c.xdsIdleChannels.Remove(serverConfig.String()); ok { - if c.logger.V(2) { - c.logger.Infof("Reviving an xdsChannel from the idle cache for server config %q", serverConfig) - } - state := s.(*channelState) - c.xdsActiveChannels[serverConfig.String()] = state - initLocked(state) - return state.channel, c.releaseChannel(serverConfig, state, deInitLocked), nil - } - if c.logger.V(2) { c.logger.Infof("Creating a new xdsChannel for server config %q", serverConfig) } @@ -345,9 +312,7 @@ func (c *clientImpl) getOrCreateChannel(serverConfig *bootstrap.ServerConfig, in } // releaseChannel is a function that is called when a reference to an xdsChannel -// needs to be released. It handles the logic of moving the channel to an idle -// cache if there are no other active references, and closing the channel if it -// remains in the idle cache for the configured duration. +// needs to be released. It handles closing channels with no active references. // // The function takes the following parameters: // - serverConfig: the server configuration for the xdsChannel @@ -360,7 +325,6 @@ func (c *clientImpl) getOrCreateChannel(serverConfig *bootstrap.ServerConfig, in func (c *clientImpl) releaseChannel(serverConfig *bootstrap.ServerConfig, state *channelState, deInitLocked func(*channelState)) func() { return grpcsync.OnceFunc(func() { c.channelsMu.Lock() - defer c.channelsMu.Unlock() if c.logger.V(2) { c.logger.Infof("Received request to release a reference to an xdsChannel for server config %q", serverConfig) @@ -372,40 +336,17 @@ func (c *clientImpl) releaseChannel(serverConfig *bootstrap.ServerConfig, state if c.logger.V(2) { c.logger.Infof("xdsChannel %p has other active references", state.channel) } + c.channelsMu.Unlock() return } - // Move the channel to the idle cache instead of closing - // immediately. If the channel remains in the idle cache for - // the configured duration, it will get closed. delete(c.xdsActiveChannels, serverConfig.String()) if c.logger.V(2) { - c.logger.Infof("Moving xdsChannel [%p] for server config %s to the idle cache", state.channel, serverConfig) + c.logger.Infof("Closing xdsChannel [%p] for server config %s", state.channel, serverConfig) } + channelToClose := state.channel + c.channelsMu.Unlock() - // The idle cache expiry timeout results in the channel getting - // closed in another serializer callback. - c.xdsIdleChannels.Add(serverConfig.String(), state, grpcsync.OnceFunc(func() { - c.channelsMu.Lock() - channelToClose := state.channel - c.channelsMu.Unlock() - - if c.logger.V(2) { - c.logger.Infof("Idle cache expiry timeout fired for xdsChannel [%p] for server config %s", state.channel, serverConfig) - } - channelToClose.close() - - // If the channel is being closed as a result of the xDS client - // being closed, closeCond is non-nil and we need to signal from - // here to unblock Close(). Holding the lock is not necessary - // to call Signal() on a condition variable. But the field - // `c.closeCond` needs to guarded by the lock, which is why we - // acquire it here. - c.channelsMu.Lock() - if c.closeCond != nil { - c.closeCond.Signal() - } - c.channelsMu.Unlock() - })) + channelToClose.close() }) } diff --git a/xds/internal/xdsclient/tests/ads_stream_ack_nack_test.go b/xds/internal/xdsclient/tests/ads_stream_ack_nack_test.go index feebc1adc915..af16b6a59bc3 100644 --- a/xds/internal/xdsclient/tests/ads_stream_ack_nack_test.go +++ b/xds/internal/xdsclient/tests/ads_stream_ack_nack_test.go @@ -337,9 +337,10 @@ func (s) TestADS_NACK_InvalidFirstResponse(t *testing.T) { // 1. A resource is requested and a good response is received. The test verifies // that an ACK is sent for this resource. // 2. The previously requested resource is no longer requested. The test -// verifies that a request with no resource names is sent out. -// 3. The same resource is requested again. The test verifies that the request -// is sent with the previously ACKed version. +// verifies that the connection to the management server is closed. +// 3. The same resource is requested again. The test verifies that a new +// request is sent with an empty version string, which corresponds to the +// first request on a new connection. func (s) TestADS_ACK_NACK_ResourceIsNotRequestedAnymore(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -349,6 +350,7 @@ func (s) TestADS_ACK_NACK_ResourceIsNotRequestedAnymore(t *testing.T) { // the test goroutine to verify ACK version and nonce. streamRequestCh := testutils.NewChannel() streamResponseCh := testutils.NewChannel() + streamCloseCh := testutils.NewChannel() mgmtServer := e2e.StartManagementServer(t, e2e.ManagementServerOptions{ OnStreamRequest: func(_ int64, req *v3discoverypb.DiscoveryRequest) error { streamRequestCh.SendContext(ctx, req) @@ -357,6 +359,9 @@ func (s) TestADS_ACK_NACK_ResourceIsNotRequestedAnymore(t *testing.T) { OnStreamResponse: func(_ context.Context, _ int64, _ *v3discoverypb.DiscoveryRequest, resp *v3discoverypb.DiscoveryResponse) { streamResponseCh.SendContext(ctx, resp) }, + OnStreamClosed: func(int64, *v3corepb.Node) { + streamCloseCh.SendContext(ctx, struct{}{}) + }, }) // Create a listener resource on the management server. @@ -425,9 +430,10 @@ func (s) TestADS_ACK_NACK_ResourceIsNotRequestedAnymore(t *testing.T) { t.Fatal("Timeout when waiting for ACK") } gotReq = r.(*v3discoverypb.DiscoveryRequest) - wantReq.VersionInfo = gotResp.GetVersionInfo() - wantReq.ResponseNonce = gotResp.GetNonce() - if diff := cmp.Diff(gotReq, wantReq, protocmp.Transform()); diff != "" { + wantACKReq := proto.Clone(wantReq).(*v3discoverypb.DiscoveryRequest) + wantACKReq.VersionInfo = gotResp.GetVersionInfo() + wantACKReq.ResponseNonce = gotResp.GetNonce() + if diff := cmp.Diff(gotReq, wantACKReq, protocmp.Transform()); diff != "" { t.Fatalf("Unexpected diff in received discovery request, diff (-got, +want):\n%s", diff) } @@ -442,39 +448,41 @@ func (s) TestADS_ACK_NACK_ResourceIsNotRequestedAnymore(t *testing.T) { t.Fatal(err) } - // Cancel the watch on the listener resource. This should result in a - // discovery request with no resource names. + // Cancel the watch on the listener resource. This should result in the + // existing connection to be management server getting closed. ldsCancel() - - // Verify that the discovery request matches expectation. - r, err = streamRequestCh.Receive(ctx) - if err != nil { - t.Fatal("Timeout when waiting for discovery request") - } - gotReq = r.(*v3discoverypb.DiscoveryRequest) - wantReq.ResourceNames = nil - if diff := cmp.Diff(gotReq, wantReq, protocmp.Transform()); diff != "" { - t.Fatalf("Unexpected diff in received discovery request, diff (-got, +want):\n%s", diff) + if _, err := streamCloseCh.Receive(ctx); err != nil { + t.Fatalf("Timeout when expecting existing connection to be closed: %v", err) } + // There is a race between two events when the last watch on an xdsChannel + // is canceled: + // - an empty discovery request being sent out + // - the ADS stream being closed + // To handle this race, we drain the request channel here so that if an + // empty discovery request was received, it is pulled out of the request + // channel and thereby guaranteeing a clean slate for the next watch + // registered below. + streamRequestCh.Drain() + // Register a watch for the same listener resource. lw = newListenerWatcher() ldsCancel = xdsresource.WatchListener(client, listenerName, lw) defer ldsCancel() - // Verify that the discovery request contains the version from the - // previously received response. + // Verify that the discovery request is identical to the first one sent out + // to the management server. r, err = streamRequestCh.Receive(ctx) if err != nil { t.Fatal("Timeout when waiting for discovery request") } gotReq = r.(*v3discoverypb.DiscoveryRequest) - wantReq.ResourceNames = []string{listenerName} if diff := cmp.Diff(gotReq, wantReq, protocmp.Transform()); diff != "" { t.Fatalf("Unexpected diff in received discovery request, diff (-got, +want):\n%s", diff) } - // TODO(https://github.com/envoyproxy/go-control-plane/issues/1002): Once - // this bug is fixed, we need to verify that the update is received by the - // watcher. + // Verify the update received by the watcher. + if err := verifyListenerUpdate(ctx, lw.updateCh, wantUpdate); err != nil { + t.Fatal(err) + } } diff --git a/xds/internal/xdsclient/tests/ads_stream_restart_test.go b/xds/internal/xdsclient/tests/ads_stream_restart_test.go index f0da932f5fd8..7e15218ea403 100644 --- a/xds/internal/xdsclient/tests/ads_stream_restart_test.go +++ b/xds/internal/xdsclient/tests/ads_stream_restart_test.go @@ -124,6 +124,7 @@ func (s) TestADS_ResourcesAreRequestedAfterStreamRestart(t *testing.T) { // Register a watch for a listener resource. lw := newListenerWatcher() ldsCancel := xdsresource.WatchListener(client, listenerName, lw) + defer ldsCancel() // Verify that an ADS stream is opened and an LDS request with the above // resource name is sent. @@ -147,52 +148,6 @@ func (s) TestADS_ResourcesAreRequestedAfterStreamRestart(t *testing.T) { t.Fatal(err) } - // Cancel the watch for the above listener resource, and verify that an LDS - // request with no resource names is sent. - ldsCancel() - if err := waitForResourceNames(ctx, t, ldsResourcesCh, []string{}); err != nil { - t.Fatal(err) - } - - // Stop the restartable listener and wait for the stream to close. - lis.Stop() - select { - case <-streamClosed: - case <-ctx.Done(): - t.Fatal("Timeout when waiting for ADS stream to close") - } - - // Restart the restartable listener and wait for the stream to open. - lis.Restart() - select { - case <-streamOpened: - case <-ctx.Done(): - t.Fatal("Timeout when waiting for ADS stream to open") - } - - // Wait for a short duration and verify that no LDS request is sent, since - // there are no resources being watched. - sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout) - defer sCancel() - select { - case <-sCtx.Done(): - case names := <-ldsResourcesCh: - t.Fatalf("LDS request sent for resource names %v, when expecting no request", names) - } - - // Register another watch for the same listener resource, and verify that an - // LDS request with the above resource name is sent. - ldsCancel = xdsresource.WatchListener(client, listenerName, lw) - if err := waitForResourceNames(ctx, t, ldsResourcesCh, []string{listenerName}); err != nil { - t.Fatal(err) - } - defer ldsCancel() - - // Verify the update received by the watcher. - if err := verifyListenerUpdate(ctx, lw.updateCh, wantListenerUpdate); err != nil { - t.Fatal(err) - } - // Create a cluster resource on the management server, in addition to the // existing listener resource. const clusterName = "cluster" @@ -255,7 +210,7 @@ func (s) TestADS_ResourcesAreRequestedAfterStreamRestart(t *testing.T) { // Wait for a short duration and verify that no CDS request is sent, since // there are no resources being watched. - sCtx, sCancel = context.WithTimeout(ctx, defaultTestShortTimeout) + sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout) defer sCancel() select { case <-sCtx.Done(): diff --git a/xds/internal/xdsclient/tests/authority_test.go b/xds/internal/xdsclient/tests/authority_test.go index 1947daf6dffe..3570be59d46f 100644 --- a/xds/internal/xdsclient/tests/authority_test.go +++ b/xds/internal/xdsclient/tests/authority_test.go @@ -23,7 +23,6 @@ import ( "encoding/json" "fmt" "testing" - "time" "github.com/google/uuid" "google.golang.org/grpc/internal/testutils" @@ -65,7 +64,7 @@ var ( // // Returns two listeners used by the default and non-default management servers // respectively, and the xDS client and its close function. -func setupForAuthorityTests(ctx context.Context, t *testing.T, idleTimeout time.Duration) (*testutils.ListenerWrapper, *testutils.ListenerWrapper, xdsclient.XDSClient, func()) { +func setupForAuthorityTests(ctx context.Context, t *testing.T) (*testutils.ListenerWrapper, *testutils.ListenerWrapper, xdsclient.XDSClient, func()) { // Create listener wrappers which notify on to a channel whenever a new // connection is accepted. We use this to track the number of transports // used by the xDS client. @@ -102,10 +101,9 @@ func setupForAuthorityTests(ctx context.Context, t *testing.T, idleTimeout time. t.Fatalf("Failed to create bootstrap configuration: %v", err) } client, close, err := xdsclient.NewForTesting(xdsclient.OptionsForTesting{ - Name: t.Name(), - Contents: bootstrapContents, - WatchExpiryTimeout: defaultTestWatchExpiryTimeout, - IdleChannelExpiryTimeout: idleTimeout, + Name: t.Name(), + Contents: bootstrapContents, + WatchExpiryTimeout: defaultTestWatchExpiryTimeout, }) if err != nil { t.Fatalf("Failed to create an xDS client: %v", err) @@ -137,7 +135,7 @@ func setupForAuthorityTests(ctx context.Context, t *testing.T, idleTimeout time. func (s) TestAuthority_XDSChannelSharing(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - lis, _, client, close := setupForAuthorityTests(ctx, t, time.Duration(0)) + lis, _, client, close := setupForAuthorityTests(ctx, t) defer close() // Verify that no connection is established to the management server at this @@ -176,13 +174,12 @@ func (s) TestAuthority_XDSChannelSharing(t *testing.T) { } } -// Test the xdsChannel idle timeout logic. The test verifies that the xDS client -// does not close xdsChannels immediately after the last watch is canceled, but -// waits for the configured idle timeout to expire before closing them. -func (s) TestAuthority_XDSChannelIdleTimeout(t *testing.T) { +// Test the xdsChannel close logic. The test verifies that the xDS client +// closes an xdsChannel immediately after the last watch is canceled. +func (s) TestAuthority_XDSChannelClose(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - lis, _, client, close := setupForAuthorityTests(ctx, t, defaultTestIdleChannelExpiryTimeout) + lis, _, client, close := setupForAuthorityTests(ctx, t) defer close() // Request the first resource. Verify that a new transport is created. @@ -203,110 +200,10 @@ func (s) TestAuthority_XDSChannelIdleTimeout(t *testing.T) { } // Cancel both watches, and verify that the connection to the management - // server is not closed immediately. + // server is closed. cdsCancel1() cdsCancel2() - sCtx, sCancel = context.WithTimeout(ctx, defaultTestShortTimeout) - defer sCancel() - if _, err := conn.CloseCh.Receive(sCtx); err != context.DeadlineExceeded { - t.Fatal("Connection to management server closed unexpectedly") - } - - // Ensure the transport is closed once the idle timeout fires. - select { - case <-conn.CloseCh.C: - case <-time.After(2 * defaultTestIdleChannelExpiryTimeout): - t.Fatal("Connection to management server not closed after idle timeout expiry") - } -} - -// Tests that xdsChannels in use and in the idle cache are all closed when the -// xDS client is closed. -func (s) TestAuthority_XDSChannelCloseOnClientClose(t *testing.T) { - // Set the idle timeout to twice the defaultTestTimeout. This will ensure - // that idle channels stay in the cache for the duration of this test, until - // explicitly closed. - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - lisDefault, lisNonDefault, client, close := setupForAuthorityTests(ctx, t, time.Duration(2*defaultTestTimeout)) - - // Request the first resource. Verify that a new transport is created to the - // default management server. - watcher := noopClusterWatcher{} - cdsCancel1 := xdsresource.WatchCluster(client, authorityTestResourceName11, watcher) - val, err := lisDefault.NewConnCh.Receive(ctx) - if err != nil { - t.Fatalf("Timed out when waiting for a new transport to be created to the management server: %v", err) - } - connDefault := val.(*testutils.ConnWrapper) - - // Request another resource which is served by the non-default authority. - // Verify that a new transport is created to the non-default management - // server. - xdsresource.WatchCluster(client, authorityTestResourceName3, watcher) - val, err = lisNonDefault.NewConnCh.Receive(ctx) - if err != nil { - t.Fatalf("Timed out when waiting for a new transport to be created to the management server: %v", err) - } - connNonDefault := val.(*testutils.ConnWrapper) - - // Cancel the first watch. This should move the default authority to the - // idle cache, but the connection should not be closed yet, because the idle - // timeout would not have fired. - cdsCancel1() - sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout) - defer sCancel() - if _, err := connDefault.CloseCh.Receive(sCtx); err != context.DeadlineExceeded { - t.Fatal("Connection to management server closed unexpectedly") - } - - // Closing the xDS client should close the connection to both management - // servers, even though we have an open watch to one of them. - close() - if _, err := connDefault.CloseCh.Receive(ctx); err != nil { - t.Fatal("Connection to management server not closed after client close") - } - if _, err := connNonDefault.CloseCh.Receive(ctx); err != nil { - t.Fatal("Connection to management server not closed after client close") - } -} - -// Tests that an xdsChannel in the idle cache is revived when a new watch is -// started on an authority. -func (s) TestAuthority_XDSChannelRevive(t *testing.T) { - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - lis, _, client, close := setupForAuthorityTests(ctx, t, defaultTestIdleChannelExpiryTimeout) - defer close() - - // Request the first resource. Verify that a new transport is created. - watcher := noopClusterWatcher{} - cdsCancel1 := xdsresource.WatchCluster(client, authorityTestResourceName11, watcher) - val, err := lis.NewConnCh.Receive(ctx) - if err != nil { - t.Fatalf("Timed out when waiting for a new transport to be created to the management server: %v", err) - } - conn := val.(*testutils.ConnWrapper) - - // Cancel the above watch. This should move the authority to the idle cache. - cdsCancel1() - - // Request the second resource. Verify that no new transport is created. - // This should move the authority out of the idle cache. - cdsCancel2 := xdsresource.WatchCluster(client, authorityTestResourceName12, watcher) - defer cdsCancel2() - sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout) - defer sCancel() - if _, err := lis.NewConnCh.Receive(sCtx); err != context.DeadlineExceeded { - t.Fatal("Unexpected new transport created to management server") - } - - // Wait for double the idle timeout, and the connection to the management - // server should not be closed, since it was revived from the idle cache. - time.Sleep(2 * defaultTestIdleChannelExpiryTimeout) - sCtx, sCancel = context.WithTimeout(ctx, defaultTestShortTimeout) - defer sCancel() - if _, err := conn.CloseCh.Receive(sCtx); err != context.DeadlineExceeded { - t.Fatal("Connection to management server closed unexpectedly") + if _, err := conn.CloseCh.Receive(ctx); err != nil { + t.Fatal("Timeout when waiting for connection to management server to be closed") } } diff --git a/xds/internal/xdsclient/tests/fallback_test.go b/xds/internal/xdsclient/tests/fallback_test.go index 514945f833d0..18c382dbd146 100644 --- a/xds/internal/xdsclient/tests/fallback_test.go +++ b/xds/internal/xdsclient/tests/fallback_test.go @@ -160,13 +160,10 @@ func (s) TestFallback_OnStartup(t *testing.T) { t.Fatalf("Failed to create bootstrap file: %v", err) } - // Create an xDS client with the above bootstrap configuration and a short - // idle channel expiry timeout. This ensures that connections to lower - // priority servers get closed quickly, for the test to verify. + // Create an xDS client with the above bootstrap configuration. xdsC, close, err := xdsclient.NewForTesting(xdsclient.OptionsForTesting{ - Name: t.Name(), - Contents: bootstrapContents, - IdleChannelExpiryTimeout: defaultTestIdleChannelExpiryTimeout, + Name: t.Name(), + Contents: bootstrapContents, }) if err != nil { t.Fatalf("Failed to create xDS client: %v", err) @@ -363,13 +360,10 @@ func (s) TestFallback_MidUpdate(t *testing.T) { t.Fatalf("Failed to create bootstrap file: %v", err) } - // Create an xDS client with the above bootstrap configuration and a short - // idle channel expiry timeout. This ensures that connections to lower - // priority servers get closed quickly, for the test to verify. + // Create an xDS client with the above bootstrap configuration. xdsC, close, err := xdsclient.NewForTesting(xdsclient.OptionsForTesting{ - Name: t.Name(), - Contents: bootstrapContents, - IdleChannelExpiryTimeout: defaultTestIdleChannelExpiryTimeout, + Name: t.Name(), + Contents: bootstrapContents, }) if err != nil { t.Fatalf("Failed to create xDS client: %v", err) @@ -556,13 +550,10 @@ func (s) TestFallback_MidStartup(t *testing.T) { t.Fatalf("Failed to create bootstrap file: %v", err) } - // Create an xDS client with the above bootstrap configuration and a short - // idle channel expiry timeout. This ensures that connections to lower - // priority servers get closed quickly, for the test to verify. + // Create an xDS client with the above bootstrap configuration. xdsC, close, err := xdsclient.NewForTesting(xdsclient.OptionsForTesting{ - Name: t.Name(), - Contents: bootstrapContents, - IdleChannelExpiryTimeout: defaultTestIdleChannelExpiryTimeout, + Name: t.Name(), + Contents: bootstrapContents, }) if err != nil { t.Fatalf("Failed to create xDS client: %v", err) diff --git a/xds/internal/xdsclient/tests/helpers_test.go b/xds/internal/xdsclient/tests/helpers_test.go index 5c4175b04df0..ab060d539a57 100644 --- a/xds/internal/xdsclient/tests/helpers_test.go +++ b/xds/internal/xdsclient/tests/helpers_test.go @@ -36,10 +36,9 @@ func Test(t *testing.T) { } const ( - defaultTestWatchExpiryTimeout = 500 * time.Millisecond - defaultTestIdleChannelExpiryTimeout = 50 * time.Millisecond - defaultTestTimeout = 10 * time.Second - defaultTestShortTimeout = 10 * time.Millisecond // For events expected to *not* happen. + defaultTestWatchExpiryTimeout = 500 * time.Millisecond + defaultTestTimeout = 10 * time.Second + defaultTestShortTimeout = 10 * time.Millisecond // For events expected to *not* happen. ldsName = "xdsclient-test-lds-resource" rdsName = "xdsclient-test-rds-resource" diff --git a/xds/internal/xdsclient/tests/lds_watchers_test.go b/xds/internal/xdsclient/tests/lds_watchers_test.go index 7b49b9b17b74..b05b9caf4adc 100644 --- a/xds/internal/xdsclient/tests/lds_watchers_test.go +++ b/xds/internal/xdsclient/tests/lds_watchers_test.go @@ -1029,7 +1029,6 @@ func (s) TestLDSWatch_NACKError(t *testing.T) { t.Fatalf("Failed to update management server with resources: %v, err: %v", resources, err) } - // Verify that the expected error is propagated to the watcher. // Verify that the expected error is propagated to the existing watcher. if err := verifyUnknownListenerError(ctx, lw.updateCh, wantListenerNACKErr); err != nil { t.Fatal(err) @@ -1039,45 +1038,18 @@ func (s) TestLDSWatch_NACKError(t *testing.T) { lw2 := newListenerWatcher() ldsCancel2 := xdsresource.WatchListener(client, ldsName, lw2) defer ldsCancel2() - // Verify that the expected error is propagated to the existing watcher. if err := verifyUnknownListenerError(ctx, lw2.updateCh, wantListenerNACKErr); err != nil { t.Fatal(err) } } -// TestLDSWatch_ResourceCaching_WithNACKError covers the case where a watch is -// registered for a resource which is already present in the cache with an old -// good update as well as latest NACK error. The test verifies that new watcher -// receives both good update and error without a new resource request being -// sent to the management server. -func TestLDSWatch_ResourceCaching_NACKError(t *testing.T) { - firstRequestReceived := false - firstAckReceived := grpcsync.NewEvent() - secondAckReceived := grpcsync.NewEvent() - secondRequestReceived := grpcsync.NewEvent() - - mgmtServer := e2e.StartManagementServer(t, e2e.ManagementServerOptions{ - OnStreamRequest: func(id int64, req *v3discoverypb.DiscoveryRequest) error { - // The first request has an empty version string. - if !firstRequestReceived && req.GetVersionInfo() == "" { - firstRequestReceived = true - return nil - } - // The first ack has a non-empty version string. - if !firstAckReceived.HasFired() && req.GetVersionInfo() != "" { - firstAckReceived.Fire() - return nil - } - // The second ack has a non-empty version string. - if !secondAckReceived.HasFired() && req.GetVersionInfo() != "" { - secondAckReceived.Fire() - return nil - } - // Any requests after the first request and two acks, are not expected. - secondRequestReceived.Fire() - return nil - }, - }) +// Tests the scenario where a watch registered for a resource results in a good +// update followed by a bad update. This results in the resource cache +// containing both the old good update and the latest NACK error. The test +// verifies that a when a new watch is registered for the same resource, the new +// watcher receives the good update followed by the NACK error. +func (s) TestLDSWatch_ResourceCaching_NACKError(t *testing.T) { + mgmtServer := e2e.StartManagementServer(t, e2e.ManagementServerOptions{}) nodeID := uuid.New().String() bc := e2e.DefaultBootstrapContents(t, nodeID, mgmtServer.Address) @@ -1150,16 +1122,6 @@ func TestLDSWatch_ResourceCaching_NACKError(t *testing.T) { if err := verifyUnknownListenerError(ctx, lw2.updateCh, wantListenerNACKErr); err != nil { t.Fatal(err) } - - // No request should get sent out as part of this watch. - sCtx, sCancel := context.WithTimeout(ctx, defaultTestShortTimeout) - defer sCancel() - select { - case <-sCtx.Done(): - case <-secondRequestReceived.Done(): - t.Fatal("xdsClient sent out request instead of using update from cache") - default: - } } // TestLDSWatch_PartialValid covers the case where a response from the From cc161defefac6266c861f61bb1ee97dabb682f9c Mon Sep 17 00:00:00 2001 From: Arjan Singh Bal <46515553+arjan-bal@users.noreply.github.com> Date: Mon, 16 Dec 2024 10:18:25 +0530 Subject: [PATCH 32/57] xds: Add support for multiple addresses per endpoint (#7858) --- .../weightedroundrobin/weightedroundrobin.go | 7 + internal/envconfig/xds.go | 6 + internal/testutils/xds/e2e/clientresources.go | 21 +- stats/opentelemetry/e2e_test.go | 4 +- test/xds/xds_client_custom_lb_test.go | 4 +- test/xds/xds_client_priority_locality_test.go | 14 +- .../clusterimpl/tests/balancer_test.go | 6 +- .../clusterresolver/clusterresolver.go | 30 ++- .../balancer/clusterresolver/configbuilder.go | 67 ++--- .../clusterresolver/configbuilder_test.go | 143 ++++++----- .../clusterresolver/e2e_test/eds_impl_test.go | 240 ++++++++++++++++-- .../clusterresolver/resource_resolver.go | 5 +- .../clusterresolver/resource_resolver_dns.go | 26 +- .../ringhash/e2e/ringhash_balancer_test.go | 12 +- xds/internal/balancer/wrrlocality/balancer.go | 7 + xds/internal/internal.go | 6 + .../xdsclient/tests/eds_watchers_test.go | 20 +- .../tests/federation_watchers_test.go | 2 +- .../xdsclient/tests/resource_update_test.go | 8 +- .../xdsclient/xdsresource/type_eds.go | 2 +- .../xdsclient/xdsresource/unmarshal_eds.go | 19 +- .../xdsresource/unmarshal_eds_test.go | 227 +++++++++++++---- 22 files changed, 632 insertions(+), 244 deletions(-) diff --git a/balancer/weightedroundrobin/weightedroundrobin.go b/balancer/weightedroundrobin/weightedroundrobin.go index 8741fdad19dc..258cdd5db280 100644 --- a/balancer/weightedroundrobin/weightedroundrobin.go +++ b/balancer/weightedroundrobin/weightedroundrobin.go @@ -56,6 +56,13 @@ func SetAddrInfo(addr resolver.Address, addrInfo AddrInfo) resolver.Address { return addr } +// SetAddrInfoInEndpoint returns a copy of endpoint in which the Attributes +// field is updated with addrInfo. +func SetAddrInfoInEndpoint(endpoint resolver.Endpoint, addrInfo AddrInfo) resolver.Endpoint { + endpoint.Attributes = endpoint.Attributes.WithValue(attributeKey{}, addrInfo) + return endpoint +} + // GetAddrInfo returns the AddrInfo stored in the BalancerAttributes field of // addr. func GetAddrInfo(addr resolver.Address) AddrInfo { diff --git a/internal/envconfig/xds.go b/internal/envconfig/xds.go index 29f234acb1b9..9afeb444d453 100644 --- a/internal/envconfig/xds.go +++ b/internal/envconfig/xds.go @@ -53,4 +53,10 @@ var ( // C2PResolverTestOnlyTrafficDirectorURI is the TD URI for testing. C2PResolverTestOnlyTrafficDirectorURI = os.Getenv("GRPC_TEST_ONLY_GOOGLE_C2P_RESOLVER_TRAFFIC_DIRECTOR_URI") + + // XDSDualstackEndpointsEnabled is true if gRPC should read the + // "additional addresses" in the xDS endpoint resource. + // TODO: https://github.com/grpc/grpc-go/issues/7866 - Control this using + // an env variable when all LB policies handle endpoints. + XDSDualstackEndpointsEnabled = false ) diff --git a/internal/testutils/xds/e2e/clientresources.go b/internal/testutils/xds/e2e/clientresources.go index 526f22aeab71..bf063a2e3bea 100644 --- a/internal/testutils/xds/e2e/clientresources.go +++ b/internal/testutils/xds/e2e/clientresources.go @@ -692,9 +692,9 @@ type LocalityOptions struct { // BackendOptions contains options to configure individual backends in a // locality. type BackendOptions struct { - // Port number on which the backend is accepting connections. All backends + // Ports on which the backend is accepting connections. All backends // are expected to run on localhost, hence host name is not stored here. - Port uint32 + Ports []uint32 // Health status of the backend. Default is UNKNOWN which is treated the // same as HEALTHY. HealthStatus v3corepb.HealthStatus @@ -722,7 +722,7 @@ type EndpointOptions struct { func DefaultEndpoint(clusterName string, host string, ports []uint32) *v3endpointpb.ClusterLoadAssignment { var bOpts []BackendOptions for _, p := range ports { - bOpts = append(bOpts, BackendOptions{Port: p, Weight: 1}) + bOpts = append(bOpts, BackendOptions{Ports: []uint32{p}, Weight: 1}) } return EndpointResourceWithOptions(EndpointOptions{ ClusterName: clusterName, @@ -747,15 +747,28 @@ func EndpointResourceWithOptions(opts EndpointOptions) *v3endpointpb.ClusterLoad if b.Weight == 0 { b.Weight = 1 } + additionalAddresses := make([]*v3endpointpb.Endpoint_AdditionalAddress, len(b.Ports)-1) + for i, p := range b.Ports[1:] { + additionalAddresses[i] = &v3endpointpb.Endpoint_AdditionalAddress{ + Address: &v3corepb.Address{Address: &v3corepb.Address_SocketAddress{ + SocketAddress: &v3corepb.SocketAddress{ + Protocol: v3corepb.SocketAddress_TCP, + Address: opts.Host, + PortSpecifier: &v3corepb.SocketAddress_PortValue{PortValue: p}, + }}, + }, + } + } lbEndpoints = append(lbEndpoints, &v3endpointpb.LbEndpoint{ HostIdentifier: &v3endpointpb.LbEndpoint_Endpoint{Endpoint: &v3endpointpb.Endpoint{ Address: &v3corepb.Address{Address: &v3corepb.Address_SocketAddress{ SocketAddress: &v3corepb.SocketAddress{ Protocol: v3corepb.SocketAddress_TCP, Address: opts.Host, - PortSpecifier: &v3corepb.SocketAddress_PortValue{PortValue: b.Port}, + PortSpecifier: &v3corepb.SocketAddress_PortValue{PortValue: b.Ports[0]}, }, }}, + AdditionalAddresses: additionalAddresses, }}, HealthStatus: b.HealthStatus, LoadBalancingWeight: &wrapperspb.UInt32Value{Value: b.Weight}, diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index e56c0fe94805..ac671e2982a3 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -436,7 +436,7 @@ func (s) TestWRRMetrics(t *testing.T) { Host: "localhost", Localities: []e2e.LocalityOptions{ { - Backends: []e2e.BackendOptions{{Port: port1}, {Port: port2}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{port1}}, {Ports: []uint32{port2}}}, Weight: 1, }, }, @@ -473,7 +473,7 @@ func (s) TestWRRMetrics(t *testing.T) { // scheduler. receivedExpectedMetrics := grpcsync.NewEvent() go func() { - for !receivedExpectedMetrics.HasFired() { + for !receivedExpectedMetrics.HasFired() && ctx.Err() == nil { client.EmptyCall(ctx, &testpb.Empty{}) time.Sleep(2 * time.Millisecond) } diff --git a/test/xds/xds_client_custom_lb_test.go b/test/xds/xds_client_custom_lb_test.go index 8d87a89753c7..2c881301cca2 100644 --- a/test/xds/xds_client_custom_lb_test.go +++ b/test/xds/xds_client_custom_lb_test.go @@ -238,11 +238,11 @@ func (s) TestWrrLocality(t *testing.T) { Host: "localhost", Localities: []e2e.LocalityOptions{ { - Backends: []e2e.BackendOptions{{Port: port1}, {Port: port2}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{port1}}, {Ports: []uint32{port2}}}, Weight: 1, }, { - Backends: []e2e.BackendOptions{{Port: port3}, {Port: port4}, {Port: port5}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{port3}}, {Ports: []uint32{port4}}, {Ports: []uint32{port5}}}, Weight: 2, }, }, diff --git a/test/xds/xds_client_priority_locality_test.go b/test/xds/xds_client_priority_locality_test.go index 0f7c5e21f3ba..607e6ea859b6 100644 --- a/test/xds/xds_client_priority_locality_test.go +++ b/test/xds/xds_client_priority_locality_test.go @@ -95,14 +95,14 @@ func (s) TestClientSideXDS_LocalityChangesPriority(t *testing.T) { Name: "my-locality-1", Weight: 1000000, Priority: 0, - Backends: []e2e.BackendOptions{{Port: ports[0]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[0]}}}, Locality: locality1, }, { Name: "my-locality-2", Weight: 1000000, Priority: 1, - Backends: []e2e.BackendOptions{{Port: ports[1]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[1]}}}, Locality: locality2, }, }, @@ -138,14 +138,14 @@ func (s) TestClientSideXDS_LocalityChangesPriority(t *testing.T) { Name: "my-locality-1", Weight: 500000, Priority: 0, - Backends: []e2e.BackendOptions{{Port: testutils.ParsePort(t, backend0.Address)}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{testutils.ParsePort(t, backend0.Address)}}}, Locality: locality1, }, { Name: "my-locality-2", Weight: 500000, Priority: 0, - Backends: []e2e.BackendOptions{{Port: testutils.ParsePort(t, backend1.Address)}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{testutils.ParsePort(t, backend1.Address)}}}, Locality: locality2, }, }, @@ -167,14 +167,14 @@ func (s) TestClientSideXDS_LocalityChangesPriority(t *testing.T) { Name: "my-locality-1", Weight: 499884, Priority: 0, - Backends: []e2e.BackendOptions{{Port: testutils.ParsePort(t, backend0.Address)}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{testutils.ParsePort(t, backend0.Address)}}}, Locality: locality1, }, { Name: "my-locality-2", Weight: 500115, Priority: 0, - Backends: []e2e.BackendOptions{{Port: testutils.ParsePort(t, backend1.Address)}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{testutils.ParsePort(t, backend1.Address)}}}, Locality: locality2, }, }, @@ -197,7 +197,7 @@ func (s) TestClientSideXDS_LocalityChangesPriority(t *testing.T) { Name: "my-locality-2", Weight: 1000000, Priority: 0, - Backends: []e2e.BackendOptions{{Port: testutils.ParsePort(t, backend1.Address)}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{testutils.ParsePort(t, backend1.Address)}}}, Locality: locality2, }, }, diff --git a/xds/internal/balancer/clusterimpl/tests/balancer_test.go b/xds/internal/balancer/clusterimpl/tests/balancer_test.go index f7d91d19597a..a8e7ac7a10ad 100644 --- a/xds/internal/balancer/clusterimpl/tests/balancer_test.go +++ b/xds/internal/balancer/clusterimpl/tests/balancer_test.go @@ -141,7 +141,7 @@ func (s) TestConfigUpdateWithSameLoadReportingServerConfig(t *testing.T) { Host: "localhost", Localities: []e2e.LocalityOptions{ { - Backends: []e2e.BackendOptions{{Port: testutils.ParsePort(t, server.Address)}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{testutils.ParsePort(t, server.Address)}}}, Weight: 1, }, }, @@ -252,13 +252,13 @@ func (s) TestLoadReportingPickFirstMultiLocality(t *testing.T) { Localities: []e2e.LocalityOptions{ { Backends: []e2e.BackendOptions{ - {Port: testutils.ParsePort(t, server1.Address)}, + {Ports: []uint32{testutils.ParsePort(t, server1.Address)}}, }, Weight: 1, }, { Backends: []e2e.BackendOptions{ - {Port: testutils.ParsePort(t, server2.Address)}, + {Ports: []uint32{testutils.ParsePort(t, server2.Address)}}, }, Weight: 2, }, diff --git a/xds/internal/balancer/clusterresolver/clusterresolver.go b/xds/internal/balancer/clusterresolver/clusterresolver.go index ae2c5fe957a2..f0a8905d374b 100644 --- a/xds/internal/balancer/clusterresolver/clusterresolver.go +++ b/xds/internal/balancer/clusterresolver/clusterresolver.go @@ -234,7 +234,7 @@ func (b *clusterResolverBalancer) updateChildConfig() { b.child = newChildBalancer(b.priorityBuilder, b.cc, b.bOpts) } - childCfgBytes, addrs, err := buildPriorityConfigJSON(b.priorities, &b.config.xdsLBPolicy) + childCfgBytes, endpoints, err := buildPriorityConfigJSON(b.priorities, &b.config.xdsLBPolicy) if err != nil { b.logger.Warningf("Failed to build child policy config: %v", err) return @@ -248,15 +248,33 @@ func (b *clusterResolverBalancer) updateChildConfig() { b.logger.Infof("Built child policy config: %s", pretty.ToJSON(childCfg)) } - endpoints := make([]resolver.Endpoint, len(addrs)) - for i, a := range addrs { - endpoints[i].Attributes = a.BalancerAttributes - endpoints[i].Addresses = []resolver.Address{a} + flattenedAddrs := make([]resolver.Address, len(endpoints)) + for i := range endpoints { + for j := range endpoints[i].Addresses { + addr := endpoints[i].Addresses[j] + addr.BalancerAttributes = endpoints[i].Attributes + // If the endpoint has multiple addresses, only the first is added + // to the flattened address list. This ensures that LB policies + // that don't support endpoints create only one subchannel to a + // backend. + if j == 0 { + flattenedAddrs[i] = addr + } + // BalancerAttributes need to be present in endpoint addresses. This + // temporary workaround is required to make load reporting work + // with the old pickfirst policy which creates SubConns with multiple + // addresses. Since the addresses can be from different localities, + // an Address.BalancerAttribute is used to identify the locality of the + // address used by the transport. This workaround can be removed once + // the old pickfirst is removed. + // See https://github.com/grpc/grpc-go/issues/7339 + endpoints[i].Addresses[j] = addr + } } if err := b.child.UpdateClientConnState(balancer.ClientConnState{ ResolverState: resolver.State{ Endpoints: endpoints, - Addresses: addrs, + Addresses: flattenedAddrs, ServiceConfig: b.configRaw, Attributes: b.attrsWithClient, }, diff --git a/xds/internal/balancer/clusterresolver/configbuilder.go b/xds/internal/balancer/clusterresolver/configbuilder.go index f62b8e6c8eb5..683f973b3fbf 100644 --- a/xds/internal/balancer/clusterresolver/configbuilder.go +++ b/xds/internal/balancer/clusterresolver/configbuilder.go @@ -48,8 +48,8 @@ type priorityConfig struct { mechanism DiscoveryMechanism // edsResp is set only if type is EDS. edsResp xdsresource.EndpointsUpdate - // addresses is set only if type is DNS. - addresses []string + // endpoints is set only if type is DNS. + endpoints []resolver.Endpoint // Each discovery mechanism has a name generator so that the child policies // can reuse names between updates (EDS updates for example). childNameGen *nameGenerator @@ -71,8 +71,8 @@ type priorityConfig struct { // ┌──────▼─────┐ ┌─────▼──────┐ // │xDSLBPolicy │ │xDSLBPolicy │ (Locality and Endpoint picking layer) // └────────────┘ └────────────┘ -func buildPriorityConfigJSON(priorities []priorityConfig, xdsLBPolicy *internalserviceconfig.BalancerConfig) ([]byte, []resolver.Address, error) { - pc, addrs, err := buildPriorityConfig(priorities, xdsLBPolicy) +func buildPriorityConfigJSON(priorities []priorityConfig, xdsLBPolicy *internalserviceconfig.BalancerConfig) ([]byte, []resolver.Endpoint, error) { + pc, endpoints, err := buildPriorityConfig(priorities, xdsLBPolicy) if err != nil { return nil, nil, fmt.Errorf("failed to build priority config: %v", err) } @@ -80,23 +80,23 @@ func buildPriorityConfigJSON(priorities []priorityConfig, xdsLBPolicy *internals if err != nil { return nil, nil, fmt.Errorf("failed to marshal built priority config struct into json: %v", err) } - return ret, addrs, nil + return ret, endpoints, nil } -func buildPriorityConfig(priorities []priorityConfig, xdsLBPolicy *internalserviceconfig.BalancerConfig) (*priority.LBConfig, []resolver.Address, error) { +func buildPriorityConfig(priorities []priorityConfig, xdsLBPolicy *internalserviceconfig.BalancerConfig) (*priority.LBConfig, []resolver.Endpoint, error) { var ( - retConfig = &priority.LBConfig{Children: make(map[string]*priority.Child)} - retAddrs []resolver.Address + retConfig = &priority.LBConfig{Children: make(map[string]*priority.Child)} + retEndpoints []resolver.Endpoint ) for _, p := range priorities { switch p.mechanism.Type { case DiscoveryMechanismTypeEDS: - names, configs, addrs, err := buildClusterImplConfigForEDS(p.childNameGen, p.edsResp, p.mechanism, xdsLBPolicy) + names, configs, endpoints, err := buildClusterImplConfigForEDS(p.childNameGen, p.edsResp, p.mechanism, xdsLBPolicy) if err != nil { return nil, nil, err } retConfig.Priorities = append(retConfig.Priorities, names...) - retAddrs = append(retAddrs, addrs...) + retEndpoints = append(retEndpoints, endpoints...) odCfgs := convertClusterImplMapToOutlierDetection(configs, p.mechanism.outlierDetection) for n, c := range odCfgs { retConfig.Children[n] = &priority.Child{ @@ -107,9 +107,9 @@ func buildPriorityConfig(priorities []priorityConfig, xdsLBPolicy *internalservi } continue case DiscoveryMechanismTypeLogicalDNS: - name, config, addrs := buildClusterImplConfigForDNS(p.childNameGen, p.addresses, p.mechanism) + name, config, endpoints := buildClusterImplConfigForDNS(p.childNameGen, p.endpoints, p.mechanism) retConfig.Priorities = append(retConfig.Priorities, name) - retAddrs = append(retAddrs, addrs...) + retEndpoints = append(retEndpoints, endpoints...) odCfg := makeClusterImplOutlierDetectionChild(config, p.mechanism.outlierDetection) retConfig.Children[name] = &priority.Child{ Config: &internalserviceconfig.BalancerConfig{Name: outlierdetection.Name, Config: odCfg}, @@ -120,7 +120,7 @@ func buildPriorityConfig(priorities []priorityConfig, xdsLBPolicy *internalservi continue } } - return retConfig, retAddrs, nil + return retConfig, retEndpoints, nil } func convertClusterImplMapToOutlierDetection(ciCfgs map[string]*clusterimpl.LBConfig, odCfg outlierdetection.LBConfig) map[string]*outlierdetection.LBConfig { @@ -137,19 +137,19 @@ func makeClusterImplOutlierDetectionChild(ciCfg *clusterimpl.LBConfig, odCfg out return &odCfgRet } -func buildClusterImplConfigForDNS(g *nameGenerator, addrStrs []string, mechanism DiscoveryMechanism) (string, *clusterimpl.LBConfig, []resolver.Address) { +func buildClusterImplConfigForDNS(g *nameGenerator, endpoints []resolver.Endpoint, mechanism DiscoveryMechanism) (string, *clusterimpl.LBConfig, []resolver.Endpoint) { // Endpoint picking policy for DNS is hardcoded to pick_first. const childPolicy = "pick_first" - retAddrs := make([]resolver.Address, 0, len(addrStrs)) + retEndpoints := make([]resolver.Endpoint, len(endpoints)) pName := fmt.Sprintf("priority-%v", g.prefix) - for _, addrStr := range addrStrs { - retAddrs = append(retAddrs, hierarchy.Set(resolver.Address{Addr: addrStr}, []string{pName})) + for i, e := range endpoints { + retEndpoints[i] = hierarchy.SetInEndpoint(e, []string{pName}) } return pName, &clusterimpl.LBConfig{ Cluster: mechanism.Cluster, TelemetryLabels: mechanism.TelemetryLabels, ChildPolicy: &internalserviceconfig.BalancerConfig{Name: childPolicy}, - }, retAddrs + }, retEndpoints } // buildClusterImplConfigForEDS returns a list of cluster_impl configs, one for @@ -161,7 +161,7 @@ func buildClusterImplConfigForDNS(g *nameGenerator, addrStrs []string, mechanism // - map{"p0":p0_config, "p1":p1_config} // - [p0_address_0, p0_address_1, p1_address_0, p1_address_1] // - p0 addresses' hierarchy attributes are set to p0 -func buildClusterImplConfigForEDS(g *nameGenerator, edsResp xdsresource.EndpointsUpdate, mechanism DiscoveryMechanism, xdsLBPolicy *internalserviceconfig.BalancerConfig) ([]string, map[string]*clusterimpl.LBConfig, []resolver.Address, error) { +func buildClusterImplConfigForEDS(g *nameGenerator, edsResp xdsresource.EndpointsUpdate, mechanism DiscoveryMechanism, xdsLBPolicy *internalserviceconfig.BalancerConfig) ([]string, map[string]*clusterimpl.LBConfig, []resolver.Endpoint, error) { drops := make([]clusterimpl.DropConfig, 0, len(edsResp.Drops)) for _, d := range edsResp.Drops { drops = append(drops, clusterimpl.DropConfig{ @@ -183,17 +183,17 @@ func buildClusterImplConfigForEDS(g *nameGenerator, edsResp xdsresource.Endpoint } retNames := g.generate(priorities) retConfigs := make(map[string]*clusterimpl.LBConfig, len(retNames)) - var retAddrs []resolver.Address + var retEndpoints []resolver.Endpoint for i, pName := range retNames { priorityLocalities := priorities[i] - cfg, addrs, err := priorityLocalitiesToClusterImpl(priorityLocalities, pName, mechanism, drops, xdsLBPolicy) + cfg, endpoints, err := priorityLocalitiesToClusterImpl(priorityLocalities, pName, mechanism, drops, xdsLBPolicy) if err != nil { return nil, nil, nil, err } retConfigs[pName] = cfg - retAddrs = append(retAddrs, addrs...) + retEndpoints = append(retEndpoints, endpoints...) } - return retNames, retConfigs, retAddrs, nil + return retNames, retConfigs, retEndpoints, nil } // groupLocalitiesByPriority returns the localities grouped by priority. @@ -244,8 +244,8 @@ func dedupSortedIntSlice(a []int) []int { // priority), and generates a cluster impl policy config, and a list of // addresses with their path hierarchy set to [priority-name, locality-name], so // priority and the xDS LB Policy know which child policy each address is for. -func priorityLocalitiesToClusterImpl(localities []xdsresource.Locality, priorityName string, mechanism DiscoveryMechanism, drops []clusterimpl.DropConfig, xdsLBPolicy *internalserviceconfig.BalancerConfig) (*clusterimpl.LBConfig, []resolver.Address, error) { - var addrs []resolver.Address +func priorityLocalitiesToClusterImpl(localities []xdsresource.Locality, priorityName string, mechanism DiscoveryMechanism, drops []clusterimpl.DropConfig, xdsLBPolicy *internalserviceconfig.BalancerConfig) (*clusterimpl.LBConfig, []resolver.Endpoint, error) { + var retEndpoints []resolver.Endpoint for _, locality := range localities { var lw uint32 = 1 if locality.Weight != 0 { @@ -262,21 +262,24 @@ func priorityLocalitiesToClusterImpl(localities []xdsresource.Locality, priority if endpoint.HealthStatus != xdsresource.EndpointHealthStatusHealthy && endpoint.HealthStatus != xdsresource.EndpointHealthStatusUnknown { continue } - addr := resolver.Address{Addr: endpoint.Address} - addr = hierarchy.Set(addr, []string{priorityName, localityStr}) - addr = internal.SetLocalityID(addr, locality.ID) + resolverEndpoint := resolver.Endpoint{} + for _, as := range endpoint.Addresses { + resolverEndpoint.Addresses = append(resolverEndpoint.Addresses, resolver.Address{Addr: as}) + } + resolverEndpoint = hierarchy.SetInEndpoint(resolverEndpoint, []string{priorityName, localityStr}) + resolverEndpoint = internal.SetLocalityIDInEndpoint(resolverEndpoint, locality.ID) // "To provide the xds_wrr_locality load balancer information about // locality weights received from EDS, the cluster resolver will // populate a new locality weight attribute for each address The // attribute will have the weight (as an integer) of the locality // the address is part of." - A52 - addr = wrrlocality.SetAddrInfo(addr, wrrlocality.AddrInfo{LocalityWeight: lw}) + resolverEndpoint = wrrlocality.SetAddrInfoInEndpoint(resolverEndpoint, wrrlocality.AddrInfo{LocalityWeight: lw}) var ew uint32 = 1 if endpoint.Weight != 0 { ew = endpoint.Weight } - addr = weightedroundrobin.SetAddrInfo(addr, weightedroundrobin.AddrInfo{Weight: lw * ew}) - addrs = append(addrs, addr) + resolverEndpoint = weightedroundrobin.SetAddrInfoInEndpoint(resolverEndpoint, weightedroundrobin.AddrInfo{Weight: lw * ew}) + retEndpoints = append(retEndpoints, resolverEndpoint) } } return &clusterimpl.LBConfig{ @@ -287,5 +290,5 @@ func priorityLocalitiesToClusterImpl(localities []xdsresource.Locality, priority TelemetryLabels: mechanism.TelemetryLabels, DropCategories: drops, ChildPolicy: xdsLBPolicy, - }, addrs, nil + }, retEndpoints, nil } diff --git a/xds/internal/balancer/clusterresolver/configbuilder_test.go b/xds/internal/balancer/clusterresolver/configbuilder_test.go index 01055886865c..e7dc2b781b5e 100644 --- a/xds/internal/balancer/clusterresolver/configbuilder_test.go +++ b/xds/internal/balancer/clusterresolver/configbuilder_test.go @@ -51,23 +51,23 @@ const ( testDropCategory = "test-drops" testDropOverMillion = 1 - localityCount = 5 - addressPerLocality = 2 + localityCount = 5 + endpointPerLocality = 2 ) var ( - testLocalityIDs []internal.LocalityID - testAddressStrs [][]string - testEndpoints [][]xdsresource.Endpoint + testLocalityIDs []internal.LocalityID + testResolverEndpoints [][]resolver.Endpoint + testEndpoints [][]xdsresource.Endpoint testLocalitiesP0, testLocalitiesP1 []xdsresource.Locality - addrCmpOpts = cmp.Options{ + endpointCmpOpts = cmp.Options{ cmp.AllowUnexported(attributes.Attributes{}), - cmp.Transformer("SortAddrs", func(in []resolver.Address) []resolver.Address { - out := append([]resolver.Address(nil), in...) // Copy input to avoid mutating it + cmp.Transformer("SortEndpoints", func(in []resolver.Endpoint) []resolver.Endpoint { + out := append([]resolver.Endpoint(nil), in...) // Copy input to avoid mutating it sort.Slice(out, func(i, j int) bool { - return out[i].Addr < out[j].Addr + return out[i].Addresses[0].Addr < out[j].Addresses[0].Addr }) return out }), @@ -85,18 +85,22 @@ func init() { for i := 0; i < localityCount; i++ { testLocalityIDs = append(testLocalityIDs, internal.LocalityID{Zone: fmt.Sprintf("test-zone-%d", i)}) var ( - addrs []string - ends []xdsresource.Endpoint + endpoints []resolver.Endpoint + ends []xdsresource.Endpoint ) - for j := 0; j < addressPerLocality; j++ { + for j := 0; j < endpointPerLocality; j++ { addr := fmt.Sprintf("addr-%d-%d", i, j) - addrs = append(addrs, addr) + endpoints = append(endpoints, resolver.Endpoint{Addresses: []resolver.Address{{Addr: addr}}}) ends = append(ends, xdsresource.Endpoint{ - Address: addr, HealthStatus: xdsresource.EndpointHealthStatusHealthy, + Addresses: []string{ + addr, + fmt.Sprintf("addr-%d-%d-additional-1", i, j), + fmt.Sprintf("addr-%d-%d-additional-2", i, j), + }, }) } - testAddressStrs = append(testAddressStrs, addrs) + testResolverEndpoints = append(testResolverEndpoints, endpoints) testEndpoints = append(testEndpoints, ends) } @@ -171,7 +175,7 @@ func TestBuildPriorityConfigJSON(t *testing.T) { mechanism: DiscoveryMechanism{ Type: DiscoveryMechanismTypeLogicalDNS, }, - addresses: testAddressStrs[4], + endpoints: testResolverEndpoints[4], childNameGen: newNameGenerator(1), }, }, nil) @@ -226,7 +230,7 @@ func TestBuildPriorityConfig(t *testing.T) { Type: DiscoveryMechanismTypeLogicalDNS, outlierDetection: noopODCfg, }, - addresses: testAddressStrs[4], + endpoints: testResolverEndpoints[4], childNameGen: newNameGenerator(1), }, }, nil) @@ -301,7 +305,7 @@ func TestBuildPriorityConfig(t *testing.T) { } func TestBuildClusterImplConfigForDNS(t *testing.T) { - gotName, gotConfig, gotAddrs := buildClusterImplConfigForDNS(newNameGenerator(3), testAddressStrs[0], DiscoveryMechanism{Cluster: testClusterName2, Type: DiscoveryMechanismTypeLogicalDNS}) + gotName, gotConfig, gotEndpoints := buildClusterImplConfigForDNS(newNameGenerator(3), testResolverEndpoints[0], DiscoveryMechanism{Cluster: testClusterName2, Type: DiscoveryMechanismTypeLogicalDNS}) wantName := "priority-3" wantConfig := &clusterimpl.LBConfig{ Cluster: testClusterName2, @@ -309,9 +313,11 @@ func TestBuildClusterImplConfigForDNS(t *testing.T) { Name: "pick_first", }, } - wantAddrs := []resolver.Address{ - hierarchy.Set(resolver.Address{Addr: testAddressStrs[0][0]}, []string{"priority-3"}), - hierarchy.Set(resolver.Address{Addr: testAddressStrs[0][1]}, []string{"priority-3"}), + e1 := resolver.Endpoint{Addresses: []resolver.Address{{Addr: testEndpoints[0][0].Addresses[0]}}} + e2 := resolver.Endpoint{Addresses: []resolver.Address{{Addr: testEndpoints[0][1].Addresses[0]}}} + wantEndpoints := []resolver.Endpoint{ + hierarchy.SetInEndpoint(e1, []string{"priority-3"}), + hierarchy.SetInEndpoint(e2, []string{"priority-3"}), } if diff := cmp.Diff(gotName, wantName); diff != "" { @@ -320,7 +326,7 @@ func TestBuildClusterImplConfigForDNS(t *testing.T) { if diff := cmp.Diff(gotConfig, wantConfig); diff != "" { t.Errorf("buildClusterImplConfigForDNS() diff (-got +want) %v", diff) } - if diff := cmp.Diff(gotAddrs, wantAddrs, addrCmpOpts); diff != "" { + if diff := cmp.Diff(gotEndpoints, wantEndpoints, endpointCmpOpts); diff != "" { t.Errorf("buildClusterImplConfigForDNS() diff (-got +want) %v", diff) } } @@ -334,7 +340,7 @@ func TestBuildClusterImplConfigForEDS(t *testing.T) { t.Fatalf("Failed to create LRS server config for testing: %v", err) } - gotNames, gotConfigs, gotAddrs, _ := buildClusterImplConfigForEDS( + gotNames, gotConfigs, gotEndpoints, _ := buildClusterImplConfigForEDS( newNameGenerator(2), xdsresource.EndpointsUpdate{ Drops: []xdsresource.OverloadDropConfig{ @@ -408,15 +414,15 @@ func TestBuildClusterImplConfigForEDS(t *testing.T) { }, }, } - wantAddrs := []resolver.Address{ - testAddrWithAttrs(testAddressStrs[0][0], 20, 1, "priority-2-0", &testLocalityIDs[0]), - testAddrWithAttrs(testAddressStrs[0][1], 20, 1, "priority-2-0", &testLocalityIDs[0]), - testAddrWithAttrs(testAddressStrs[1][0], 80, 1, "priority-2-0", &testLocalityIDs[1]), - testAddrWithAttrs(testAddressStrs[1][1], 80, 1, "priority-2-0", &testLocalityIDs[1]), - testAddrWithAttrs(testAddressStrs[2][0], 20, 1, "priority-2-1", &testLocalityIDs[2]), - testAddrWithAttrs(testAddressStrs[2][1], 20, 1, "priority-2-1", &testLocalityIDs[2]), - testAddrWithAttrs(testAddressStrs[3][0], 80, 1, "priority-2-1", &testLocalityIDs[3]), - testAddrWithAttrs(testAddressStrs[3][1], 80, 1, "priority-2-1", &testLocalityIDs[3]), + wantEndpoints := []resolver.Endpoint{ + testEndpointWithAttrs(testEndpoints[0][0].Addresses, 20, 1, "priority-2-0", &testLocalityIDs[0]), + testEndpointWithAttrs(testEndpoints[0][1].Addresses, 20, 1, "priority-2-0", &testLocalityIDs[0]), + testEndpointWithAttrs(testEndpoints[1][0].Addresses, 80, 1, "priority-2-0", &testLocalityIDs[1]), + testEndpointWithAttrs(testEndpoints[1][1].Addresses, 80, 1, "priority-2-0", &testLocalityIDs[1]), + testEndpointWithAttrs(testEndpoints[2][0].Addresses, 20, 1, "priority-2-1", &testLocalityIDs[2]), + testEndpointWithAttrs(testEndpoints[2][1].Addresses, 20, 1, "priority-2-1", &testLocalityIDs[2]), + testEndpointWithAttrs(testEndpoints[3][0].Addresses, 80, 1, "priority-2-1", &testLocalityIDs[3]), + testEndpointWithAttrs(testEndpoints[3][1].Addresses, 80, 1, "priority-2-1", &testLocalityIDs[3]), } if diff := cmp.Diff(gotNames, wantNames); diff != "" { @@ -425,7 +431,7 @@ func TestBuildClusterImplConfigForEDS(t *testing.T) { if diff := cmp.Diff(gotConfigs, wantConfigs); diff != "" { t.Errorf("buildClusterImplConfigForEDS() diff (-got +want) %v", diff) } - if diff := cmp.Diff(gotAddrs, wantAddrs, addrCmpOpts); diff != "" { + if diff := cmp.Diff(gotEndpoints, wantEndpoints, endpointCmpOpts); diff != "" { t.Errorf("buildClusterImplConfigForEDS() diff (-got +want) %v", diff) } @@ -526,29 +532,29 @@ func TestDedupSortedIntSlice(t *testing.T) { func TestPriorityLocalitiesToClusterImpl(t *testing.T) { tests := []struct { - name string - localities []xdsresource.Locality - priorityName string - mechanism DiscoveryMechanism - childPolicy *iserviceconfig.BalancerConfig - wantConfig *clusterimpl.LBConfig - wantAddrs []resolver.Address - wantErr bool + name string + localities []xdsresource.Locality + priorityName string + mechanism DiscoveryMechanism + childPolicy *iserviceconfig.BalancerConfig + wantConfig *clusterimpl.LBConfig + wantEndpoints []resolver.Endpoint + wantErr bool }{{ name: "round robin as child, no LRS", localities: []xdsresource.Locality{ { Endpoints: []xdsresource.Endpoint{ - {Address: "addr-1-1", HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 90}, - {Address: "addr-1-2", HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 10}, + {Addresses: []string{"addr-1-1"}, HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 90}, + {Addresses: []string{"addr-1-2"}, HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 10}, }, ID: internal.LocalityID{Zone: "test-zone-1"}, Weight: 20, }, { Endpoints: []xdsresource.Endpoint{ - {Address: "addr-2-1", HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 90}, - {Address: "addr-2-2", HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 10}, + {Addresses: []string{"addr-2-1"}, HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 90}, + {Addresses: []string{"addr-2-2"}, HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 10}, }, ID: internal.LocalityID{Zone: "test-zone-2"}, Weight: 80, @@ -567,11 +573,11 @@ func TestPriorityLocalitiesToClusterImpl(t *testing.T) { EDSServiceName: testEDSService, ChildPolicy: &iserviceconfig.BalancerConfig{Name: roundrobin.Name}, }, - wantAddrs: []resolver.Address{ - testAddrWithAttrs("addr-1-1", 20, 90, "test-priority", &internal.LocalityID{Zone: "test-zone-1"}), - testAddrWithAttrs("addr-1-2", 20, 10, "test-priority", &internal.LocalityID{Zone: "test-zone-1"}), - testAddrWithAttrs("addr-2-1", 80, 90, "test-priority", &internal.LocalityID{Zone: "test-zone-2"}), - testAddrWithAttrs("addr-2-2", 80, 10, "test-priority", &internal.LocalityID{Zone: "test-zone-2"}), + wantEndpoints: []resolver.Endpoint{ + testEndpointWithAttrs([]string{"addr-1-1"}, 20, 90, "test-priority", &internal.LocalityID{Zone: "test-zone-1"}), + testEndpointWithAttrs([]string{"addr-1-2"}, 20, 10, "test-priority", &internal.LocalityID{Zone: "test-zone-1"}), + testEndpointWithAttrs([]string{"addr-2-1"}, 80, 90, "test-priority", &internal.LocalityID{Zone: "test-zone-2"}), + testEndpointWithAttrs([]string{"addr-2-2"}, 80, 10, "test-priority", &internal.LocalityID{Zone: "test-zone-2"}), }, }, { @@ -579,16 +585,16 @@ func TestPriorityLocalitiesToClusterImpl(t *testing.T) { localities: []xdsresource.Locality{ { Endpoints: []xdsresource.Endpoint{ - {Address: "addr-1-1", HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 90}, - {Address: "addr-1-2", HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 10}, + {Addresses: []string{"addr-1-1"}, HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 90}, + {Addresses: []string{"addr-1-2"}, HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 10}, }, ID: internal.LocalityID{Zone: "test-zone-1"}, Weight: 20, }, { Endpoints: []xdsresource.Endpoint{ - {Address: "addr-2-1", HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 90}, - {Address: "addr-2-2", HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 10}, + {Addresses: []string{"addr-2-1"}, HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 90}, + {Addresses: []string{"addr-2-2"}, HealthStatus: xdsresource.EndpointHealthStatusHealthy, Weight: 10}, }, ID: internal.LocalityID{Zone: "test-zone-2"}, Weight: 80, @@ -603,11 +609,11 @@ func TestPriorityLocalitiesToClusterImpl(t *testing.T) { Config: &ringhash.LBConfig{MinRingSize: 1, MaxRingSize: 2}, }, }, - wantAddrs: []resolver.Address{ - testAddrWithAttrs("addr-1-1", 20, 90, "test-priority", &internal.LocalityID{Zone: "test-zone-1"}), - testAddrWithAttrs("addr-1-2", 20, 10, "test-priority", &internal.LocalityID{Zone: "test-zone-1"}), - testAddrWithAttrs("addr-2-1", 80, 90, "test-priority", &internal.LocalityID{Zone: "test-zone-2"}), - testAddrWithAttrs("addr-2-2", 80, 10, "test-priority", &internal.LocalityID{Zone: "test-zone-2"}), + wantEndpoints: []resolver.Endpoint{ + testEndpointWithAttrs([]string{"addr-1-1"}, 20, 90, "test-priority", &internal.LocalityID{Zone: "test-zone-1"}), + testEndpointWithAttrs([]string{"addr-1-2"}, 20, 10, "test-priority", &internal.LocalityID{Zone: "test-zone-1"}), + testEndpointWithAttrs([]string{"addr-2-1"}, 80, 90, "test-priority", &internal.LocalityID{Zone: "test-zone-2"}), + testEndpointWithAttrs([]string{"addr-2-2"}, 80, 10, "test-priority", &internal.LocalityID{Zone: "test-zone-2"}), }, }, } @@ -620,7 +626,7 @@ func TestPriorityLocalitiesToClusterImpl(t *testing.T) { if diff := cmp.Diff(got, tt.wantConfig); diff != "" { t.Errorf("localitiesToWeightedTarget() diff (-got +want) %v", diff) } - if diff := cmp.Diff(got1, tt.wantAddrs, cmp.AllowUnexported(attributes.Attributes{})); diff != "" { + if diff := cmp.Diff(got1, tt.wantEndpoints, cmp.AllowUnexported(attributes.Attributes{})); diff != "" { t.Errorf("localitiesToWeightedTarget() diff (-got +want) %v", diff) } }) @@ -635,17 +641,20 @@ func assertString(f func() (string, error)) string { return s } -func testAddrWithAttrs(addrStr string, localityWeight, endpointWeight uint32, priority string, lID *internal.LocalityID) resolver.Address { - addr := resolver.Address{Addr: addrStr} +func testEndpointWithAttrs(addrStrs []string, localityWeight, endpointWeight uint32, priority string, lID *internal.LocalityID) resolver.Endpoint { + endpoint := resolver.Endpoint{} + for _, a := range addrStrs { + endpoint.Addresses = append(endpoint.Addresses, resolver.Address{Addr: a}) + } path := []string{priority} if lID != nil { path = append(path, assertString(lID.ToString)) - addr = internal.SetLocalityID(addr, *lID) + endpoint = internal.SetLocalityIDInEndpoint(endpoint, *lID) } - addr = hierarchy.Set(addr, path) - addr = wrrlocality.SetAddrInfo(addr, wrrlocality.AddrInfo{LocalityWeight: localityWeight}) - addr = weightedroundrobin.SetAddrInfo(addr, weightedroundrobin.AddrInfo{Weight: localityWeight * endpointWeight}) - return addr + endpoint = hierarchy.SetInEndpoint(endpoint, path) + endpoint = wrrlocality.SetAddrInfoInEndpoint(endpoint, wrrlocality.AddrInfo{LocalityWeight: localityWeight}) + endpoint = weightedroundrobin.SetAddrInfoInEndpoint(endpoint, weightedroundrobin.AddrInfo{Weight: localityWeight * endpointWeight}) + return endpoint } func TestConvertClusterImplMapToOutlierDetection(t *testing.T) { diff --git a/xds/internal/balancer/clusterresolver/e2e_test/eds_impl_test.go b/xds/internal/balancer/clusterresolver/e2e_test/eds_impl_test.go index af6224585aca..89760f6fd23e 100644 --- a/xds/internal/balancer/clusterresolver/e2e_test/eds_impl_test.go +++ b/xds/internal/balancer/clusterresolver/e2e_test/eds_impl_test.go @@ -20,6 +20,7 @@ import ( "context" "errors" "fmt" + "net" "strings" "testing" "time" @@ -27,9 +28,13 @@ import ( "github.com/google/go-cmp/cmp" "github.com/google/uuid" "google.golang.org/grpc" + "google.golang.org/grpc/balancer" + "google.golang.org/grpc/balancer/roundrobin" "google.golang.org/grpc/codes" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/internal" + "google.golang.org/grpc/internal/balancer/stub" + "google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/stubserver" "google.golang.org/grpc/internal/testutils" @@ -141,7 +146,7 @@ func (s) TestEDS_OneLocality(t *testing.T) { resources := clientEndpointsResource(nodeID, edsServiceName, []e2e.LocalityOptions{{ Name: localityName1, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[0]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[0]}}}, }}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -196,7 +201,7 @@ func (s) TestEDS_OneLocality(t *testing.T) { resources = clientEndpointsResource(nodeID, edsServiceName, []e2e.LocalityOptions{{ Name: localityName1, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[0]}, {Port: ports[1]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[0]}}, {Ports: []uint32{ports[1]}}}, }}) if err := managementServer.Update(ctx, resources); err != nil { t.Fatal(err) @@ -210,7 +215,7 @@ func (s) TestEDS_OneLocality(t *testing.T) { resources = clientEndpointsResource(nodeID, edsServiceName, []e2e.LocalityOptions{{ Name: localityName1, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[1]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[1]}}}, }}) if err := managementServer.Update(ctx, resources); err != nil { t.Fatal(err) @@ -223,7 +228,7 @@ func (s) TestEDS_OneLocality(t *testing.T) { resources = clientEndpointsResource(nodeID, edsServiceName, []e2e.LocalityOptions{{ Name: localityName1, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[2]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[2]}}}, }}) if err := managementServer.Update(ctx, resources); err != nil { t.Fatal(err) @@ -270,12 +275,12 @@ func (s) TestEDS_MultipleLocalities(t *testing.T) { { Name: localityName1, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[0]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[0]}}}, }, { Name: localityName2, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[1]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[1]}}}, }, }) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) @@ -332,17 +337,17 @@ func (s) TestEDS_MultipleLocalities(t *testing.T) { { Name: localityName1, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[0]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[0]}}}, }, { Name: localityName2, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[1]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[1]}}}, }, { Name: localityName3, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[2]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[2]}}}, }, }) if err := managementServer.Update(ctx, resources); err != nil { @@ -358,12 +363,12 @@ func (s) TestEDS_MultipleLocalities(t *testing.T) { { Name: localityName2, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[1]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[1]}}}, }, { Name: localityName3, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[2]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[2]}}}, }, }) if err := managementServer.Update(ctx, resources); err != nil { @@ -380,12 +385,12 @@ func (s) TestEDS_MultipleLocalities(t *testing.T) { { Name: localityName2, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[1]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[1]}}}, }, { Name: localityName3, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[2]}, {Port: ports[3]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[2]}}, {Ports: []uint32{ports[3]}}}, }, }) if err := managementServer.Update(ctx, resources); err != nil { @@ -421,24 +426,24 @@ func (s) TestEDS_EndpointsHealth(t *testing.T) { Name: localityName1, Weight: 1, Backends: []e2e.BackendOptions{ - {Port: ports[0], HealthStatus: v3corepb.HealthStatus_UNKNOWN}, - {Port: ports[1], HealthStatus: v3corepb.HealthStatus_HEALTHY}, - {Port: ports[2], HealthStatus: v3corepb.HealthStatus_UNHEALTHY}, - {Port: ports[3], HealthStatus: v3corepb.HealthStatus_DRAINING}, - {Port: ports[4], HealthStatus: v3corepb.HealthStatus_TIMEOUT}, - {Port: ports[5], HealthStatus: v3corepb.HealthStatus_DEGRADED}, + {Ports: []uint32{ports[0]}, HealthStatus: v3corepb.HealthStatus_UNKNOWN}, + {Ports: []uint32{ports[1]}, HealthStatus: v3corepb.HealthStatus_HEALTHY}, + {Ports: []uint32{ports[2]}, HealthStatus: v3corepb.HealthStatus_UNHEALTHY}, + {Ports: []uint32{ports[3]}, HealthStatus: v3corepb.HealthStatus_DRAINING}, + {Ports: []uint32{ports[4]}, HealthStatus: v3corepb.HealthStatus_TIMEOUT}, + {Ports: []uint32{ports[5]}, HealthStatus: v3corepb.HealthStatus_DEGRADED}, }, }, { Name: localityName2, Weight: 1, Backends: []e2e.BackendOptions{ - {Port: ports[6], HealthStatus: v3corepb.HealthStatus_UNKNOWN}, - {Port: ports[7], HealthStatus: v3corepb.HealthStatus_HEALTHY}, - {Port: ports[8], HealthStatus: v3corepb.HealthStatus_UNHEALTHY}, - {Port: ports[9], HealthStatus: v3corepb.HealthStatus_DRAINING}, - {Port: ports[10], HealthStatus: v3corepb.HealthStatus_TIMEOUT}, - {Port: ports[11], HealthStatus: v3corepb.HealthStatus_DEGRADED}, + {Ports: []uint32{ports[6]}, HealthStatus: v3corepb.HealthStatus_UNKNOWN}, + {Ports: []uint32{ports[7]}, HealthStatus: v3corepb.HealthStatus_HEALTHY}, + {Ports: []uint32{ports[8]}, HealthStatus: v3corepb.HealthStatus_UNHEALTHY}, + {Ports: []uint32{ports[9]}, HealthStatus: v3corepb.HealthStatus_DRAINING}, + {Ports: []uint32{ports[10]}, HealthStatus: v3corepb.HealthStatus_TIMEOUT}, + {Ports: []uint32{ports[11]}, HealthStatus: v3corepb.HealthStatus_DEGRADED}, }, }, }) @@ -567,7 +572,7 @@ func (s) TestEDS_EmptyUpdate(t *testing.T) { resources = clientEndpointsResource(nodeID, edsServiceName, []e2e.LocalityOptions{{ Name: localityName1, Weight: 1, - Backends: []e2e.BackendOptions{{Port: ports[0]}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{ports[0]}}}, }}) if err := managementServer.Update(ctx, resources); err != nil { t.Fatal(err) @@ -911,7 +916,7 @@ func (s) TestEDS_BadUpdateWithoutPreviousGoodUpdate(t *testing.T) { resources := clientEndpointsResource(nodeID, edsServiceName, []e2e.LocalityOptions{{ Name: localityName1, Weight: 1, - Backends: []e2e.BackendOptions{{Port: testutils.ParsePort(t, server.Address)}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{testutils.ParsePort(t, server.Address)}}}, }}) resources.Endpoints[0].Endpoints[0].LbEndpoints[0].LoadBalancingWeight = &wrapperspb.UInt32Value{Value: 0} ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) @@ -984,7 +989,7 @@ func (s) TestEDS_BadUpdateWithPreviousGoodUpdate(t *testing.T) { resources := clientEndpointsResource(nodeID, edsServiceName, []e2e.LocalityOptions{{ Name: localityName1, Weight: 1, - Backends: []e2e.BackendOptions{{Port: testutils.ParsePort(t, server.Address)}}, + Backends: []e2e.BackendOptions{{Ports: []uint32{testutils.ParsePort(t, server.Address)}}}, }}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -1138,3 +1143,182 @@ func waitForProducedZeroAddressesError(ctx context.Context, t *testing.T, client } return errors.New("timeout when waiting for RPCs to fail with UNAVAILABLE status and produced zero addresses") } + +// Test runs a server which listens on multiple ports. The test updates xds resouce +// cache to contain a single endpoint with multiple addresses. The test intercepts +// the resolver updates sent to the petiole policy and verifies that the +// additional endpoint addresses are correctly propagated. +func (s) TestEDS_EndpointWithMultipleAddresses(t *testing.T) { + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + // Start a backend server which listens to multiple ports to simulate a + // backend with multiple addresses. + server := &stubserver.StubServer{ + EmptyCallF: func(context.Context, *testpb.Empty) (*testpb.Empty, error) { return &testpb.Empty{}, nil }, + UnaryCallF: func(context.Context, *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { + return &testpb.SimpleResponse{}, nil + }, + } + lis1, err := net.Listen("tcp", "localhost:0") + if err != nil { + t.Fatalf("Failed to create listener: %v", err) + } + defer lis1.Close() + lis2, err := net.Listen("tcp", "localhost:0") + if err != nil { + t.Fatalf("Failed to create listener: %v", err) + } + defer lis2.Close() + lis3, err := net.Listen("tcp", "localhost:0") + if err != nil { + t.Fatalf("Failed to create listener: %v", err) + } + defer lis3.Close() + + server.Listener = lis1 + if err := server.StartServer(); err != nil { + t.Fatalf("Failed to start stub server: %v", err) + } + go server.S.Serve(lis2) + go server.S.Serve(lis3) + + t.Logf("Started test service backend at addresses %q, %q, %q", lis1.Addr(), lis2.Addr(), lis3.Addr()) + + ports := []uint32{ + testutils.ParsePort(t, lis1.Addr().String()), + testutils.ParsePort(t, lis2.Addr().String()), + testutils.ParsePort(t, lis3.Addr().String()), + } + + testCases := []struct { + name string + dualstackEndpointsEnabled bool + wantEndpointPorts []uint32 + wantAddrPorts []uint32 + }{ + { + name: "flag_enabled", + dualstackEndpointsEnabled: true, + wantEndpointPorts: ports, + wantAddrPorts: ports[:1], + }, + { + name: "flag_disabled", + wantEndpointPorts: ports[:1], + wantAddrPorts: ports[:1], + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + origDualstackEndpointsEnabled := envconfig.XDSDualstackEndpointsEnabled + defer func() { + envconfig.XDSDualstackEndpointsEnabled = origDualstackEndpointsEnabled + }() + envconfig.XDSDualstackEndpointsEnabled = tc.dualstackEndpointsEnabled + + // Wrap the round robin balancer to intercept resolver updates. + originalRRBuilder := balancer.Get(roundrobin.Name) + defer func() { + balancer.Register(originalRRBuilder) + }() + resolverUpdateCh := make(chan resolver.State, 1) + stub.Register(roundrobin.Name, stub.BalancerFuncs{ + Init: func(bd *stub.BalancerData) { + bd.Data = originalRRBuilder.Build(bd.ClientConn, bd.BuildOptions) + }, + Close: func(bd *stub.BalancerData) { + bd.Data.(balancer.Balancer).Close() + }, + UpdateClientConnState: func(bd *stub.BalancerData, ccs balancer.ClientConnState) error { + resolverUpdateCh <- ccs.ResolverState + return bd.Data.(balancer.Balancer).UpdateClientConnState(ccs) + }, + }) + + // Spin up a management server to receive xDS resources from. + mgmtServer := e2e.StartManagementServer(t, e2e.ManagementServerOptions{}) + + // Create bootstrap configuration pointing to the above management server. + nodeID := uuid.New().String() + bootstrapContents := e2e.DefaultBootstrapContents(t, nodeID, mgmtServer.Address) + + // Create xDS resources for consumption by the test. We start off with a + // single backend in a single EDS locality. + resources := clientEndpointsResource(nodeID, edsServiceName, []e2e.LocalityOptions{{ + Name: localityName1, + Weight: 1, + Backends: []e2e.BackendOptions{{ + Ports: ports, + }}, + }}) + if err := mgmtServer.Update(ctx, resources); err != nil { + t.Fatal(err) + } + + // Create an xDS client talking to the above management server, configured + // with a short watch expiry timeout. + xdsClient, close, err := xdsclient.NewForTesting(xdsclient.OptionsForTesting{ + Name: t.Name(), + Contents: bootstrapContents, + }) + if err != nil { + t.Fatalf("Failed to create an xDS client: %v", err) + } + defer close() + + // Create a manual resolver and push a service config specifying the use of + // the cluster_resolver LB policy with a single discovery mechanism. + r := manual.NewBuilderWithScheme("whatever") + jsonSC := fmt.Sprintf(`{ + "loadBalancingConfig":[{ + "cluster_resolver_experimental":{ + "discoveryMechanisms": [{ + "cluster": "%s", + "type": "EDS", + "edsServiceName": "%s", + "outlierDetection": {} + }], + "xdsLbPolicy":[{"round_robin":{}}] + } + }] + }`, clusterName, edsServiceName) + scpr := internal.ParseServiceConfig.(func(string) *serviceconfig.ParseResult)(jsonSC) + r.InitialState(xdsclient.SetClient(resolver.State{ServiceConfig: scpr}, xdsClient)) + + cc, err := grpc.NewClient(r.Scheme()+":///test.service", grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(r)) + if err != nil { + t.Fatalf("failed to create new client for local test server: %v", err) + } + defer cc.Close() + client := testgrpc.NewTestServiceClient(cc) + if err := rrutil.CheckRoundRobinRPCs(ctx, client, []resolver.Address{{Addr: lis1.Addr().String()}}); err != nil { + t.Fatal(err) + } + + var rs resolver.State + select { + case rs = <-resolverUpdateCh: + case <-ctx.Done(): + t.Fatalf("Context timed out waiting for resolver update.") + } + + gotEndpointPorts := []uint32{} + for _, a := range rs.Endpoints[0].Addresses { + gotEndpointPorts = append(gotEndpointPorts, testutils.ParsePort(t, a.Addr)) + } + if diff := cmp.Diff(gotEndpointPorts, tc.wantEndpointPorts); diff != "" { + t.Errorf("Unexpected endpoint address ports in resolver update, diff (-got +want): %v", diff) + } + + gotAddrPorts := []uint32{} + for _, a := range rs.Addresses { + gotAddrPorts = append(gotAddrPorts, testutils.ParsePort(t, a.Addr)) + } + if diff := cmp.Diff(gotAddrPorts, tc.wantAddrPorts); diff != "" { + t.Errorf("Unexpected address ports in resolver update, diff (-got +want): %v", diff) + } + }) + } +} diff --git a/xds/internal/balancer/clusterresolver/resource_resolver.go b/xds/internal/balancer/clusterresolver/resource_resolver.go index 3bcfba8732a3..d9315c3acef5 100644 --- a/xds/internal/balancer/clusterresolver/resource_resolver.go +++ b/xds/internal/balancer/clusterresolver/resource_resolver.go @@ -24,6 +24,7 @@ import ( "google.golang.org/grpc/internal/grpclog" "google.golang.org/grpc/internal/grpcsync" + "google.golang.org/grpc/resolver" "google.golang.org/grpc/xds/internal/xdsclient/xdsresource" ) @@ -294,8 +295,8 @@ func (rr *resourceResolver) generateLocked(onDone xdsresource.OnDoneFunc) { switch uu := u.(type) { case xdsresource.EndpointsUpdate: ret = append(ret, priorityConfig{mechanism: rDM.dm, edsResp: uu, childNameGen: rDM.childNameGen}) - case []string: - ret = append(ret, priorityConfig{mechanism: rDM.dm, addresses: uu, childNameGen: rDM.childNameGen}) + case []resolver.Endpoint: + ret = append(ret, priorityConfig{mechanism: rDM.dm, endpoints: uu, childNameGen: rDM.childNameGen}) } } select { diff --git a/xds/internal/balancer/clusterresolver/resource_resolver_dns.go b/xds/internal/balancer/clusterresolver/resource_resolver_dns.go index cfc871d3b59d..5f7a21153057 100644 --- a/xds/internal/balancer/clusterresolver/resource_resolver_dns.go +++ b/xds/internal/balancer/clusterresolver/resource_resolver_dns.go @@ -47,7 +47,7 @@ type dnsDiscoveryMechanism struct { logger *grpclog.PrefixLogger mu sync.Mutex - addrs []string + endpoints []resolver.Endpoint updateReceived bool } @@ -103,7 +103,7 @@ func (dr *dnsDiscoveryMechanism) lastUpdate() (any, bool) { if !dr.updateReceived { return nil, false } - return dr.addrs, true + return dr.endpoints, true } func (dr *dnsDiscoveryMechanism) resolveNow() { @@ -133,23 +133,15 @@ func (dr *dnsDiscoveryMechanism) UpdateState(state resolver.State) error { } dr.mu.Lock() - var addrs []string - if len(state.Endpoints) > 0 { - // Assume 1 address per endpoint, which is how DNS is expected to - // behave. The slice will grow as needed, however. - addrs = make([]string, 0, len(state.Endpoints)) - for _, e := range state.Endpoints { - for _, a := range e.Addresses { - addrs = append(addrs, a.Addr) - } - } - } else { - addrs = make([]string, len(state.Addresses)) + var endpoints = state.Endpoints + if len(endpoints) == 0 { + endpoints = make([]resolver.Endpoint, len(state.Addresses)) for i, a := range state.Addresses { - addrs[i] = a.Addr + endpoints[i] = resolver.Endpoint{Addresses: []resolver.Address{a}} + endpoints[i].Attributes = a.BalancerAttributes } } - dr.addrs = addrs + dr.endpoints = endpoints dr.updateReceived = true dr.mu.Unlock() @@ -172,7 +164,7 @@ func (dr *dnsDiscoveryMechanism) ReportError(err error) { dr.mu.Unlock() return } - dr.addrs = nil + dr.endpoints = nil dr.updateReceived = true dr.mu.Unlock() diff --git a/xds/internal/balancer/ringhash/e2e/ringhash_balancer_test.go b/xds/internal/balancer/ringhash/e2e/ringhash_balancer_test.go index 94efa99dcdf0..afc60924b3f8 100644 --- a/xds/internal/balancer/ringhash/e2e/ringhash_balancer_test.go +++ b/xds/internal/balancer/ringhash/e2e/ringhash_balancer_test.go @@ -182,7 +182,7 @@ func backendOptions(t *testing.T, serverAddrs []string) []e2e.BackendOptions { var backendOpts []e2e.BackendOptions for _, addr := range serverAddrs { - backendOpts = append(backendOpts, e2e.BackendOptions{Port: testutils.ParsePort(t, addr)}) + backendOpts = append(backendOpts, e2e.BackendOptions{Ports: []uint32{testutils.ParsePort(t, addr)}}) } return backendOpts } @@ -872,9 +872,9 @@ func (s) TestRingHash_EndpointWeights(t *testing.T) { const clusterName = "cluster" backendOpts := []e2e.BackendOptions{ - {Port: testutils.ParsePort(t, backends[0])}, - {Port: testutils.ParsePort(t, backends[1])}, - {Port: testutils.ParsePort(t, backends[2]), Weight: 2}, + {Ports: []uint32{testutils.ParsePort(t, backends[0])}}, + {Ports: []uint32{testutils.ParsePort(t, backends[1])}}, + {Ports: []uint32{testutils.ParsePort(t, backends[2])}, Weight: 2}, } endpoints := e2e.EndpointResourceWithOptions(e2e.EndpointOptions{ @@ -1209,14 +1209,14 @@ func (s) TestRingHash_RandomHashingDistributionAccordingToLocalityAndEndpointWei Localities: []e2e.LocalityOptions{ { Backends: []e2e.BackendOptions{{ - Port: testutils.ParsePort(t, backends[0]), + Ports: []uint32{testutils.ParsePort(t, backends[0])}, Weight: endpoint1Weight, }}, Weight: locality1Weight, }, { Backends: []e2e.BackendOptions{{ - Port: testutils.ParsePort(t, backends[1]), + Ports: []uint32{testutils.ParsePort(t, backends[1])}, Weight: endpoint2Weight, }}, Weight: locality2Weight, diff --git a/xds/internal/balancer/wrrlocality/balancer.go b/xds/internal/balancer/wrrlocality/balancer.go index 943ee7806ba1..2b289a81143c 100644 --- a/xds/internal/balancer/wrrlocality/balancer.go +++ b/xds/internal/balancer/wrrlocality/balancer.go @@ -120,6 +120,13 @@ func SetAddrInfo(addr resolver.Address, addrInfo AddrInfo) resolver.Address { return addr } +// SetAddrInfoInEndpoint returns a copy of endpoint in which the Attributes +// field is updated with AddrInfo. +func SetAddrInfoInEndpoint(endpoint resolver.Endpoint, addrInfo AddrInfo) resolver.Endpoint { + endpoint.Attributes = endpoint.Attributes.WithValue(attributeKey{}, addrInfo) + return endpoint +} + func (a AddrInfo) String() string { return fmt.Sprintf("Locality Weight: %d", a.LocalityWeight) } diff --git a/xds/internal/internal.go b/xds/internal/internal.go index 1d8a6b03f1b3..74c919521551 100644 --- a/xds/internal/internal.go +++ b/xds/internal/internal.go @@ -86,6 +86,12 @@ func SetLocalityID(addr resolver.Address, l LocalityID) resolver.Address { return addr } +// SetLocalityIDInEndpoint sets locality ID in endpoint to l. +func SetLocalityIDInEndpoint(endpoint resolver.Endpoint, l LocalityID) resolver.Endpoint { + endpoint.Attributes = endpoint.Attributes.WithValue(localityKey, l) + return endpoint +} + // ResourceTypeMapForTesting maps TypeUrl to corresponding ResourceType. var ResourceTypeMapForTesting map[string]any diff --git a/xds/internal/xdsclient/tests/eds_watchers_test.go b/xds/internal/xdsclient/tests/eds_watchers_test.go index ef27178fac22..21021b8992bb 100644 --- a/xds/internal/xdsclient/tests/eds_watchers_test.go +++ b/xds/internal/xdsclient/tests/eds_watchers_test.go @@ -176,7 +176,7 @@ func (s) TestEDSWatch(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost1, edsPort1), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost1, edsPort1)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", @@ -199,7 +199,7 @@ func (s) TestEDSWatch(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost1, edsPort1), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost1, edsPort1)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", @@ -335,7 +335,7 @@ func (s) TestEDSWatch_TwoWatchesForSameResourceName(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost1, edsPort1), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost1, edsPort1)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", @@ -351,7 +351,7 @@ func (s) TestEDSWatch_TwoWatchesForSameResourceName(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost2, edsPort2), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost2, edsPort2)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", @@ -373,7 +373,7 @@ func (s) TestEDSWatch_TwoWatchesForSameResourceName(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost1, edsPort1), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost1, edsPort1)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", @@ -389,7 +389,7 @@ func (s) TestEDSWatch_TwoWatchesForSameResourceName(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost2, edsPort2), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost2, edsPort2)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", @@ -581,7 +581,7 @@ func (s) TestEDSWatch_ThreeWatchesForDifferentResourceNames(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost1, edsPort1), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost1, edsPort1)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", @@ -669,7 +669,7 @@ func (s) TestEDSWatch_ResourceCaching(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost1, edsPort1), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost1, edsPort1)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", @@ -795,7 +795,7 @@ func (s) TestEDSWatch_ValidResponseCancelsExpiryTimerBehavior(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost1, edsPort1), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost1, edsPort1)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", @@ -952,7 +952,7 @@ func (s) TestEDSWatch_PartialValid(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: fmt.Sprintf("%s:%d", edsHost1, edsPort1), Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{fmt.Sprintf("%s:%d", edsHost1, edsPort1)}, Weight: 1}}, ID: internal.LocalityID{ Region: "region-1", Zone: "zone-1", diff --git a/xds/internal/xdsclient/tests/federation_watchers_test.go b/xds/internal/xdsclient/tests/federation_watchers_test.go index 78f69518cd6a..65233c344733 100644 --- a/xds/internal/xdsclient/tests/federation_watchers_test.go +++ b/xds/internal/xdsclient/tests/federation_watchers_test.go @@ -287,7 +287,7 @@ func (s) TestFederation_EndpointsResourceContextParamOrder(t *testing.T) { update: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: "localhost:666", Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{"localhost:666"}, Weight: 1}}, Weight: 1, ID: internal.LocalityID{ Region: "region-1", diff --git a/xds/internal/xdsclient/tests/resource_update_test.go b/xds/internal/xdsclient/tests/resource_update_test.go index b493c820c774..0460385d0fb7 100644 --- a/xds/internal/xdsclient/tests/resource_update_test.go +++ b/xds/internal/xdsclient/tests/resource_update_test.go @@ -1077,13 +1077,13 @@ func (s) TestHandleEndpointsResponseFromManagementServer(t *testing.T) { wantUpdate: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: "addr1:314", Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{"addr1:314"}, Weight: 1}}, ID: internal.LocalityID{SubZone: "locality-1"}, Priority: 1, Weight: 1, }, { - Endpoints: []xdsresource.Endpoint{{Address: "addr2:159", Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{"addr2:159"}, Weight: 1}}, ID: internal.LocalityID{SubZone: "locality-2"}, Priority: 0, Weight: 1, @@ -1111,13 +1111,13 @@ func (s) TestHandleEndpointsResponseFromManagementServer(t *testing.T) { wantUpdate: xdsresource.EndpointsUpdate{ Localities: []xdsresource.Locality{ { - Endpoints: []xdsresource.Endpoint{{Address: "addr1:314", Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{"addr1:314"}, Weight: 1}}, ID: internal.LocalityID{SubZone: "locality-1"}, Priority: 1, Weight: 1, }, { - Endpoints: []xdsresource.Endpoint{{Address: "addr2:159", Weight: 1}}, + Endpoints: []xdsresource.Endpoint{{Addresses: []string{"addr2:159"}, Weight: 1}}, ID: internal.LocalityID{SubZone: "locality-2"}, Priority: 0, Weight: 1, diff --git a/xds/internal/xdsclient/xdsresource/type_eds.go b/xds/internal/xdsclient/xdsresource/type_eds.go index 1254d250c99b..f94a17e7c66a 100644 --- a/xds/internal/xdsclient/xdsresource/type_eds.go +++ b/xds/internal/xdsclient/xdsresource/type_eds.go @@ -49,7 +49,7 @@ const ( // Endpoint contains information of an endpoint. type Endpoint struct { - Address string + Addresses []string HealthStatus EndpointHealthStatus Weight uint32 } diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_eds.go b/xds/internal/xdsclient/xdsresource/unmarshal_eds.go index f65845b702c8..fd780d6632d2 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_eds.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_eds.go @@ -26,6 +26,7 @@ import ( v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3" v3endpointpb "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3" v3typepb "github.com/envoyproxy/go-control-plane/envoy/type/v3" + "google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/internal/pretty" "google.golang.org/grpc/xds/internal" "google.golang.org/protobuf/proto" @@ -93,14 +94,22 @@ func parseEndpoints(lbEndpoints []*v3endpointpb.LbEndpoint, uniqueEndpointAddrs } weight = w.GetValue() } - addr := parseAddress(lbEndpoint.GetEndpoint().GetAddress().GetSocketAddress()) - if uniqueEndpointAddrs[addr] { - return nil, fmt.Errorf("duplicate endpoint with the same address %s", addr) + addrs := []string{parseAddress(lbEndpoint.GetEndpoint().GetAddress().GetSocketAddress())} + if envconfig.XDSDualstackEndpointsEnabled { + for _, sa := range lbEndpoint.GetEndpoint().GetAdditionalAddresses() { + addrs = append(addrs, parseAddress(sa.GetAddress().GetSocketAddress())) + } + } + + for _, a := range addrs { + if uniqueEndpointAddrs[a] { + return nil, fmt.Errorf("duplicate endpoint with the same address %s", a) + } + uniqueEndpointAddrs[a] = true } - uniqueEndpointAddrs[addr] = true endpoints = append(endpoints, Endpoint{ HealthStatus: EndpointHealthStatus(lbEndpoint.GetHealthStatus()), - Address: addr, + Addresses: addrs, Weight: weight, }) } diff --git a/xds/internal/xdsclient/xdsresource/unmarshal_eds_test.go b/xds/internal/xdsclient/xdsresource/unmarshal_eds_test.go index f2419eac55a4..e8df0c3c3593 100644 --- a/xds/internal/xdsclient/xdsresource/unmarshal_eds_test.go +++ b/xds/internal/xdsclient/xdsresource/unmarshal_eds_test.go @@ -28,6 +28,8 @@ import ( v3discoverypb "github.com/envoyproxy/go-control-plane/envoy/service/discovery/v3" v3typepb "github.com/envoyproxy/go-control-plane/envoy/type/v3" "github.com/google/go-cmp/cmp" + "github.com/google/go-cmp/cmp/cmpopts" + "google.golang.org/grpc/internal/envconfig" "google.golang.org/grpc/internal/pretty" "google.golang.org/grpc/internal/testutils" "google.golang.org/grpc/xds/internal" @@ -47,8 +49,8 @@ func (s) TestEDSParseRespProto(t *testing.T) { name: "missing-priority", m: func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-1", 1, 0, []string{"addr1:314"}, nil) - clab0.addLocality("locality-2", 1, 2, []string{"addr2:159"}, nil) + clab0.addLocality("locality-1", 1, 0, []endpointOpts{{addrWithPort: "addr1:314"}}, nil) + clab0.addLocality("locality-2", 1, 2, []endpointOpts{{addrWithPort: "addr2:159"}}, nil) return clab0.Build() }(), want: EndpointsUpdate{}, @@ -58,7 +60,7 @@ func (s) TestEDSParseRespProto(t *testing.T) { name: "missing-locality-ID", m: func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("", 1, 0, []string{"addr1:314"}, nil) + clab0.addLocality("", 1, 0, []endpointOpts{{addrWithPort: "addr1:314"}}, nil) return clab0.Build() }(), want: EndpointsUpdate{}, @@ -68,7 +70,7 @@ func (s) TestEDSParseRespProto(t *testing.T) { name: "zero-endpoint-weight", m: func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-0", 1, 0, []string{"addr1:314"}, &addLocalityOptions{Weight: []uint32{0}}) + clab0.addLocality("locality-0", 1, 0, []endpointOpts{{addrWithPort: "addr1:314"}}, &addLocalityOptions{Weight: []uint32{0}}) return clab0.Build() }(), want: EndpointsUpdate{}, @@ -78,8 +80,8 @@ func (s) TestEDSParseRespProto(t *testing.T) { name: "duplicate-locality-in-the-same-priority", m: func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-0", 1, 0, []string{"addr1:314"}, nil) - clab0.addLocality("locality-0", 1, 0, []string{"addr1:314"}, nil) // Duplicate locality with the same priority. + clab0.addLocality("locality-0", 1, 0, []endpointOpts{{addrWithPort: "addr1:314"}}, nil) + clab0.addLocality("locality-0", 1, 0, []endpointOpts{{addrWithPort: "addr1:314"}}, nil) // Duplicate locality with the same priority. return clab0.Build() }(), want: EndpointsUpdate{}, @@ -89,10 +91,10 @@ func (s) TestEDSParseRespProto(t *testing.T) { name: "missing locality weight", m: func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-1", 0, 1, []string{"addr1:314"}, &addLocalityOptions{ + clab0.addLocality("locality-1", 0, 1, []endpointOpts{{addrWithPort: "addr1:314"}}, &addLocalityOptions{ Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_HEALTHY}, }) - clab0.addLocality("locality-2", 0, 0, []string{"addr2:159"}, &addLocalityOptions{ + clab0.addLocality("locality-2", 0, 0, []endpointOpts{{addrWithPort: "addr2:159"}}, &addLocalityOptions{ Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_HEALTHY}, }) return clab0.Build() @@ -103,9 +105,9 @@ func (s) TestEDSParseRespProto(t *testing.T) { name: "max sum of weights at the same priority exceeded", m: func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-1", 1, 0, []string{"addr1:314"}, nil) - clab0.addLocality("locality-2", 4294967295, 1, []string{"addr2:159"}, nil) - clab0.addLocality("locality-3", 1, 1, []string{"addr2:88"}, nil) + clab0.addLocality("locality-1", 1, 0, []endpointOpts{{addrWithPort: "addr1:314"}}, nil) + clab0.addLocality("locality-2", 4294967295, 1, []endpointOpts{{addrWithPort: "addr2:159"}}, nil) + clab0.addLocality("locality-3", 1, 1, []endpointOpts{{addrWithPort: "addr2:88"}}, nil) return clab0.Build() }(), want: EndpointsUpdate{}, @@ -115,8 +117,8 @@ func (s) TestEDSParseRespProto(t *testing.T) { name: "duplicate endpoint address", m: func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-1", 1, 1, []string{"addr:997"}, nil) - clab0.addLocality("locality-2", 1, 0, []string{"addr:997"}, nil) + clab0.addLocality("locality-1", 1, 1, []endpointOpts{{addrWithPort: "addr:997"}}, nil) + clab0.addLocality("locality-2", 1, 0, []endpointOpts{{addrWithPort: "addr:997"}}, nil) return clab0.Build() }(), want: EndpointsUpdate{}, @@ -126,11 +128,11 @@ func (s) TestEDSParseRespProto(t *testing.T) { name: "good", m: func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-1", 1, 1, []string{"addr1:314"}, &addLocalityOptions{ + clab0.addLocality("locality-1", 1, 1, []endpointOpts{{addrWithPort: "addr1:314"}}, &addLocalityOptions{ Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_UNHEALTHY}, Weight: []uint32{271}, }) - clab0.addLocality("locality-2", 1, 0, []string{"addr2:159"}, &addLocalityOptions{ + clab0.addLocality("locality-2", 1, 0, []endpointOpts{{addrWithPort: "addr2:159"}}, &addLocalityOptions{ Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_DRAINING}, Weight: []uint32{828}, }) @@ -141,7 +143,7 @@ func (s) TestEDSParseRespProto(t *testing.T) { Localities: []Locality{ { Endpoints: []Endpoint{{ - Address: "addr1:314", + Addresses: []string{"addr1:314"}, HealthStatus: EndpointHealthStatusUnhealthy, Weight: 271, }}, @@ -151,7 +153,7 @@ func (s) TestEDSParseRespProto(t *testing.T) { }, { Endpoints: []Endpoint{{ - Address: "addr2:159", + Addresses: []string{"addr2:159"}, HealthStatus: EndpointHealthStatusDraining, Weight: 828, }}, @@ -167,12 +169,12 @@ func (s) TestEDSParseRespProto(t *testing.T) { name: "good duplicate locality with different priority", m: func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-1", 1, 1, []string{"addr1:314"}, &addLocalityOptions{ + clab0.addLocality("locality-1", 1, 1, []endpointOpts{{addrWithPort: "addr1:314"}}, &addLocalityOptions{ Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_UNHEALTHY}, Weight: []uint32{271}, }) // Same locality name, but with different priority. - clab0.addLocality("locality-1", 1, 0, []string{"addr2:159"}, &addLocalityOptions{ + clab0.addLocality("locality-1", 1, 0, []endpointOpts{{addrWithPort: "addr2:159"}}, &addLocalityOptions{ Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_DRAINING}, Weight: []uint32{828}, }) @@ -183,7 +185,7 @@ func (s) TestEDSParseRespProto(t *testing.T) { Localities: []Locality{ { Endpoints: []Endpoint{{ - Address: "addr1:314", + Addresses: []string{"addr1:314"}, HealthStatus: EndpointHealthStatusUnhealthy, Weight: 271, }}, @@ -193,7 +195,7 @@ func (s) TestEDSParseRespProto(t *testing.T) { }, { Endpoints: []Endpoint{{ - Address: "addr2:159", + Addresses: []string{"addr2:159"}, HealthStatus: EndpointHealthStatusDraining, Weight: 828, }}, @@ -213,7 +215,118 @@ func (s) TestEDSParseRespProto(t *testing.T) { t.Errorf("parseEDSRespProto() error = %v, wantErr %v", err, tt.wantErr) return } - if d := cmp.Diff(got, tt.want); d != "" { + if d := cmp.Diff(got, tt.want, cmpopts.EquateEmpty()); d != "" { + t.Errorf("parseEDSRespProto() got = %v, want %v, diff: %v", got, tt.want, d) + } + }) + } +} + +func (s) TestEDSParseRespProtoAdditionalAddrs(t *testing.T) { + origDualstackEndpointsEnabled := envconfig.XDSDualstackEndpointsEnabled + defer func() { + envconfig.XDSDualstackEndpointsEnabled = origDualstackEndpointsEnabled + }() + envconfig.XDSDualstackEndpointsEnabled = true + + tests := []struct { + name string + m *v3endpointpb.ClusterLoadAssignment + want EndpointsUpdate + wantErr bool + }{ + { + name: "duplicate primary address in self additional addresses", + m: func() *v3endpointpb.ClusterLoadAssignment { + clab0 := newClaBuilder("test", nil) + clab0.addLocality("locality-1", 1, 0, []endpointOpts{{addrWithPort: "addr:998", additionalAddrWithPorts: []string{"addr:998"}}}, nil) + return clab0.Build() + }(), + want: EndpointsUpdate{}, + wantErr: true, + }, + { + name: "duplicate primary address in other locality additional addresses", + m: func() *v3endpointpb.ClusterLoadAssignment { + clab0 := newClaBuilder("test", nil) + clab0.addLocality("locality-1", 1, 1, []endpointOpts{{addrWithPort: "addr:997"}}, nil) + clab0.addLocality("locality-2", 1, 0, []endpointOpts{{addrWithPort: "addr:998", additionalAddrWithPorts: []string{"addr:997"}}}, nil) + return clab0.Build() + }(), + want: EndpointsUpdate{}, + wantErr: true, + }, + { + name: "duplicate additional address in self additional addresses", + m: func() *v3endpointpb.ClusterLoadAssignment { + clab0 := newClaBuilder("test", nil) + clab0.addLocality("locality-1", 1, 0, []endpointOpts{{addrWithPort: "addr:998", additionalAddrWithPorts: []string{"addr:999", "addr:999"}}}, nil) + return clab0.Build() + }(), + want: EndpointsUpdate{}, + wantErr: true, + }, + { + name: "duplicate additional address in other locality additional addresses", + m: func() *v3endpointpb.ClusterLoadAssignment { + clab0 := newClaBuilder("test", nil) + clab0.addLocality("locality-1", 1, 1, []endpointOpts{{addrWithPort: "addr:997", additionalAddrWithPorts: []string{"addr:1000"}}}, nil) + clab0.addLocality("locality-2", 1, 0, []endpointOpts{{addrWithPort: "addr:998", additionalAddrWithPorts: []string{"addr:1000"}}}, nil) + return clab0.Build() + }(), + want: EndpointsUpdate{}, + wantErr: true, + }, + { + name: "multiple localities", + m: func() *v3endpointpb.ClusterLoadAssignment { + clab0 := newClaBuilder("test", nil) + clab0.addLocality("locality-1", 1, 1, []endpointOpts{{addrWithPort: "addr1:997", additionalAddrWithPorts: []string{"addr1:1000"}}}, &addLocalityOptions{ + Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_UNHEALTHY}, + Weight: []uint32{271}, + }) + clab0.addLocality("locality-2", 1, 0, []endpointOpts{{addrWithPort: "addr2:998", additionalAddrWithPorts: []string{"addr2:1000"}}}, &addLocalityOptions{ + Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_HEALTHY}, + Weight: []uint32{828}, + }) + return clab0.Build() + }(), + want: EndpointsUpdate{ + Drops: nil, + Localities: []Locality{ + { + Endpoints: []Endpoint{{ + Addresses: []string{"addr1:997", "addr1:1000"}, + HealthStatus: EndpointHealthStatusUnhealthy, + Weight: 271, + }}, + ID: internal.LocalityID{SubZone: "locality-1"}, + Priority: 1, + Weight: 1, + }, + { + Endpoints: []Endpoint{{ + Addresses: []string{"addr2:998", "addr2:1000"}, + HealthStatus: EndpointHealthStatusHealthy, + Weight: 828, + }}, + ID: internal.LocalityID{SubZone: "locality-2"}, + Priority: 0, + Weight: 1, + }, + }, + }, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + got, err := parseEDSRespProto(tt.m) + if (err != nil) != tt.wantErr { + t.Errorf("parseEDSRespProto() error = %v, wantErr %v", err, tt.wantErr) + return + } + if d := cmp.Diff(got, tt.want, cmpopts.EquateEmpty()); d != "" { t.Errorf("parseEDSRespProto() got = %v, want %v, diff: %v", got, tt.want, d) } }) @@ -223,11 +336,11 @@ func (s) TestEDSParseRespProto(t *testing.T) { func (s) TestUnmarshalEndpoints(t *testing.T) { var v3EndpointsAny = testutils.MarshalAny(t, func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-1", 1, 1, []string{"addr1:314"}, &addLocalityOptions{ + clab0.addLocality("locality-1", 1, 1, []endpointOpts{{addrWithPort: "addr1:314"}}, &addLocalityOptions{ Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_UNHEALTHY}, Weight: []uint32{271}, }) - clab0.addLocality("locality-2", 1, 0, []string{"addr2:159"}, &addLocalityOptions{ + clab0.addLocality("locality-2", 1, 0, []endpointOpts{{addrWithPort: "addr2:159"}}, &addLocalityOptions{ Health: []v3corepb.HealthStatus{v3corepb.HealthStatus_DRAINING}, Weight: []uint32{828}, }) @@ -258,8 +371,8 @@ func (s) TestUnmarshalEndpoints(t *testing.T) { name: "bad endpoints resource", resource: testutils.MarshalAny(t, func() *v3endpointpb.ClusterLoadAssignment { clab0 := newClaBuilder("test", nil) - clab0.addLocality("locality-1", 1, 0, []string{"addr1:314"}, nil) - clab0.addLocality("locality-2", 1, 2, []string{"addr2:159"}, nil) + clab0.addLocality("locality-1", 1, 0, []endpointOpts{{addrWithPort: "addr1:314"}}, nil) + clab0.addLocality("locality-2", 1, 2, []endpointOpts{{addrWithPort: "addr2:159"}}, nil) return clab0.Build() }()), wantName: "test", @@ -274,7 +387,7 @@ func (s) TestUnmarshalEndpoints(t *testing.T) { Localities: []Locality{ { Endpoints: []Endpoint{{ - Address: "addr1:314", + Addresses: []string{"addr1:314"}, HealthStatus: EndpointHealthStatusUnhealthy, Weight: 271, }}, @@ -284,7 +397,7 @@ func (s) TestUnmarshalEndpoints(t *testing.T) { }, { Endpoints: []Endpoint{{ - Address: "addr2:159", + Addresses: []string{"addr2:159"}, HealthStatus: EndpointHealthStatusDraining, Weight: 828, }}, @@ -305,7 +418,7 @@ func (s) TestUnmarshalEndpoints(t *testing.T) { Localities: []Locality{ { Endpoints: []Endpoint{{ - Address: "addr1:314", + Addresses: []string{"addr1:314"}, HealthStatus: EndpointHealthStatusUnhealthy, Weight: 271, }}, @@ -315,7 +428,7 @@ func (s) TestUnmarshalEndpoints(t *testing.T) { }, { Endpoints: []Endpoint{{ - Address: "addr2:159", + Addresses: []string{"addr2:159"}, HealthStatus: EndpointHealthStatusDraining, Weight: 828, }}, @@ -379,29 +492,49 @@ type addLocalityOptions struct { Weight []uint32 } +type endpointOpts struct { + addrWithPort string + additionalAddrWithPorts []string +} + +func addressFromStr(addrWithPort string) *v3corepb.Address { + host, portStr, err := net.SplitHostPort(addrWithPort) + if err != nil { + panic("failed to split " + addrWithPort) + } + port, err := strconv.Atoi(portStr) + if err != nil { + panic("failed to atoi " + portStr) + } + + return &v3corepb.Address{ + Address: &v3corepb.Address_SocketAddress{ + SocketAddress: &v3corepb.SocketAddress{ + Protocol: v3corepb.SocketAddress_TCP, + Address: host, + PortSpecifier: &v3corepb.SocketAddress_PortValue{PortValue: uint32(port)}, + }, + }, + } +} + // addLocality adds a locality to the builder. -func (clab *claBuilder) addLocality(subzone string, weight uint32, priority uint32, addrsWithPort []string, opts *addLocalityOptions) { +func (clab *claBuilder) addLocality(subzone string, weight uint32, priority uint32, endpoints []endpointOpts, opts *addLocalityOptions) { var lbEndPoints []*v3endpointpb.LbEndpoint - for i, a := range addrsWithPort { - host, portStr, err := net.SplitHostPort(a) - if err != nil { - panic("failed to split " + a) - } - port, err := strconv.Atoi(portStr) - if err != nil { - panic("failed to atoi " + portStr) + for i, e := range endpoints { + var additionalAddrs []*v3endpointpb.Endpoint_AdditionalAddress + for _, a := range e.additionalAddrWithPorts { + additionalAddrs = append(additionalAddrs, &v3endpointpb.Endpoint_AdditionalAddress{ + Address: addressFromStr(a), + }) } - lbe := &v3endpointpb.LbEndpoint{ HostIdentifier: &v3endpointpb.LbEndpoint_Endpoint{ Endpoint: &v3endpointpb.Endpoint{ - Address: &v3corepb.Address{ - Address: &v3corepb.Address_SocketAddress{ - SocketAddress: &v3corepb.SocketAddress{ - Protocol: v3corepb.SocketAddress_TCP, - Address: host, - PortSpecifier: &v3corepb.SocketAddress_PortValue{ - PortValue: uint32(port)}}}}}}, + Address: addressFromStr(e.addrWithPort), + AdditionalAddresses: additionalAddrs, + }, + }, } if opts != nil { if i < len(opts.Health) { From d0716f9e629bfdf7f90923e063ffb650fd9f8f48 Mon Sep 17 00:00:00 2001 From: Zach Reyes <39203661+zasweq@users.noreply.github.com> Date: Mon, 16 Dec 2024 00:49:44 -0500 Subject: [PATCH 33/57] examples/features/csm_observability: Make CSM Observability example server listen on an IPV4 address (#7933) --- examples/features/csm_observability/server/main.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/features/csm_observability/server/main.go b/examples/features/csm_observability/server/main.go index 3cafe23316b3..7a87517cbde6 100644 --- a/examples/features/csm_observability/server/main.go +++ b/examples/features/csm_observability/server/main.go @@ -66,7 +66,7 @@ func main() { cleanup := csm.EnableObservability(context.Background(), opentelemetry.Options{MetricsOptions: opentelemetry.MetricsOptions{MeterProvider: provider}}) defer cleanup() - lis, err := net.Listen("tcp", ":"+*port) + lis, err := net.Listen("tcp4", "0.0.0.0:"+*port) if err != nil { log.Fatalf("Failed to listen: %v", err) } From e8055ea11f96238b0fb14b1062abb598c33fc9c3 Mon Sep 17 00:00:00 2001 From: Purnesh Dixit Date: Tue, 17 Dec 2024 09:00:58 +0530 Subject: [PATCH 34/57] grpcs: update `WithContextDialer` documentation to include using passthrough resolver (#7916) --- dialoptions.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dialoptions.go b/dialoptions.go index 7494ae591f16..f3a045296a46 100644 --- a/dialoptions.go +++ b/dialoptions.go @@ -428,6 +428,11 @@ func WithTimeout(d time.Duration) DialOption { // returned by f, gRPC checks the error's Temporary() method to decide if it // should try to reconnect to the network address. // +// Note that gRPC by default performs name resolution on the target passed to +// NewClient. To bypass name resolution and cause the target string to be +// passed directly to the dialer here instead, use the "passthrough" resolver +// by specifying it in the target string, e.g. "passthrough:target". +// // Note: All supported releases of Go (as of December 2023) override the OS // defaults for TCP keepalive time and interval to 15s. To enable TCP keepalive // with OS defaults for keepalive time and interval, use a net.Dialer that sets From b3bdacbb55852e26d7a9439a2b398411812564d7 Mon Sep 17 00:00:00 2001 From: pvsravani Date: Wed, 18 Dec 2024 18:14:42 +0000 Subject: [PATCH 35/57] test: switching to stubserver in tests instead of testservice (#7925) --- test/balancer_test.go | 52 ++++++++++++++++++++++++++----------------- 1 file changed, 31 insertions(+), 21 deletions(-) diff --git a/test/balancer_test.go b/test/balancer_test.go index bb45e11d97c1..9cb41e5a6a2c 100644 --- a/test/balancer_test.go +++ b/test/balancer_test.go @@ -417,11 +417,15 @@ func (s) TestAddressAttributesInNewSubConn(t *testing.T) { if err != nil { t.Fatal(err) } - - s := grpc.NewServer() - testgrpc.RegisterTestServiceServer(s, &testServer{}) - go s.Serve(lis) - defer s.Stop() + stub := &stubserver.StubServer{ + Listener: lis, + EmptyCallF: func(_ context.Context, _ *testpb.Empty) (*testpb.Empty, error) { + return &testpb.Empty{}, nil + }, + S: grpc.NewServer(), + } + stubserver.StartTestService(t, stub) + defer stub.S.Stop() t.Logf("Started gRPC server at %s...", lis.Addr().String()) creds := &attrTransportCreds{} @@ -548,15 +552,16 @@ func (s) TestServersSwap(t *testing.T) { if err != nil { t.Fatalf("Error while listening. Err: %v", err) } - s := grpc.NewServer() - ts := &funcServer{ - unaryCall: func(context.Context, *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { + + stub := &stubserver.StubServer{ + Listener: lis, + UnaryCallF: func(_ context.Context, _ *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { return &testpb.SimpleResponse{Username: username}, nil }, + S: grpc.NewServer(), } - testgrpc.RegisterTestServiceServer(s, ts) - go s.Serve(lis) - return lis.Addr().String(), s.Stop + stubserver.StartTestService(t, stub) + return lis.Addr().String(), stub.S.Stop } const one = "1" addr1, cleanup := reg(one) @@ -603,16 +608,16 @@ func (s) TestWaitForReady(t *testing.T) { if err != nil { t.Fatalf("Error while listening. Err: %v", err) } - s := grpc.NewServer() - defer s.Stop() const one = "1" - ts := &funcServer{ - unaryCall: func(context.Context, *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { + stub := &stubserver.StubServer{ + Listener: lis, + UnaryCallF: func(_ context.Context, _ *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { return &testpb.SimpleResponse{Username: one}, nil }, + S: grpc.NewServer(), } - testgrpc.RegisterTestServiceServer(s, ts) - go s.Serve(lis) + stubserver.StartTestService(t, stub) + defer stub.S.Stop() // Initialize client r := manual.NewBuilderWithScheme("whatever") @@ -740,10 +745,15 @@ func (s) TestAuthorityInBuildOptions(t *testing.T) { t.Fatal(err) } - s := grpc.NewServer() - testgrpc.RegisterTestServiceServer(s, &testServer{}) - go s.Serve(lis) - defer s.Stop() + stub := &stubserver.StubServer{ + Listener: lis, + EmptyCallF: func(_ context.Context, _ *testpb.Empty) (*testpb.Empty, error) { + return &testpb.Empty{}, nil + }, + S: grpc.NewServer(), + } + stubserver.StartTestService(t, stub) + defer stub.S.Stop() t.Logf("Started gRPC server at %s...", lis.Addr().String()) r := manual.NewBuilderWithScheme("whatever") From 56a14ba1f818a0ca297f5e21bf689064a714ee68 Mon Sep 17 00:00:00 2001 From: janardhanvissa <47281167+janardhanvissa@users.noreply.github.com> Date: Thu, 19 Dec 2024 03:43:10 +0530 Subject: [PATCH 36/57] cleanup: replace dial with newclient (#7920) --- balancer/grpclb/grpclb_test.go | 39 ++++++++++++++++++---------------- balancer/rls/picker_test.go | 12 +++++------ resolver_balancer_ext_test.go | 10 +++++---- test/creds_test.go | 27 +++++++++++------------ test/end2end_test.go | 2 +- 5 files changed, 48 insertions(+), 42 deletions(-) diff --git a/balancer/grpclb/grpclb_test.go b/balancer/grpclb/grpclb_test.go index 62dc947e0ee5..e234ace6576f 100644 --- a/balancer/grpclb/grpclb_test.go +++ b/balancer/grpclb/grpclb_test.go @@ -460,7 +460,7 @@ func (s) TestGRPCLB_Basic(t *testing.T) { } cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, dopts...) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend %v", err) } defer cc.Close() @@ -517,7 +517,7 @@ func (s) TestGRPCLB_Weighted(t *testing.T) { } cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, dopts...) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend %v", err) } defer cc.Close() @@ -597,7 +597,7 @@ func (s) TestGRPCLB_DropRequest(t *testing.T) { } cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, dopts...) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend %v", err) } defer cc.Close() testC := testgrpc.NewTestServiceClient(cc) @@ -769,7 +769,7 @@ func (s) TestGRPCLB_BalancerDisconnects(t *testing.T) { } cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, dopts...) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend %v", err) } defer cc.Close() testC := testgrpc.NewTestServiceClient(cc) @@ -940,7 +940,7 @@ func (s) TestGRPCLB_ExplicitFallback(t *testing.T) { } cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, dopts...) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend %v", err) } defer cc.Close() testC := testgrpc.NewTestServiceClient(cc) @@ -1008,11 +1008,12 @@ func (s) TestGRPCLB_FallBackWithNoServerAddress(t *testing.T) { grpc.WithTransportCredentials(&serverNameCheckCreds{}), grpc.WithContextDialer(fakeNameDialer), } - cc, err := grpc.Dial(r.Scheme()+":///"+beServerName, dopts...) + cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, dopts...) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend %v", err) } defer cc.Close() + cc.Connect() testC := testgrpc.NewTestServiceClient(cc) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) @@ -1102,10 +1103,11 @@ func (s) TestGRPCLB_PickFirst(t *testing.T) { grpc.WithTransportCredentials(&serverNameCheckCreds{}), grpc.WithContextDialer(fakeNameDialer), } - cc, err := grpc.Dial(r.Scheme()+":///"+beServerName, dopts...) + cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, dopts...) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend: %v", err) } + cc.Connect() defer cc.Close() // Push a service config with grpclb as the load balancing policy and @@ -1198,7 +1200,7 @@ func (s) TestGRPCLB_BackendConnectionErrorPropagation(t *testing.T) { grpc.WithTransportCredentials(&serverNameCheckCreds{}), grpc.WithContextDialer(fakeNameDialer)) if err != nil { - t.Fatalf("Failed to create new client to the backend %v", err) + t.Fatalf("Failed to create a client for the backend: %v", err) } defer cc.Close() testC := testgrpc.NewTestServiceClient(cc) @@ -1241,10 +1243,11 @@ func testGRPCLBEmptyServerList(t *testing.T, svcfg string) { grpc.WithTransportCredentials(&serverNameCheckCreds{}), grpc.WithContextDialer(fakeNameDialer), } - cc, err := grpc.DialContext(ctx, r.Scheme()+":///"+beServerName, dopts...) + cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, dopts...) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend %v", err) } + cc.Connect() defer cc.Close() testC := testgrpc.NewTestServiceClient(cc) @@ -1311,15 +1314,16 @@ func (s) TestGRPCLBWithTargetNameFieldInConfig(t *testing.T) { // Push the backend address to the remote balancer. tss.ls.sls <- sl - cc, err := grpc.Dial(r.Scheme()+":///"+beServerName, + cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, grpc.WithResolvers(r), grpc.WithTransportCredentials(&serverNameCheckCreds{}), grpc.WithContextDialer(fakeNameDialer), grpc.WithUserAgent(testUserAgent)) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend %v", err) } defer cc.Close() + cc.Connect() testC := testgrpc.NewTestServiceClient(cc) // Push a resolver update with grpclb configuration which does not contain the @@ -1418,15 +1422,14 @@ func runAndCheckStats(t *testing.T, drop bool, statsChan chan *lbpb.ClientStats, tss.ls.statsDura = 100 * time.Millisecond creds := serverNameCheckCreds{} - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - cc, err := grpc.DialContext(ctx, r.Scheme()+":///"+beServerName, grpc.WithResolvers(r), + cc, err := grpc.NewClient(r.Scheme()+":///"+beServerName, grpc.WithResolvers(r), grpc.WithTransportCredentials(&creds), grpc.WithPerRPCCredentials(failPreRPCCred{}), grpc.WithContextDialer(fakeNameDialer)) if err != nil { - t.Fatalf("Failed to dial to the backend %v", err) + t.Fatalf("Failed to create a client for the backend %v", err) } + cc.Connect() defer cc.Close() rstate := resolver.State{ServiceConfig: r.CC.ParseServiceConfig(grpclbConfig)} diff --git a/balancer/rls/picker_test.go b/balancer/rls/picker_test.go index cb08c433f9e8..a0bdbc827921 100644 --- a/balancer/rls/picker_test.go +++ b/balancer/rls/picker_test.go @@ -267,9 +267,9 @@ func (s) Test_RLSDefaultTargetPicksMetric(t *testing.T) { r := startManualResolverWithConfig(t, rlsConfig) tmr := stats.NewTestMetricsRecorder() - cc, err := grpc.Dial(r.Scheme()+":///", grpc.WithResolvers(r), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithStatsHandler(tmr)) + cc, err := grpc.NewClient(r.Scheme()+":///", grpc.WithResolvers(r), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithStatsHandler(tmr)) if err != nil { - t.Fatalf("grpc.Dial() failed: %v", err) + t.Fatalf("grpc.NewClient() failed: %v", err) } defer cc.Close() @@ -314,9 +314,9 @@ func (s) Test_RLSTargetPicksMetric(t *testing.T) { tmr := stats.NewTestMetricsRecorder() // Dial the backend. - cc, err := grpc.Dial(r.Scheme()+":///", grpc.WithResolvers(r), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithStatsHandler(tmr)) + cc, err := grpc.NewClient(r.Scheme()+":///", grpc.WithResolvers(r), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithStatsHandler(tmr)) if err != nil { - t.Fatalf("grpc.Dial() failed: %v", err) + t.Fatalf("grpc.NewClient() failed: %v", err) } defer cc.Close() @@ -352,9 +352,9 @@ func (s) Test_RLSFailedPicksMetric(t *testing.T) { tmr := stats.NewTestMetricsRecorder() // Dial the backend. - cc, err := grpc.Dial(r.Scheme()+":///", grpc.WithResolvers(r), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithStatsHandler(tmr)) + cc, err := grpc.NewClient(r.Scheme()+":///", grpc.WithResolvers(r), grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithStatsHandler(tmr)) if err != nil { - t.Fatalf("grpc.Dial() failed: %v", err) + t.Fatalf("grpc.NewClient() failed: %v", err) } defer cc.Close() diff --git a/resolver_balancer_ext_test.go b/resolver_balancer_ext_test.go index 417a5781ef80..75fac51a66db 100644 --- a/resolver_balancer_ext_test.go +++ b/resolver_balancer_ext_test.go @@ -66,11 +66,12 @@ func (s) TestResolverBalancerInteraction(t *testing.T) { rb.ResolveNowCallback = func(resolver.ResolveNowOptions) { close(rnCh) } resolver.Register(rb) - cc, err := grpc.Dial(name+":///", grpc.WithTransportCredentials(insecure.NewCredentials())) + cc, err := grpc.NewClient(name+":///", grpc.WithTransportCredentials(insecure.NewCredentials())) if err != nil { - t.Fatalf("grpc.Dial error: %v", err) + t.Fatalf("grpc.NewClient error: %v", err) } defer cc.Close() + cc.Connect() select { case <-rnCh: case <-time.After(defaultTestTimeout): @@ -109,11 +110,12 @@ func (s) TestResolverBuildFailure(t *testing.T) { resolver.Register(&resolverBuilderWithErr{errCh: resErrCh, scheme: name}) resErrCh <- nil - cc, err := grpc.Dial(name+":///", grpc.WithTransportCredentials(insecure.NewCredentials())) + cc, err := grpc.NewClient(name+":///", grpc.WithTransportCredentials(insecure.NewCredentials())) if err != nil { - t.Fatalf("grpc.Dial error: %v", err) + t.Fatalf("grpc.NewClient error: %v", err) } defer cc.Close() + cc.Connect() enterIdle(cc) const errStr = "test error from resolver builder" t.Log("pushing res err") diff --git a/test/creds_test.go b/test/creds_test.go index ff3818e0ad54..c9b8862442db 100644 --- a/test/creds_test.go +++ b/test/creds_test.go @@ -236,9 +236,9 @@ func (s) TestFailFastRPCErrorOnBadCertificates(t *testing.T) { opts := []grpc.DialOption{grpc.WithTransportCredentials(clientAlwaysFailCred{})} ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - cc, err := grpc.DialContext(ctx, te.srvAddr, opts...) + cc, err := grpc.NewClient(te.srvAddr, opts...) if err != nil { - t.Fatalf("Dial(_) = %v, want %v", err, nil) + t.Fatalf("NewClient(_) = %v, want %v", err, nil) } defer cc.Close() @@ -262,16 +262,14 @@ func (s) TestWaitForReadyRPCErrorOnBadCertificates(t *testing.T) { defer te.tearDown() opts := []grpc.DialOption{grpc.WithTransportCredentials(clientAlwaysFailCred{})} - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) - defer cancel() - cc, err := grpc.DialContext(ctx, te.srvAddr, opts...) + cc, err := grpc.NewClient(te.srvAddr, opts...) if err != nil { - t.Fatalf("Dial(_) = %v, want %v", err, nil) + t.Fatalf("NewClient(_) = %v, want %v", err, nil) } defer cc.Close() tc := testgrpc.NewTestServiceClient(cc) - ctx, cancel = context.WithTimeout(context.Background(), defaultTestShortTimeout) + ctx, cancel := context.WithTimeout(context.Background(), defaultTestShortTimeout) defer cancel() if _, err = tc.EmptyCall(ctx, &testpb.Empty{}, grpc.WaitForReady(true)); !strings.Contains(err.Error(), clientAlwaysFailCredErrorMsg) { t.Fatalf("TestService/EmptyCall(_, _) = _, %v, want err.Error() contains %q", err, clientAlwaysFailCredErrorMsg) @@ -437,11 +435,12 @@ func (s) TestCredsHandshakeAuthority(t *testing.T) { r := manual.NewBuilderWithScheme("whatever") - cc, err := grpc.Dial(r.Scheme()+":///"+testAuthority, grpc.WithTransportCredentials(cred), grpc.WithResolvers(r)) + cc, err := grpc.NewClient(r.Scheme()+":///"+testAuthority, grpc.WithTransportCredentials(cred), grpc.WithResolvers(r)) if err != nil { - t.Fatalf("grpc.Dial(%q) = %v", lis.Addr().String(), err) + t.Fatalf("grpc.NewClient(%q) = %v", lis.Addr().String(), err) } defer cc.Close() + cc.Connect() r.UpdateState(resolver.State{Addresses: []resolver.Address{{Addr: lis.Addr().String()}}}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) @@ -469,11 +468,12 @@ func (s) TestCredsHandshakeServerNameAuthority(t *testing.T) { r := manual.NewBuilderWithScheme("whatever") - cc, err := grpc.Dial(r.Scheme()+":///"+testAuthority, grpc.WithTransportCredentials(cred), grpc.WithResolvers(r)) + cc, err := grpc.NewClient(r.Scheme()+":///"+testAuthority, grpc.WithTransportCredentials(cred), grpc.WithResolvers(r)) if err != nil { - t.Fatalf("grpc.Dial(%q) = %v", lis.Addr().String(), err) + t.Fatalf("grpc.NewClient(%q) = %v", lis.Addr().String(), err) } defer cc.Close() + cc.Connect() r.UpdateState(resolver.State{Addresses: []resolver.Address{{Addr: lis.Addr().String(), ServerName: testServerName}}}) ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) @@ -524,11 +524,12 @@ func (s) TestServerCredsDispatch(t *testing.T) { go s.Serve(lis) defer s.Stop() - cc, err := grpc.Dial(lis.Addr().String(), grpc.WithTransportCredentials(cred)) + cc, err := grpc.NewClient(lis.Addr().String(), grpc.WithTransportCredentials(cred)) if err != nil { - t.Fatalf("grpc.Dial(%q) = %v", lis.Addr().String(), err) + t.Fatalf("grpc.NewClient(%q) = %v", lis.Addr().String(), err) } defer cc.Close() + cc.Connect() rawConn := cred.getRawConn() // Give grpc a chance to see the error and potentially close the connection. diff --git a/test/end2end_test.go b/test/end2end_test.go index 2238f8f12422..19972ecebe17 100644 --- a/test/end2end_test.go +++ b/test/end2end_test.go @@ -609,7 +609,7 @@ func (te *test) listenAndServe(ts testgrpc.TestServiceServer, listen func(networ if te.serverInitialConnWindowSize > 0 { sopts = append(sopts, grpc.InitialConnWindowSize(te.serverInitialConnWindowSize)) } - la := "localhost:0" + la := ":0" switch te.e.network { case "unix": la = "/tmp/testsock" + fmt.Sprintf("%d", time.Now().UnixNano()) From fc09e2c78de98797ebe240f1a25da69b2dacceee Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Mon, 9 Dec 2024 08:57:26 +0530 Subject: [PATCH 37/57] final rebase with master --- clientconn.go | 11 +- stats/handlers.go | 5 + stats/opentelemetry/client_metrics.go | 96 +++- stats/opentelemetry/e2e_test.go | 673 ++++++++++++++++++++++++-- stats/opentelemetry/opentelemetry.go | 33 ++ stats/opentelemetry/server_metrics.go | 32 +- stats/opentelemetry/trace.go | 140 ++++++ stream.go | 39 +- 8 files changed, 943 insertions(+), 86 deletions(-) create mode 100644 stats/opentelemetry/trace.go diff --git a/clientconn.go b/clientconn.go index 4f57b55434f9..5f326647e11f 100644 --- a/clientconn.go +++ b/clientconn.go @@ -130,9 +130,10 @@ func (dcs *defaultConfigSelector) SelectConfig(rpcInfo iresolver.RPCInfo) (*ires // function. func NewClient(target string, opts ...DialOption) (conn *ClientConn, err error) { cc := &ClientConn{ - target: target, - conns: make(map[*addrConn]struct{}), - dopts: defaultDialOptions(), + target: target, + conns: make(map[*addrConn]struct{}), + dopts: defaultDialOptions(), + nameResolutionDelayed: false, } cc.retryThrottler.Store((*retryThrottler)(nil)) @@ -604,6 +605,10 @@ type ClientConn struct { idlenessMgr *idle.Manager metricsRecorderList *stats.MetricsRecorderList + // To track if there was a delay in name resolution, helping to track + // latency issues in gRPC connection setup. + nameResolutionDelayed bool + // The following provide their own synchronization, and therefore don't // require cc.mu to be held to access them. csMgr *connectivityStateManager diff --git a/stats/handlers.go b/stats/handlers.go index dc03731e45ef..c1dd190c1609 100644 --- a/stats/handlers.go +++ b/stats/handlers.go @@ -38,6 +38,11 @@ type RPCTagInfo struct { // FailFast indicates if this RPC is failfast. // This field is only valid on client side, it's always false on server side. FailFast bool + // NameResolutionDelay indicates whether there was a delay in name + // resolution. + // + // This field is only valid on client side, it's always false on server side. + NameResolutionDelay bool } // Handler defines the interface for the related stats handling (e.g., RPCs, connections). diff --git a/stats/opentelemetry/client_metrics.go b/stats/opentelemetry/client_metrics.go index 265791e5a261..cb13182d7b9a 100644 --- a/stats/opentelemetry/client_metrics.go +++ b/stats/opentelemetry/client_metrics.go @@ -18,10 +18,15 @@ package opentelemetry import ( "context" + "strings" "sync/atomic" "time" + "go.opentelemetry.io/otel" + otelcodes "go.opentelemetry.io/otel/codes" + "go.opentelemetry.io/otel/trace" "google.golang.org/grpc" + grpccodes "google.golang.org/grpc/codes" estats "google.golang.org/grpc/experimental/stats" istats "google.golang.org/grpc/internal/stats" "google.golang.org/grpc/metadata" @@ -33,6 +38,7 @@ import ( ) type clientStatsHandler struct { + statsHandler estats.MetricsRecorder options Options clientMetrics clientMetrics @@ -68,6 +74,15 @@ func (h *clientStatsHandler) initializeMetrics() { rm.registerMetrics(metrics, meter) } +func (h *clientStatsHandler) initializeTracing() { + if isTracingDisabled(h.options.TraceOptions) { + return + } + + otel.SetTextMapPropagator(h.options.TraceOptions.TextMapPropagator) + otel.SetTracerProvider(h.options.TraceOptions.TracerProvider) +} + func (h *clientStatsHandler) unaryInterceptor(ctx context.Context, method string, req, reply any, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { ci := &callInfo{ target: cc.CanonicalTarget(), @@ -85,8 +100,12 @@ func (h *clientStatsHandler) unaryInterceptor(ctx context.Context, method string } startTime := time.Now() + var span *trace.Span + if !isTracingDisabled(h.options.TraceOptions) { + ctx, span = h.createCallTraceSpan(ctx, method) + } err := invoker(ctx, method, req, reply, cc, opts...) - h.perCallMetrics(ctx, err, startTime, ci) + h.perCallTracesAndMetrics(ctx, err, startTime, ci, span) return err } @@ -119,22 +138,50 @@ func (h *clientStatsHandler) streamInterceptor(ctx context.Context, desc *grpc.S } startTime := time.Now() - + var span *trace.Span + if !isTracingDisabled(h.options.TraceOptions) { + ctx, span = h.createCallTraceSpan(ctx, method) + } callback := func(err error) { - h.perCallMetrics(ctx, err, startTime, ci) + h.perCallTracesAndMetrics(ctx, err, startTime, ci, span) } opts = append([]grpc.CallOption{grpc.OnFinish(callback)}, opts...) return streamer(ctx, desc, cc, method, opts...) } -func (h *clientStatsHandler) perCallMetrics(ctx context.Context, err error, startTime time.Time, ci *callInfo) { - callLatency := float64(time.Since(startTime)) / float64(time.Second) // calculate ASAP - attrs := otelmetric.WithAttributeSet(otelattribute.NewSet( - otelattribute.String("grpc.method", ci.method), - otelattribute.String("grpc.target", ci.target), - otelattribute.String("grpc.status", canonicalString(status.Code(err))), - )) - h.clientMetrics.callDuration.Record(ctx, callLatency, attrs) +// perCallTracesAndMetrics records per call trace spans and metrics. +func (h *clientStatsHandler) perCallTracesAndMetrics(ctx context.Context, err error, startTime time.Time, ci *callInfo, ts *trace.Span) { + if !isTracingDisabled(h.options.TraceOptions) && ts != nil { + s := status.Convert(err) + if s.Code() == grpccodes.OK { + (*ts).SetStatus(otelcodes.Ok, s.Message()) + } else { + (*ts).SetStatus(otelcodes.Error, s.Message()) + } + (*ts).End() + } + if !isMetricsDisabled(h.options.MetricsOptions) { + callLatency := float64(time.Since(startTime)) / float64(time.Second) + attrs := otelmetric.WithAttributeSet(otelattribute.NewSet( + otelattribute.String("grpc.method", ci.method), + otelattribute.String("grpc.target", ci.target), + otelattribute.String("grpc.status", canonicalString(status.Code(err))), + )) + h.clientMetrics.callDuration.Record(ctx, callLatency, attrs) + } +} + +// createCallTraceSpan creates a call span to put in the provided context using +// provided TraceProvider. If TraceProvider is nil, it returns context as is. +func (h *clientStatsHandler) createCallTraceSpan(ctx context.Context, method string) (context.Context, *trace.Span) { + if h.options.TraceOptions.TracerProvider == nil { + logger.Error("TraceProvider is not provided in trace options") + return ctx, nil + } + mn := strings.Replace(removeLeadingSlash(method), "/", ".", -1) + tracer := otel.Tracer("grpc-open-telemetry") + ctx, span := tracer.Start(ctx, mn, trace.WithSpanKind(trace.SpanKindClient)) + return ctx, &span } // TagConn exists to satisfy stats.Handler. @@ -163,15 +210,21 @@ func (h *clientStatsHandler) TagRPC(ctx context.Context, info *stats.RPCTagInfo) } ctx = istats.SetLabels(ctx, labels) } - ai := &attemptInfo{ // populates information about RPC start. - startTime: time.Now(), - xdsLabels: labels.TelemetryLabels, - method: info.FullMethodName, + ai := &attemptInfo{} + startTime := time.Now() + if !isTracingDisabled(h.options.TraceOptions) { + callSpan := trace.SpanFromContext(ctx) + if info.NameResolutionDelay { + callSpan.AddEvent("Delayed name resolution complete") + } + ctx, ai = h.traceTagRPC(trace.ContextWithSpan(ctx, callSpan), info) } - ri := &rpcInfo{ + ai.startTime = startTime + ai.xdsLabels = labels.TelemetryLabels + ai.method = info.FullMethodName + return setRPCInfo(ctx, &rpcInfo{ ai: ai, - } - return setRPCInfo(ctx, ri) + }) } func (h *clientStatsHandler) HandleRPC(ctx context.Context, rs stats.RPCStats) { @@ -180,7 +233,12 @@ func (h *clientStatsHandler) HandleRPC(ctx context.Context, rs stats.RPCStats) { logger.Error("ctx passed into client side stats handler metrics event handling has no client attempt data present") return } - h.processRPCEvent(ctx, rs, ri.ai) + if !isMetricsDisabled(h.options.MetricsOptions) { + h.processRPCEvent(ctx, rs, ri.ai) + } + if !isTracingDisabled(h.options.TraceOptions) { + h.populateSpan(ctx, rs, ri.ai) + } } func (h *clientStatsHandler) processRPCEvent(ctx context.Context, s stats.RPCStats, ai *attemptInfo) { diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index ac671e2982a3..5a3400fc0f06 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -14,16 +14,21 @@ * limitations under the License. */ -package opentelemetry_test +package opentelemetry import ( "context" "fmt" - "io" "testing" "time" + otelinternaltracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" + + "go.opentelemetry.io/otel" + otelcodes "go.opentelemetry.io/otel/codes" + trace2 "go.opentelemetry.io/otel/trace" + v3clusterpb "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3" v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3" v3endpointpb "github.com/envoyproxy/go-control-plane/envoy/config/endpoint/v3" @@ -35,45 +40,69 @@ import ( "google.golang.org/protobuf/types/known/durationpb" "google.golang.org/protobuf/types/known/wrapperspb" + "go.opentelemetry.io/otel/attribute" + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/sdk/metric" + "go.opentelemetry.io/otel/sdk/metric/metricdata" + "go.opentelemetry.io/otel/sdk/metric/metricdata/metricdatatest" + "go.opentelemetry.io/otel/sdk/trace" + "go.opentelemetry.io/otel/sdk/trace/tracetest" "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/encoding/gzip" "google.golang.org/grpc/internal/grpcsync" - "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/stubserver" itestutils "google.golang.org/grpc/internal/testutils" "google.golang.org/grpc/internal/testutils/xds/e2e" setup "google.golang.org/grpc/internal/testutils/xds/e2e/setup" testgrpc "google.golang.org/grpc/interop/grpc_testing" testpb "google.golang.org/grpc/interop/grpc_testing" + "google.golang.org/grpc/metadata" "google.golang.org/grpc/orca" - "google.golang.org/grpc/stats/opentelemetry" "google.golang.org/grpc/stats/opentelemetry/internal/testutils" - - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/sdk/metric" - "go.opentelemetry.io/otel/sdk/metric/metricdata" - "go.opentelemetry.io/otel/sdk/metric/metricdata/metricdatatest" ) -var defaultTestTimeout = 5 * time.Second +// traceSpanInfo is the information received about the span. This is a subset +// of information that is important to verify that gRPC has knobs over, which +// goes through a stable OpenTelemetry API with well-defined behavior. This keeps +// the robustness of assertions over time. +type traceSpanInfo struct { + spanKind string + name string + events []trace.Event + attributes []attribute.KeyValue +} -type s struct { - grpctest.Tester +// defaultMetricsOptions creates default metrics options +func defaultMetricsOptions(_ *testing.T, methodAttributeFilter func(string) bool) (*MetricsOptions, *metric.ManualReader) { + reader := metric.NewManualReader() + provider := metric.NewMeterProvider(metric.WithReader(reader)) + metricsOptions := &MetricsOptions{ + MeterProvider: provider, + Metrics: DefaultMetrics(), + MethodAttributeFilter: methodAttributeFilter, + } + return metricsOptions, reader } -func Test(t *testing.T) { - grpctest.RunSubTests(t, s{}) +// defaultTraceOptions function to create default trace options +func defaultTraceOptions(_ *testing.T) (*TraceOptions, *tracetest.InMemoryExporter) { + spanExporter := tracetest.NewInMemoryExporter() + spanProcessor := trace.NewSimpleSpanProcessor(spanExporter) + tracerProvider := trace.NewTracerProvider(trace.WithSpanProcessor(spanProcessor)) + textMapPropagator := propagation.NewCompositeTextMapPropagator(GRPCTraceBinPropagator{}) + traceOptions := &TraceOptions{ + TracerProvider: tracerProvider, + TextMapPropagator: textMapPropagator, + } + return traceOptions, spanExporter } // setupStubServer creates a stub server with OpenTelemetry component configured on client -// and server side. It returns a reader for metrics emitted from OpenTelemetry -// component and the server. -func setupStubServer(t *testing.T, methodAttributeFilter func(string) bool) (*metric.ManualReader, *stubserver.StubServer) { - reader := metric.NewManualReader() - provider := metric.NewMeterProvider(metric.WithReader(reader)) +// and server side and returns the server. +func setupStubServer(t *testing.T, metricsOptions *MetricsOptions, traceOptions *TraceOptions) *stubserver.StubServer { ss := &stubserver.StubServer{ - UnaryCallF: func(ctx context.Context, in *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { + UnaryCallF: func(_ context.Context, in *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { return &testpb.SimpleResponse{Payload: &testpb.Payload{ Body: make([]byte, len(in.GetPayload().GetBody())), }}, nil @@ -88,20 +117,19 @@ func setupStubServer(t *testing.T, methodAttributeFilter func(string) bool) (*me }, } - if err := ss.Start([]grpc.ServerOption{opentelemetry.ServerOption(opentelemetry.Options{ - MetricsOptions: opentelemetry.MetricsOptions{ - MeterProvider: provider, - Metrics: opentelemetry.DefaultMetrics(), - MethodAttributeFilter: methodAttributeFilter, - }})}, opentelemetry.DialOption(opentelemetry.Options{ - MetricsOptions: opentelemetry.MetricsOptions{ - MeterProvider: provider, - Metrics: opentelemetry.DefaultMetrics(), - }, - })); err != nil { + otelOptions := Options{} + if metricsOptions != nil { + otelOptions.MetricsOptions = *metricsOptions + } + if traceOptions != nil { + otelOptions.TraceOptions = *traceOptions + } + + if err := ss.Start([]grpc.ServerOption{ServerOption(otelOptions)}, + DialOption(otelOptions)); err != nil { t.Fatalf("Error starting endpoint server: %v", err) } - return reader, ss + return ss } // TestMethodAttributeFilter tests the method attribute filter. The method @@ -112,7 +140,8 @@ func (s) TestMethodAttributeFilter(t *testing.T) { // Will allow duplex/any other type of RPC. return str != testgrpc.TestService_UnaryCall_FullMethodName } - reader, ss := setupStubServer(t, maf) + mo, reader := defaultMetricsOptions(t, maf) + ss := setupStubServer(t, mo, nil) defer ss.Stop() // Make a Unary and Streaming RPC. The Unary RPC should be filtered by the @@ -197,7 +226,8 @@ func (s) TestMethodAttributeFilter(t *testing.T) { // on the Client (no StaticMethodCallOption set) and Server. The method // attribute on subsequent metrics should be bucketed in "other". func (s) TestAllMetricsOneFunction(t *testing.T) { - reader, ss := setupStubServer(t, nil) + mo, reader := defaultMetricsOptions(t, nil) + ss := setupStubServer(t, mo, nil) defer ss.Stop() ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() @@ -362,7 +392,7 @@ func metricsDataFromReader(ctx context.Context, reader *metric.ManualReader) map func (s) TestWRRMetrics(t *testing.T) { cmr := orca.NewServerMetricsRecorder().(orca.CallMetricsRecorder) backend1 := stubserver.StartTestService(t, &stubserver.StubServer{ - EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { + EmptyCallF: func(ctx context.Context, _ *testpb.Empty) (*testpb.Empty, error) { if r := orca.CallMetricsRecorderFromContext(ctx); r != nil { // Copy metrics from what the test set in cmr into r. sm := cmr.(orca.ServerMetricsProvider).ServerMetrics() @@ -380,7 +410,7 @@ func (s) TestWRRMetrics(t *testing.T) { cmr.SetApplicationUtilization(1.0) backend2 := stubserver.StartTestService(t, &stubserver.StubServer{ - EmptyCallF: func(ctx context.Context, in *testpb.Empty) (*testpb.Empty, error) { + EmptyCallF: func(ctx context.Context, _ *testpb.Empty) (*testpb.Empty, error) { if r := orca.CallMetricsRecorderFromContext(ctx); r != nil { // Copy metrics from what the test set in cmr into r. sm := cmr.(orca.ServerMetricsProvider).ServerMetrics() @@ -452,14 +482,14 @@ func (s) TestWRRMetrics(t *testing.T) { reader := metric.NewManualReader() provider := metric.NewMeterProvider(metric.WithReader(reader)) - mo := opentelemetry.MetricsOptions{ + mo := MetricsOptions{ MeterProvider: provider, - Metrics: opentelemetry.DefaultMetrics().Add("grpc.lb.wrr.rr_fallback", "grpc.lb.wrr.endpoint_weight_not_yet_usable", "grpc.lb.wrr.endpoint_weight_stale", "grpc.lb.wrr.endpoint_weights"), + Metrics: DefaultMetrics().Add("grpc.lb.wrr.rr_fallback", "grpc.lb.wrr.endpoint_weight_not_yet_usable", "grpc.lb.wrr.endpoint_weight_stale", "grpc.lb.wrr.endpoint_weights"), OptionalLabels: []string{"grpc.lb.locality"}, } target := fmt.Sprintf("xds:///%s", serviceName) - cc, err := grpc.NewClient(target, grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(xdsResolver), opentelemetry.DialOption(opentelemetry.Options{MetricsOptions: mo})) + cc, err := grpc.NewClient(target, grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(xdsResolver), DialOption(Options{MetricsOptions: mo})) if err != nil { t.Fatalf("Failed to dial local test server: %v", err) } @@ -582,3 +612,568 @@ func pollForWantMetrics(ctx context.Context, t *testing.T, reader *metric.Manual return fmt.Errorf("error waiting for metrics %v: %v", wantMetrics, ctx.Err()) } + +// TestServerWithMetricsAndTraceOptions tests emitted metrics and traces from +// OpenTelemetry instrumentation component. It then configures a system with a gRPC +// Client and gRPC server with the OpenTelemetry Dial and Server Option configured +// specifying all the metrics and traces provided by this package, and makes a Unary +// RPC and a Streaming RPC. These two RPCs should cause certain recording for each +// registered metric observed through a Manual Metrics Reader on the provided +// OpenTelemetry SDK's Meter Provider. It also verifies the traces are recorded +// correctly. +func (s) TestServerWithMetricsAndTraceOptions(t *testing.T) { + // Create default metrics options + mo, reader := defaultMetricsOptions(t, nil) + // Create default trace options + to, exporter := defaultTraceOptions(t) + + ss := setupStubServer(t, mo, to) + defer ss.Stop() + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + // Create a parent span for the client call + ctx, _ = otel.Tracer("grpc-open-telemetry").Start(ctx, "test-parent-span") + md, _ := metadata.FromOutgoingContext(ctx) + otel.GetTextMapPropagator().Inject(ctx, otelinternaltracing.NewOutgoingCarrier(ctx)) + ctx = metadata.NewOutgoingContext(ctx, md) + + // Make two RPC's, a unary RPC and a streaming RPC. These should cause + // certain metrics and traces to be emitted. + if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ + Body: make([]byte, 10000), + }}, grpc.UseCompressor(gzip.Name)); err != nil { // Deterministic compression. + t.Fatalf("Unexpected error from UnaryCall: %v", err) + } + stream, err := ss.Client.FullDuplexCall(ctx) + if err != nil { + t.Fatalf("ss.Client.FullDuplexCall failed: %f", err) + } + + stream.CloseSend() + if _, err = stream.Recv(); err != io.EOF { + t.Fatalf("stream.Recv received an unexpected error: %v, expected an EOF error", err) + } + + // Verify metrics + rm := &metricdata.ResourceMetrics{} + reader.Collect(ctx, rm) + + gotMetrics := map[string]metricdata.Metrics{} + for _, sm := range rm.ScopeMetrics { + for _, m := range sm.Metrics { + gotMetrics[m.Name] = m + } + } + + wantMetrics := testutils.MetricData(testutils.MetricDataOptions{ + Target: ss.Target, + UnaryCompressedMessageSize: float64(57), + }) + testutils.CompareMetrics(ctx, t, reader, gotMetrics, wantMetrics) + + // Verify traces + spans := exporter.GetSpans() + if got, want := len(spans), 6; got != want { + t.Fatalf("Got %d spans, want %d", got, want) + } + + // Add assertions for specific span attributes and events as needed. + // For example, to check if the server span has the correct status: + serverSpan := spans[0] + if got, want := serverSpan.Status.Code, otelcodes.Ok; got != want { + t.Errorf("Got status code %v, want %v", got, want) + } +} + +// TestSpan verifies that the gRPC Trace Binary propagator +// correctly propagates span context between a client and server using the +// grpc-trace-bin header. It sets up a stub server with OpenTelemetry tracing +// enabled, makes a unary RPC. +func (s) TestSpan(t *testing.T) { + // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter + traceOptions, spanExporter := defaultTraceOptions(t) + // Start the server with OpenTelemetry options + ss := setupStubServer(t, nil, traceOptions) + defer ss.Stop() + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + // Make two RPC's, a unary RPC and a streaming RPC. These should cause + // certain metrics to be emitted, which should be able to be observed + // through the Metric Reader. + if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ + Body: make([]byte, 10000), + }}); err != nil { + t.Fatalf("Unexpected error from UnaryCall: %v", err) + } + stream, err := ss.Client.FullDuplexCall(ctx) + if err != nil { + t.Fatalf("ss.Client.FullDuplexCall failed: %f", err) + } + + stream.CloseSend() + if _, err = stream.Recv(); err != io.EOF { + t.Fatalf("stream.Recv received an unexpected error: %v, expected an EOF error", err) + } + // Get the spans from the exporter + spans := spanExporter.GetSpans() + if got, want := len(spans), 6; got != want { + t.Fatalf("Got %d spans, want %d", got, want) + } + + wantSI := []traceSpanInfo{ + { + name: "grpc.testing.TestService.UnaryCall", + spanKind: trace2.SpanKindServer.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + }, + { + Key: "FailFast", + }, + { + Key: "previous-rpc-attempts", + }, + { + Key: "transparent-retry", + }, + }, + events: []trace.Event{ + { + Name: "Inbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + }, + { + Key: "message-size", + }, + { + Key: "message-size-compressed", + }, + }, + }, + { + Name: "Outbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + }, + { + Key: "message-size", + }, + { + Key: "message-size-compressed", + }, + }, + }, + }, + }, + { + name: "Attempt.grpc.testing.TestService.UnaryCall", + spanKind: trace2.SpanKindInternal.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + }, + { + Key: "FailFast", + }, + { + Key: "previous-rpc-attempts", + }, + { + Key: "transparent-retry", + }, + }, + events: []trace.Event{ + { + Name: "Outbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + }, + { + Key: "message-size", + }, + { + Key: "message-size-compressed", + }, + }, + }, + { + Name: "Inbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + }, + { + Key: "message-size", + }, + { + Key: "message-size-compressed", + }, + }, + }, + }, + }, + { + name: "grpc.testing.TestService.UnaryCall", + spanKind: trace2.SpanKindClient.String(), + attributes: []attribute.KeyValue{}, + events: []trace.Event{}, + }, + { + name: "grpc.testing.TestService.FullDuplexCall", + spanKind: trace2.SpanKindServer.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + }, + { + Key: "FailFast", + }, + { + Key: "previous-rpc-attempts", + }, + { + Key: "transparent-retry", + }, + }, + events: []trace.Event{}, + }, + { + name: "grpc.testing.TestService.FullDuplexCall", + spanKind: trace2.SpanKindClient.String(), + attributes: []attribute.KeyValue{}, + events: []trace.Event{}, + }, + { + name: "Attempt.grpc.testing.TestService.FullDuplexCall", + spanKind: trace2.SpanKindInternal.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + }, + { + Key: "FailFast", + }, + { + Key: "previous-rpc-attempts", + }, + { + Key: "transparent-retry", + }, + }, + events: []trace.Event{}, + }, + } + + // Check that same traceID is used in client and server. + if got, want := spans[0].SpanContext.TraceID(), spans[2].SpanContext.TraceID(); got != want { + t.Fatal("TraceID mismatch in client span and server span.") + } + // Check that the attempt span id of client matches the span id of server + // SpanContext. + if got, want := spans[0].Parent.SpanID(), spans[1].SpanContext.SpanID(); got != want { + t.Fatal("SpanID mismatch in client span and server span.") + } + + // Check that same traceID is used in client and server. + if got, want := spans[3].SpanContext.TraceID(), spans[4].SpanContext.TraceID(); got != want { + t.Fatal("TraceID mismatch in client span and server span.") + } + // Check that the attempt span id of client matches the span id of server + // SpanContext. + if got, want := spans[3].Parent.SpanID(), spans[5].SpanContext.SpanID(); got != want { + t.Fatal("SpanID mismatch in client span and server span.") + } + + for index, span := range spans { + // Check that the attempt span has the correct status + if got, want := spans[index].Status.Code, otelcodes.Ok; got != want { + t.Errorf("Got status code %v, want %v", got, want) + } + // name + if got, want := span.Name, wantSI[index].name; got != want { + t.Errorf("Span name is %q, want %q", got, want) + } + // spanKind + if got, want := span.SpanKind.String(), wantSI[index].spanKind; got != want { + t.Errorf("Got span kind %q, want %q", got, want) + } + // attributes + if got, want := len(span.Attributes), len(wantSI[index].attributes); got != want { + t.Errorf("Got attributes list of size %q, want %q", got, want) + } + for idx, att := range span.Attributes { + if got, want := att.Key, wantSI[index].attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %q as %q, want %q", span.Name, got, want) + } + } + // events + if got, want := len(span.Events), len(wantSI[index].events); got != want { + t.Errorf("Event length is %q, want %q", got, want) + } + for eventIdx, event := range span.Events { + if got, want := event.Name, wantSI[index].events[eventIdx].Name; got != want { + t.Errorf("Got event name for span name %q as %q, want %q", span.Name, got, want) + } + for idx, att := range span.Attributes { + if got, want := att.Key, wantSI[eventIdx].attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %q with event name %q, as %q, want %q", span.Name, event.Name, got, want) + } + } + } + } +} + +// TestSpan_WithW3CContextPropagator sets up a stub server with OpenTelemetry tracing +// enabled makes a unary and a streaming RPC, and then, asserts that the correct +// number of spans are created with the expected spans. +func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { + // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter + traceOptions, spanExporter := defaultTraceOptions(t) + // Set the W3CContextPropagator as part of TracingOptions. + traceOptions.TextMapPropagator = propagation.NewCompositeTextMapPropagator(propagation.TraceContext{}) + // Start the server with OpenTelemetry options + ss := setupStubServer(t, nil, traceOptions) + defer ss.Stop() + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + // Make two RPC's, a unary RPC and a streaming RPC. These should cause + // certain metrics to be emitted, which should be able to be observed + // through the Metric Reader. + if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ + Body: make([]byte, 10000), + }}); err != nil { + t.Fatalf("Unexpected error from UnaryCall: %v", err) + } + stream, err := ss.Client.FullDuplexCall(ctx) + if err != nil { + t.Fatalf("ss.Client.FullDuplexCall failed: %f", err) + } + + stream.CloseSend() + if _, err = stream.Recv(); err != io.EOF { + t.Fatalf("stream.Recv received an unexpected error: %v, expected an EOF error", err) + } + // Get the spans from the exporter + spans := spanExporter.GetSpans() + if got, want := len(spans), 6; got != want { + t.Fatalf("Got %d spans, want %d", got, want) + } + + wantSI := []traceSpanInfo{ + { + name: "grpc.testing.TestService.UnaryCall", + spanKind: trace2.SpanKindServer.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + }, + { + Key: "FailFast", + }, + { + Key: "previous-rpc-attempts", + }, + { + Key: "transparent-retry", + }, + }, + events: []trace.Event{ + { + Name: "Inbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + }, + { + Key: "message-size", + }, + { + Key: "message-size-compressed", + }, + }, + }, + { + Name: "Outbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + }, + { + Key: "message-size", + }, + { + Key: "message-size-compressed", + }, + }, + }, + }, + }, + { + name: "Attempt.grpc.testing.TestService.UnaryCall", + spanKind: trace2.SpanKindInternal.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + }, + { + Key: "FailFast", + }, + { + Key: "previous-rpc-attempts", + }, + { + Key: "transparent-retry", + }, + }, + events: []trace.Event{ + { + Name: "Outbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + }, + { + Key: "message-size", + }, + { + Key: "message-size-compressed", + }, + }, + }, + { + Name: "Inbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + }, + { + Key: "message-size", + }, + { + Key: "message-size-compressed", + }, + }, + }, + }, + }, + { + name: "grpc.testing.TestService.UnaryCall", + spanKind: trace2.SpanKindClient.String(), + attributes: []attribute.KeyValue{}, + events: []trace.Event{}, + }, + { + name: "grpc.testing.TestService.FullDuplexCall", + spanKind: trace2.SpanKindServer.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + }, + { + Key: "FailFast", + }, + { + Key: "previous-rpc-attempts", + }, + { + Key: "transparent-retry", + }, + }, + events: []trace.Event{}, + }, + { + name: "grpc.testing.TestService.FullDuplexCall", + spanKind: trace2.SpanKindClient.String(), + attributes: []attribute.KeyValue{}, + events: []trace.Event{}, + }, + { + name: "Attempt.grpc.testing.TestService.FullDuplexCall", + spanKind: trace2.SpanKindInternal.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + }, + { + Key: "FailFast", + }, + { + Key: "previous-rpc-attempts", + }, + { + Key: "transparent-retry", + }, + }, + events: []trace.Event{}, + }, + } + + // Check that same traceID is used in client and server. + if got, want := spans[0].SpanContext.TraceID(), spans[2].SpanContext.TraceID(); got != want { + t.Fatal("TraceID mismatch in client span and server span.") + } + // Check that the attempt span id of client matches the span id of server + // SpanContext. + if got, want := spans[0].Parent.SpanID(), spans[1].SpanContext.SpanID(); got != want { + t.Fatal("SpanID mismatch in client span and server span.") + } + + // Check that same traceID is used in client and server. + if got, want := spans[3].SpanContext.TraceID(), spans[4].SpanContext.TraceID(); got != want { + t.Fatal("TraceID mismatch in client span and server span.") + } + // Check that the attempt span id of client matches the span id of server + // SpanContext. + if got, want := spans[3].Parent.SpanID(), spans[5].SpanContext.SpanID(); got != want { + t.Fatal("SpanID mismatch in client span and server span.") + } + + for index, span := range spans { + // Check that the attempt span has the correct status + if got, want := spans[index].Status.Code, otelcodes.Ok; got != want { + t.Errorf("Got status code %v, want %v", got, want) + } + // name + if got, want := span.Name, wantSI[index].name; got != want { + t.Errorf("Span name is %q, want %q", got, want) + } + // spanKind + if got, want := span.SpanKind.String(), wantSI[index].spanKind; got != want { + t.Errorf("Got span kind %q, want %q", got, want) + } + // attributes + if got, want := len(span.Attributes), len(wantSI[index].attributes); got != want { + t.Errorf("Got attributes list of size %q, want %q", got, want) + } + for idx, att := range span.Attributes { + if got, want := att.Key, wantSI[index].attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %q as %q, want %q", span.Name, got, want) + } + } + // events + if got, want := len(span.Events), len(wantSI[index].events); got != want { + t.Errorf("Event length is %q, want %q", got, want) + } + for eventIdx, event := range span.Events { + if got, want := event.Name, wantSI[index].events[eventIdx].Name; got != want { + t.Errorf("Got event name for span name %q as %q, want %q", span.Name, got, want) + } + for idx, att := range span.Attributes { + if got, want := att.Key, wantSI[eventIdx].attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %q with event name %q, as %q, want %q", span.Name, event.Name, got, want) + } + } + } + } +} diff --git a/stats/opentelemetry/opentelemetry.go b/stats/opentelemetry/opentelemetry.go index dcc424775f14..94e01b268233 100644 --- a/stats/opentelemetry/opentelemetry.go +++ b/stats/opentelemetry/opentelemetry.go @@ -38,6 +38,8 @@ import ( otelattribute "go.opentelemetry.io/otel/attribute" otelmetric "go.opentelemetry.io/otel/metric" "go.opentelemetry.io/otel/metric/noop" + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/trace" ) func init() { @@ -56,6 +58,8 @@ var joinDialOptions = internal.JoinDialOptions.(func(...grpc.DialOption) grpc.Di type Options struct { // MetricsOptions are the metrics options for OpenTelemetry instrumentation. MetricsOptions MetricsOptions + // TraceOptions are the tracing options for OpenTelemetry instrumentation. + TraceOptions TraceOptions } // MetricsOptions are the metrics options for OpenTelemetry instrumentation. @@ -90,6 +94,16 @@ type MetricsOptions struct { pluginOption otelinternal.PluginOption } +// TraceOptions are the tracing options for OpenTelemetry instrumentation. +type TraceOptions struct { + // TracerProvider is the OpenTelemetry tracer which is required to + // record traces/trace spans for instrumentation + TracerProvider trace.TracerProvider + + // TextMapPropagator propagates span context through text map carrier. + TextMapPropagator propagation.TextMapPropagator +} + // DialOption returns a dial option which enables OpenTelemetry instrumentation // code for a grpc.ClientConn. // @@ -105,6 +119,7 @@ type MetricsOptions struct { func DialOption(o Options) grpc.DialOption { csh := &clientStatsHandler{options: o} csh.initializeMetrics() + csh.initializeTracing() return joinDialOptions(grpc.WithChainUnaryInterceptor(csh.unaryInterceptor), grpc.WithChainStreamInterceptor(csh.streamInterceptor), grpc.WithStatsHandler(csh)) } @@ -125,6 +140,7 @@ var joinServerOptions = internal.JoinServerOptions.(func(...grpc.ServerOption) g func ServerOption(o Options) grpc.ServerOption { ssh := &serverStatsHandler{options: o} ssh.initializeMetrics() + ssh.initializeTracing() return joinServerOptions(grpc.ChainUnaryInterceptor(ssh.unaryInterceptor), grpc.ChainStreamInterceptor(ssh.streamInterceptor), grpc.StatsHandler(ssh)) } @@ -171,6 +187,14 @@ func removeLeadingSlash(mn string) string { return strings.TrimLeft(mn, "/") } +func isMetricsDisabled(mo MetricsOptions) bool { + return mo.MeterProvider == nil +} + +func isTracingDisabled(to TraceOptions) bool { + return to.TracerProvider == nil || to.TextMapPropagator == nil +} + // attemptInfo is RPC information scoped to the RPC attempt life span client // side, and the RPC life span server side. type attemptInfo struct { @@ -187,6 +211,15 @@ type attemptInfo struct { pluginOptionLabels map[string]string // pluginOptionLabels to attach to metrics emitted xdsLabels map[string]string + + // traceSpan is data used for recording traces. + traceSpan trace.Span + // message counters for sent and received messages (used for + // generating message IDs), and the number of previous RPC attempts for the + // associated call. + countSentMsg uint32 + countRecvMsg uint32 + previousRPCAttempts uint32 } type clientMetrics struct { diff --git a/stats/opentelemetry/server_metrics.go b/stats/opentelemetry/server_metrics.go index 4765afa8ed53..f61c13571415 100644 --- a/stats/opentelemetry/server_metrics.go +++ b/stats/opentelemetry/server_metrics.go @@ -28,11 +28,13 @@ import ( "google.golang.org/grpc/stats" "google.golang.org/grpc/status" + "go.opentelemetry.io/otel" otelattribute "go.opentelemetry.io/otel/attribute" otelmetric "go.opentelemetry.io/otel/metric" ) type serverStatsHandler struct { + statsHandler estats.MetricsRecorder options Options serverMetrics serverMetrics @@ -66,6 +68,15 @@ func (h *serverStatsHandler) initializeMetrics() { rm.registerMetrics(metrics, meter) } +func (h *serverStatsHandler) initializeTracing() { + if !isTracingDisabled(h.options.TraceOptions) { + return + } + + otel.SetTextMapPropagator(h.options.TraceOptions.TextMapPropagator) + otel.SetTracerProvider(h.options.TraceOptions.TracerProvider) +} + // attachLabelsTransportStream intercepts SetHeader and SendHeader calls of the // underlying ServerTransportStream to attach metadataExchangeLabels. type attachLabelsTransportStream struct { @@ -197,14 +208,16 @@ func (h *serverStatsHandler) TagRPC(ctx context.Context, info *stats.RPCTagInfo) } } - ai := &attemptInfo{ - startTime: time.Now(), - method: removeLeadingSlash(method), + ai := &attemptInfo{} + startTime := time.Now() + if !isTracingDisabled(h.options.TraceOptions) { + ctx, ai = h.traceTagRPC(ctx, info) } - ri := &rpcInfo{ + ai.startTime = startTime + ai.method = removeLeadingSlash(method) + return setRPCInfo(ctx, &rpcInfo{ ai: ai, - } - return setRPCInfo(ctx, ri) + }) } // HandleRPC implements per RPC tracing and stats implementation. @@ -214,7 +227,12 @@ func (h *serverStatsHandler) HandleRPC(ctx context.Context, rs stats.RPCStats) { logger.Error("ctx passed into server side stats handler metrics event handling has no server call data present") return } - h.processRPCData(ctx, rs, ri.ai) + if !isTracingDisabled(h.options.TraceOptions) { + h.populateSpan(ctx, rs, ri.ai) + } + if !isMetricsDisabled(h.options.MetricsOptions) { + h.processRPCData(ctx, rs, ri.ai) + } } func (h *serverStatsHandler) processRPCData(ctx context.Context, s stats.RPCStats, ai *attemptInfo) { diff --git a/stats/opentelemetry/trace.go b/stats/opentelemetry/trace.go new file mode 100644 index 000000000000..0101fc85f752 --- /dev/null +++ b/stats/opentelemetry/trace.go @@ -0,0 +1,140 @@ +/* + * Copyright 2024 gRPC 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 opentelemetry + +import ( + "context" + "strings" + "sync/atomic" + + "go.opentelemetry.io/otel" + "go.opentelemetry.io/otel/attribute" + otelcodes "go.opentelemetry.io/otel/codes" + "go.opentelemetry.io/otel/trace" + "google.golang.org/grpc/stats" + otelinternaltracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" + "google.golang.org/grpc/status" +) + +// traceTagRPC populates provided context with a new span using the +// TextMapPropagator supplied in trace options and internal itracing.carrier. +// It creates a new outgoing carrier which serializes information about this +// span into gRPC Metadata, if TextMapPropagator is provided in the trace +// options. if TextMapPropagator is not provided, it returns the context as is. +func (h *clientStatsHandler) traceTagRPC(ctx context.Context, rti *stats.RPCTagInfo) (context.Context, *attemptInfo) { + if h.options.TraceOptions.TextMapPropagator == nil { + return ctx, nil + } + + mn := "Attempt." + strings.Replace(removeLeadingSlash(rti.FullMethodName), "/", ".", -1) + tracer := otel.Tracer("grpc-open-telemetry") + ctx, span := tracer.Start(ctx, mn) + carrier := otelinternaltracing.NewOutgoingCarrier(ctx) + otel.GetTextMapPropagator().Inject(ctx, carrier) + + return carrier.Context(), &attemptInfo{ + traceSpan: span, + countSentMsg: 0, // msg events scoped to scope of context, per attempt client side + countRecvMsg: 0, + } +} + +// traceTagRPC populates context with new span data using the TextMapPropagator +// supplied in trace options and internal itracing.Carrier. It creates a new +// incoming carrier which extracts an existing span context (if present) by +// deserializing from provided context. If valid span context is extracted, it +// is set as parent of the new span otherwise new span remains the root span. +// If TextMapPropagator is not provided in the trace options, it returns context +// as is. +func (h *serverStatsHandler) traceTagRPC(ctx context.Context, rti *stats.RPCTagInfo) (context.Context, *attemptInfo) { + if h.options.TraceOptions.TextMapPropagator == nil { + return ctx, nil + } + + mn := strings.Replace(removeLeadingSlash(rti.FullMethodName), "/", ".", -1) + var span trace.Span + tracer := otel.Tracer("grpc-open-telemetry") + ctx = otel.GetTextMapPropagator().Extract(ctx, otelinternaltracing.NewIncomingCarrier(ctx)) + // If the context.Context provided in `ctx` to tracer.Start(), contains a + // Span then the newly-created Span will be a child of that span, + // otherwise it will be a root span. + ctx, span = tracer.Start(ctx, mn, trace.WithSpanKind(trace.SpanKindServer)) + return ctx, &attemptInfo{ + traceSpan: span, + countSentMsg: 0, + countRecvMsg: 0, + } +} + +// statsHandler holds common functionality for both client and server stats +// handler. +type statsHandler struct{} + +// populateSpan populates span information based on stats passed in, representing +// invariants of the RPC lifecycle. It ends the span, triggering its export. +// This function handles attempt spans on the client-side and call spans on the +// server-side. +func (h *statsHandler) populateSpan(_ context.Context, rs stats.RPCStats, ai *attemptInfo) { + if ai == nil || ai.traceSpan == nil { + // Shouldn't happen, tagRPC call comes before this function gets called + // which populates this information. + logger.Error("ctx passed into stats handler tracing event handling has no traceSpan present") + return + } + span := ai.traceSpan + + switch rs := rs.(type) { + case *stats.Begin: + // Note: Go always added Client and FailFast attributes even though they are not + // defined by the OpenCensus gRPC spec. Thus, they are unimportant for + // correctness. + span.SetAttributes( + attribute.Bool("Client", rs.Client), + attribute.Bool("FailFast", rs.Client), + attribute.Int64("previous-rpc-attempts", int64(ai.previousRPCAttempts)), + attribute.Bool("transparent-retry", rs.IsTransparentRetryAttempt), + ) + // increment previous rpc attempts applicable for next attempt + atomic.AddUint32(&ai.previousRPCAttempts, 1) + case *stats.PickerUpdated: + span.AddEvent("Delayed LB pick complete") + case *stats.InPayload: + // message id - "must be calculated as two different counters starting + // from one for sent messages and one for received messages." + mi := atomic.AddUint32(&ai.countRecvMsg, 1) + span.AddEvent("Inbound compressed message", trace.WithAttributes( + attribute.Int64("sequence-number", int64(mi)), + attribute.Int64("message-size", int64(rs.Length)), + attribute.Int64("message-size-compressed", int64(rs.CompressedLength)), + )) + case *stats.OutPayload: + mi := atomic.AddUint32(&ai.countSentMsg, 1) + span.AddEvent("Outbound compressed message", trace.WithAttributes( + attribute.Int64("sequence-number", int64(mi)), + attribute.Int64("message-size", int64(rs.Length)), + attribute.Int64("message-size-compressed", int64(rs.CompressedLength)), + )) + case *stats.End: + if rs.Error != nil { + s := status.Convert(rs.Error) + span.SetStatus(otelcodes.Error, s.Message()) + } else { + span.SetStatus(otelcodes.Ok, "Ok") + } + span.End() + } +} diff --git a/stream.go b/stream.go index 17e2267b3320..7309e1eb6d07 100644 --- a/stream.go +++ b/stream.go @@ -213,13 +213,13 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth // Provide an opportunity for the first RPC to see the first service config // provided by the resolver. if err := cc.waitForResolvedAddrs(ctx); err != nil { + cc.nameResolutionDelayed = true return nil, err } - var mc serviceconfig.MethodConfig var onCommit func() - newStream := func(ctx context.Context, done func()) (iresolver.ClientStream, error) { - return newClientStreamWithParams(ctx, desc, cc, method, mc, onCommit, done, opts...) + var newStream = func(ctx context.Context, done func()) (iresolver.ClientStream, error) { + return newClientStreamWithParams(ctx, desc, cc, method, mc, onCommit, done, cc.nameResolutionDelayed, opts...) } rpcInfo := iresolver.RPCInfo{Context: ctx, Method: method} @@ -257,7 +257,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth return newStream(ctx, func() {}) } -func newClientStreamWithParams(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, mc serviceconfig.MethodConfig, onCommit, doneFunc func(), opts ...CallOption) (_ iresolver.ClientStream, err error) { +func newClientStreamWithParams(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, mc serviceconfig.MethodConfig, onCommit, doneFunc func(), nameResolutionDelayed bool, opts ...CallOption) (_ iresolver.ClientStream, err error) { c := defaultCallInfo() if mc.WaitForReady != nil { c.failFast = !*mc.WaitForReady @@ -321,19 +321,20 @@ func newClientStreamWithParams(ctx context.Context, desc *StreamDesc, cc *Client } cs := &clientStream{ - callHdr: callHdr, - ctx: ctx, - methodConfig: &mc, - opts: opts, - callInfo: c, - cc: cc, - desc: desc, - codec: c.codec, - cp: cp, - comp: comp, - cancel: cancel, - firstAttempt: true, - onCommit: onCommit, + callHdr: callHdr, + ctx: ctx, + methodConfig: &mc, + opts: opts, + callInfo: c, + cc: cc, + desc: desc, + codec: c.codec, + cp: cp, + comp: comp, + cancel: cancel, + firstAttempt: true, + onCommit: onCommit, + nameResolutionDelayed: nameResolutionDelayed, } if !cc.dopts.disableRetry { cs.retryThrottler = cc.retryThrottler.Load().(*retryThrottler) @@ -417,7 +418,7 @@ func (cs *clientStream) newAttemptLocked(isTransparent bool) (*csAttempt, error) var beginTime time.Time shs := cs.cc.dopts.copts.StatsHandlers for _, sh := range shs { - ctx = sh.TagRPC(ctx, &stats.RPCTagInfo{FullMethodName: method, FailFast: cs.callInfo.failFast}) + ctx = sh.TagRPC(ctx, &stats.RPCTagInfo{FullMethodName: method, FailFast: cs.callInfo.failFast, NameResolutionDelay: cs.nameResolutionDelayed}) beginTime = time.Now() begin := &stats.Begin{ Client: true, @@ -555,6 +556,8 @@ type clientStream struct { // synchronized. serverHeaderBinlogged bool + nameResolutionDelayed bool + mu sync.Mutex firstAttempt bool // if true, transparent retry is valid numRetries int // exclusive of transparent retry attempt(s) From 2747371d0d8126b50e06032e020e4789da32f514 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Mon, 9 Dec 2024 12:08:43 +0530 Subject: [PATCH 38/57] update e2e tests --- stats/opentelemetry/e2e_test.go | 203 +++++++++++++++++++++----------- 1 file changed, 134 insertions(+), 69 deletions(-) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index 5a3400fc0f06..fcfb157333a6 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -67,6 +67,7 @@ import ( // goes through a stable OpenTelemetry API with well-defined behavior. This keeps // the robustness of assertions over time. type traceSpanInfo struct { + sc trace2.SpanContext spanKind string name string events []trace.Event @@ -687,9 +688,9 @@ func (s) TestServerWithMetricsAndTraceOptions(t *testing.T) { } } -// TestSpan verifies that the gRPC Trace Binary propagator -// correctly propagates span context between a client and server using the -// grpc-trace-bin header. It sets up a stub server with OpenTelemetry tracing +// TestSpan verifies that the gRPC Trace Binary propagator correctly +// propagates span context between a client and server using the grpc- +// trace-bin header. It sets up a stub server with OpenTelemetry tracing // enabled, makes a unary RPC. func (s) TestSpan(t *testing.T) { // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter @@ -730,16 +731,20 @@ func (s) TestSpan(t *testing.T) { spanKind: trace2.SpanKindServer.String(), attributes: []attribute.KeyValue{ { - Key: "Client", + Key: "Client", + Value: attribute.IntValue(0), }, { - Key: "FailFast", + Key: "FailFast", + Value: attribute.IntValue(0), }, { - Key: "previous-rpc-attempts", + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), }, { - Key: "transparent-retry", + Key: "transparent-retry", + Value: attribute.IntValue(0), }, }, events: []trace.Event{ @@ -747,13 +752,16 @@ func (s) TestSpan(t *testing.T) { Name: "Inbound compressed message", Attributes: []attribute.KeyValue{ { - Key: "sequence-number", + Key: "sequence-number", + Value: attribute.IntValue(1), }, { - Key: "message-size", + Key: "message-size", + Value: attribute.IntValue(10006), }, { - Key: "message-size-compressed", + Key: "message-size-compressed", + Value: attribute.IntValue(10006), }, }, }, @@ -761,13 +769,16 @@ func (s) TestSpan(t *testing.T) { Name: "Outbound compressed message", Attributes: []attribute.KeyValue{ { - Key: "sequence-number", + Key: "sequence-number", + Value: attribute.IntValue(1), }, { - Key: "message-size", + Key: "message-size", + Value: attribute.IntValue(10006), }, { - Key: "message-size-compressed", + Key: "message-size-compressed", + Value: attribute.IntValue(10006), }, }, }, @@ -778,16 +789,20 @@ func (s) TestSpan(t *testing.T) { spanKind: trace2.SpanKindInternal.String(), attributes: []attribute.KeyValue{ { - Key: "Client", + Key: "Client", + Value: attribute.IntValue(1), }, { - Key: "FailFast", + Key: "FailFast", + Value: attribute.IntValue(1), }, { - Key: "previous-rpc-attempts", + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), }, { - Key: "transparent-retry", + Key: "transparent-retry", + Value: attribute.IntValue(0), }, }, events: []trace.Event{ @@ -795,13 +810,16 @@ func (s) TestSpan(t *testing.T) { Name: "Outbound compressed message", Attributes: []attribute.KeyValue{ { - Key: "sequence-number", + Key: "sequence-number", + Value: attribute.IntValue(1), }, { - Key: "message-size", + Key: "message-size", + Value: attribute.IntValue(10006), }, { - Key: "message-size-compressed", + Key: "message-size-compressed", + Value: attribute.IntValue(10006), }, }, }, @@ -809,13 +827,16 @@ func (s) TestSpan(t *testing.T) { Name: "Inbound compressed message", Attributes: []attribute.KeyValue{ { - Key: "sequence-number", + Key: "sequence-number", + Value: attribute.IntValue(1), }, { - Key: "message-size", + Key: "message-size", + Value: attribute.IntValue(10006), }, { - Key: "message-size-compressed", + Key: "message-size-compressed", + Value: attribute.IntValue(10006), }, }, }, @@ -832,16 +853,20 @@ func (s) TestSpan(t *testing.T) { spanKind: trace2.SpanKindServer.String(), attributes: []attribute.KeyValue{ { - Key: "Client", + Key: "Client", + Value: attribute.IntValue(0), }, { - Key: "FailFast", + Key: "FailFast", + Value: attribute.IntValue(0), }, { - Key: "previous-rpc-attempts", + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), }, { - Key: "transparent-retry", + Key: "transparent-retry", + Value: attribute.IntValue(0), }, }, events: []trace.Event{}, @@ -857,16 +882,20 @@ func (s) TestSpan(t *testing.T) { spanKind: trace2.SpanKindInternal.String(), attributes: []attribute.KeyValue{ { - Key: "Client", + Key: "Client", + Value: attribute.IntValue(1), }, { - Key: "FailFast", + Key: "FailFast", + Value: attribute.IntValue(1), }, { - Key: "previous-rpc-attempts", + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), }, { - Key: "transparent-retry", + Key: "transparent-retry", + Value: attribute.IntValue(0), }, }, events: []trace.Event{}, @@ -912,7 +941,10 @@ func (s) TestSpan(t *testing.T) { } for idx, att := range span.Attributes { if got, want := att.Key, wantSI[index].attributes[idx].Key; got != want { - t.Errorf("Got attribute key for span name %q as %q, want %q", span.Name, got, want) + t.Errorf("Got attribute key for span name %v as %v, want %v", span.Name, got, want) + } + if got, want := att.Value, wantSI[index].attributes[idx].Value; got != want { + t.Errorf("Got attribute value for span name %v as %v, want %v", span.Name, got, want) } } // events @@ -923,9 +955,12 @@ func (s) TestSpan(t *testing.T) { if got, want := event.Name, wantSI[index].events[eventIdx].Name; got != want { t.Errorf("Got event name for span name %q as %q, want %q", span.Name, got, want) } - for idx, att := range span.Attributes { - if got, want := att.Key, wantSI[eventIdx].attributes[idx].Key; got != want { - t.Errorf("Got attribute key for span name %q with event name %q, as %q, want %q", span.Name, event.Name, got, want) + for idx, att := range event.Attributes { + if got, want := att.Key, wantSI[index].events[eventIdx].Attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %q with event name %v, as %v, want %v", span.Name, event.Name, got, want) + } + if got, want := att.Value, wantSI[index].events[eventIdx].Attributes[idx].Value; got != want { + t.Errorf("Got attribute value for span name %v with event name %v, as %v, want %v", span.Name, event.Name, got, want) } } } @@ -935,7 +970,7 @@ func (s) TestSpan(t *testing.T) { // TestSpan_WithW3CContextPropagator sets up a stub server with OpenTelemetry tracing // enabled makes a unary and a streaming RPC, and then, asserts that the correct // number of spans are created with the expected spans. -func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { +func TestSpan_WithW3CContextPropagator(t *testing.T) { // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter traceOptions, spanExporter := defaultTraceOptions(t) // Set the W3CContextPropagator as part of TracingOptions. @@ -976,16 +1011,20 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { spanKind: trace2.SpanKindServer.String(), attributes: []attribute.KeyValue{ { - Key: "Client", + Key: "Client", + Value: attribute.IntValue(0), }, { - Key: "FailFast", + Key: "FailFast", + Value: attribute.IntValue(0), }, { - Key: "previous-rpc-attempts", + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), }, { - Key: "transparent-retry", + Key: "transparent-retry", + Value: attribute.IntValue(0), }, }, events: []trace.Event{ @@ -993,13 +1032,16 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { Name: "Inbound compressed message", Attributes: []attribute.KeyValue{ { - Key: "sequence-number", + Key: "sequence-number", + Value: attribute.IntValue(1), }, { - Key: "message-size", + Key: "message-size", + Value: attribute.IntValue(10006), }, { - Key: "message-size-compressed", + Key: "message-size-compressed", + Value: attribute.IntValue(10006), }, }, }, @@ -1007,13 +1049,16 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { Name: "Outbound compressed message", Attributes: []attribute.KeyValue{ { - Key: "sequence-number", + Key: "sequence-number", + Value: attribute.IntValue(1), }, { - Key: "message-size", + Key: "message-size", + Value: attribute.IntValue(10006), }, { - Key: "message-size-compressed", + Key: "message-size-compressed", + Value: attribute.IntValue(10006), }, }, }, @@ -1024,16 +1069,20 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { spanKind: trace2.SpanKindInternal.String(), attributes: []attribute.KeyValue{ { - Key: "Client", + Key: "Client", + Value: attribute.IntValue(1), }, { - Key: "FailFast", + Key: "FailFast", + Value: attribute.IntValue(1), }, { - Key: "previous-rpc-attempts", + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), }, { - Key: "transparent-retry", + Key: "transparent-retry", + Value: attribute.IntValue(0), }, }, events: []trace.Event{ @@ -1041,13 +1090,16 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { Name: "Outbound compressed message", Attributes: []attribute.KeyValue{ { - Key: "sequence-number", + Key: "sequence-number", + Value: attribute.IntValue(1), }, { - Key: "message-size", + Key: "message-size", + Value: attribute.IntValue(10006), }, { - Key: "message-size-compressed", + Key: "message-size-compressed", + Value: attribute.IntValue(10006), }, }, }, @@ -1055,13 +1107,16 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { Name: "Inbound compressed message", Attributes: []attribute.KeyValue{ { - Key: "sequence-number", + Key: "sequence-number", + Value: attribute.IntValue(1), }, { - Key: "message-size", + Key: "message-size", + Value: attribute.IntValue(10006), }, { - Key: "message-size-compressed", + Key: "message-size-compressed", + Value: attribute.IntValue(10006), }, }, }, @@ -1078,16 +1133,20 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { spanKind: trace2.SpanKindServer.String(), attributes: []attribute.KeyValue{ { - Key: "Client", + Key: "Client", + Value: attribute.IntValue(0), }, { - Key: "FailFast", + Key: "FailFast", + Value: attribute.IntValue(0), }, { - Key: "previous-rpc-attempts", + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), }, { - Key: "transparent-retry", + Key: "transparent-retry", + Value: attribute.IntValue(0), }, }, events: []trace.Event{}, @@ -1103,16 +1162,20 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { spanKind: trace2.SpanKindInternal.String(), attributes: []attribute.KeyValue{ { - Key: "Client", + Key: "Client", + Value: attribute.IntValue(1), }, { - Key: "FailFast", + Key: "FailFast", + Value: attribute.IntValue(1), }, { - Key: "previous-rpc-attempts", + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), }, { - Key: "transparent-retry", + Key: "transparent-retry", + Value: attribute.IntValue(0), }, }, events: []trace.Event{}, @@ -1138,7 +1201,6 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { if got, want := spans[3].Parent.SpanID(), spans[5].SpanContext.SpanID(); got != want { t.Fatal("SpanID mismatch in client span and server span.") } - for index, span := range spans { // Check that the attempt span has the correct status if got, want := spans[index].Status.Code, otelcodes.Ok; got != want { @@ -1158,7 +1220,7 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { } for idx, att := range span.Attributes { if got, want := att.Key, wantSI[index].attributes[idx].Key; got != want { - t.Errorf("Got attribute key for span name %q as %q, want %q", span.Name, got, want) + t.Errorf("Got attribute key for span name %v as %v, want %v", span.Name, got, want) } } // events @@ -1169,9 +1231,12 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { if got, want := event.Name, wantSI[index].events[eventIdx].Name; got != want { t.Errorf("Got event name for span name %q as %q, want %q", span.Name, got, want) } - for idx, att := range span.Attributes { - if got, want := att.Key, wantSI[eventIdx].attributes[idx].Key; got != want { - t.Errorf("Got attribute key for span name %q with event name %q, as %q, want %q", span.Name, event.Name, got, want) + for idx, att := range event.Attributes { + if got, want := att.Key, wantSI[index].events[eventIdx].Attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %q with event name %v, as %v, want %v", span.Name, event.Name, got, want) + } + if got, want := att.Value, wantSI[index].events[eventIdx].Attributes[idx].Value; got != want { + t.Errorf("Got attribute value for span name %v with event name %v, as %v, want %v", span.Name, event.Name, got, want) } } } From 96f92a0e8d55b43a69ff3f88de6367b5d0af66cd Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Tue, 10 Dec 2024 12:18:23 +0530 Subject: [PATCH 39/57] fix e2e tests --- internal/stats/metrics_recorder_list.go | 5 ++++- stats/opentelemetry/client_metrics.go | 2 ++ stats/opentelemetry/e2e_test.go | 21 +++++---------------- 3 files changed, 11 insertions(+), 17 deletions(-) diff --git a/internal/stats/metrics_recorder_list.go b/internal/stats/metrics_recorder_list.go index 79044657be15..4307e5f1e41b 100644 --- a/internal/stats/metrics_recorder_list.go +++ b/internal/stats/metrics_recorder_list.go @@ -58,8 +58,11 @@ func verifyLabels(desc *estats.MetricDescriptor, labelsRecv ...string) { // count associated with the provided handle. func (l *MetricsRecorderList) RecordInt64Count(handle *estats.Int64CountHandle, incr int64, labels ...string) { verifyLabels(handle.Descriptor(), labels...) - + fmt.Println("Inside MetricsRecorderList with metric recorders as: ", l.metricsRecorders) for _, metricRecorder := range l.metricsRecorders { + if metricRecorder == nil { + fmt.Println("metric recorder is nil.") + } metricRecorder.RecordInt64Count(handle, incr, labels...) } } diff --git a/stats/opentelemetry/client_metrics.go b/stats/opentelemetry/client_metrics.go index cb13182d7b9a..86dcaaa16654 100644 --- a/stats/opentelemetry/client_metrics.go +++ b/stats/opentelemetry/client_metrics.go @@ -18,6 +18,7 @@ package opentelemetry import ( "context" + "fmt" "strings" "sync/atomic" "time" @@ -70,6 +71,7 @@ func (h *clientStatsHandler) initializeMetrics() { rm := ®istryMetrics{ optionalLabels: h.options.MetricsOptions.OptionalLabels, } + fmt.Println("Setting metrics recorder.") h.MetricsRecorder = rm rm.registerMetrics(metrics, meter) } diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index fcfb157333a6..8127264f1caa 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -23,9 +23,6 @@ import ( "testing" "time" - otelinternaltracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" - - "go.opentelemetry.io/otel" otelcodes "go.opentelemetry.io/otel/codes" trace2 "go.opentelemetry.io/otel/trace" @@ -57,7 +54,6 @@ import ( setup "google.golang.org/grpc/internal/testutils/xds/e2e/setup" testgrpc "google.golang.org/grpc/interop/grpc_testing" testpb "google.golang.org/grpc/interop/grpc_testing" - "google.golang.org/grpc/metadata" "google.golang.org/grpc/orca" "google.golang.org/grpc/stats/opentelemetry/internal/testutils" ) @@ -634,12 +630,6 @@ func (s) TestServerWithMetricsAndTraceOptions(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() - // Create a parent span for the client call - ctx, _ = otel.Tracer("grpc-open-telemetry").Start(ctx, "test-parent-span") - md, _ := metadata.FromOutgoingContext(ctx) - otel.GetTextMapPropagator().Inject(ctx, otelinternaltracing.NewOutgoingCarrier(ctx)) - ctx = metadata.NewOutgoingContext(ctx, md) - // Make two RPC's, a unary RPC and a streaming RPC. These should cause // certain metrics and traces to be emitted. if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ @@ -693,10 +683,11 @@ func (s) TestServerWithMetricsAndTraceOptions(t *testing.T) { // trace-bin header. It sets up a stub server with OpenTelemetry tracing // enabled, makes a unary RPC. func (s) TestSpan(t *testing.T) { + mo, _ := defaultMetricsOptions(t, nil) // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter traceOptions, spanExporter := defaultTraceOptions(t) // Start the server with OpenTelemetry options - ss := setupStubServer(t, nil, traceOptions) + ss := setupStubServer(t, mo, traceOptions) defer ss.Stop() ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) @@ -943,9 +934,6 @@ func (s) TestSpan(t *testing.T) { if got, want := att.Key, wantSI[index].attributes[idx].Key; got != want { t.Errorf("Got attribute key for span name %v as %v, want %v", span.Name, got, want) } - if got, want := att.Value, wantSI[index].attributes[idx].Value; got != want { - t.Errorf("Got attribute value for span name %v as %v, want %v", span.Name, got, want) - } } // events if got, want := len(span.Events), len(wantSI[index].events); got != want { @@ -970,13 +958,14 @@ func (s) TestSpan(t *testing.T) { // TestSpan_WithW3CContextPropagator sets up a stub server with OpenTelemetry tracing // enabled makes a unary and a streaming RPC, and then, asserts that the correct // number of spans are created with the expected spans. -func TestSpan_WithW3CContextPropagator(t *testing.T) { +func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { + mo, _ := defaultMetricsOptions(t, nil) // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter traceOptions, spanExporter := defaultTraceOptions(t) // Set the W3CContextPropagator as part of TracingOptions. traceOptions.TextMapPropagator = propagation.NewCompositeTextMapPropagator(propagation.TraceContext{}) // Start the server with OpenTelemetry options - ss := setupStubServer(t, nil, traceOptions) + ss := setupStubServer(t, mo, traceOptions) defer ss.Stop() ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) From f650e37082c18f0bc9fe4e0726f6b60cc139f7bc Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Tue, 10 Dec 2024 12:37:51 +0530 Subject: [PATCH 40/57] remove fmt logs --- internal/stats/metrics_recorder_list.go | 5 +---- stats/opentelemetry/client_metrics.go | 2 -- 2 files changed, 1 insertion(+), 6 deletions(-) diff --git a/internal/stats/metrics_recorder_list.go b/internal/stats/metrics_recorder_list.go index 4307e5f1e41b..79044657be15 100644 --- a/internal/stats/metrics_recorder_list.go +++ b/internal/stats/metrics_recorder_list.go @@ -58,11 +58,8 @@ func verifyLabels(desc *estats.MetricDescriptor, labelsRecv ...string) { // count associated with the provided handle. func (l *MetricsRecorderList) RecordInt64Count(handle *estats.Int64CountHandle, incr int64, labels ...string) { verifyLabels(handle.Descriptor(), labels...) - fmt.Println("Inside MetricsRecorderList with metric recorders as: ", l.metricsRecorders) + for _, metricRecorder := range l.metricsRecorders { - if metricRecorder == nil { - fmt.Println("metric recorder is nil.") - } metricRecorder.RecordInt64Count(handle, incr, labels...) } } diff --git a/stats/opentelemetry/client_metrics.go b/stats/opentelemetry/client_metrics.go index 86dcaaa16654..cb13182d7b9a 100644 --- a/stats/opentelemetry/client_metrics.go +++ b/stats/opentelemetry/client_metrics.go @@ -18,7 +18,6 @@ package opentelemetry import ( "context" - "fmt" "strings" "sync/atomic" "time" @@ -71,7 +70,6 @@ func (h *clientStatsHandler) initializeMetrics() { rm := ®istryMetrics{ optionalLabels: h.options.MetricsOptions.OptionalLabels, } - fmt.Println("Setting metrics recorder.") h.MetricsRecorder = rm rm.registerMetrics(metrics, meter) } From 175d4c5e5e6ebc8ce5d8d7bb57b79b9b1f49c75e Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Wed, 11 Dec 2024 10:27:14 +0530 Subject: [PATCH 41/57] move grpc_trace_bin_propagator to experimental --- stats/opentelemetry/e2e_test.go | 3 ++- .../{ => experimental}/grpc_trace_bin_propagator.go | 2 +- .../grpc_trace_bin_propagator_test.go | 11 ++++++++++- 3 files changed, 13 insertions(+), 3 deletions(-) rename stats/opentelemetry/{ => experimental}/grpc_trace_bin_propagator.go (99%) rename stats/opentelemetry/{ => experimental}/grpc_trace_bin_propagator_test.go (97%) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index 8127264f1caa..f6b3f8cfc2bf 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -19,6 +19,7 @@ package opentelemetry import ( "context" "fmt" + "google.golang.org/grpc/stats/opentelemetry/experimental" "io" "testing" "time" @@ -87,7 +88,7 @@ func defaultTraceOptions(_ *testing.T) (*TraceOptions, *tracetest.InMemoryExport spanExporter := tracetest.NewInMemoryExporter() spanProcessor := trace.NewSimpleSpanProcessor(spanExporter) tracerProvider := trace.NewTracerProvider(trace.WithSpanProcessor(spanProcessor)) - textMapPropagator := propagation.NewCompositeTextMapPropagator(GRPCTraceBinPropagator{}) + textMapPropagator := propagation.NewCompositeTextMapPropagator(experimental.GRPCTraceBinPropagator{}) traceOptions := &TraceOptions{ TracerProvider: tracerProvider, TextMapPropagator: textMapPropagator, diff --git a/stats/opentelemetry/grpc_trace_bin_propagator.go b/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go similarity index 99% rename from stats/opentelemetry/grpc_trace_bin_propagator.go rename to stats/opentelemetry/experimental/grpc_trace_bin_propagator.go index e8a3986d4f4a..2b6742c2e61d 100644 --- a/stats/opentelemetry/grpc_trace_bin_propagator.go +++ b/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go @@ -16,7 +16,7 @@ * */ -package opentelemetry +package experimental import ( "context" diff --git a/stats/opentelemetry/grpc_trace_bin_propagator_test.go b/stats/opentelemetry/experimental/grpc_trace_bin_propagator_test.go similarity index 97% rename from stats/opentelemetry/grpc_trace_bin_propagator_test.go rename to stats/opentelemetry/experimental/grpc_trace_bin_propagator_test.go index 2d575af4a581..022743e9b9df 100644 --- a/stats/opentelemetry/grpc_trace_bin_propagator_test.go +++ b/stats/opentelemetry/experimental/grpc_trace_bin_propagator_test.go @@ -16,10 +16,11 @@ * */ -package opentelemetry +package experimental import ( "context" + "google.golang.org/grpc/internal/grpctest" "testing" "github.com/google/go-cmp/cmp" @@ -28,6 +29,14 @@ import ( itracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" ) +type s struct { + grpctest.Tester +} + +func Test(t *testing.T) { + grpctest.RunSubTests(t, s{}) +} + var validSpanContext = oteltrace.SpanContext{}.WithTraceID( oteltrace.TraceID{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16}).WithSpanID( oteltrace.SpanID{17, 18, 19, 20, 21, 22, 23, 24}).WithTraceFlags( From c30c44ac0fdb81586d2fcdbf9350fa97fa277992 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Wed, 11 Dec 2024 13:14:40 +0530 Subject: [PATCH 42/57] move TraceOptions api to experimental --- stats/opentelemetry/e2e_test.go | 6 ++-- stats/opentelemetry/experimental/api.go | 21 +++++++++++++ stats/opentelemetry/opentelemetry.go | 39 +++++++++---------------- 3 files changed, 38 insertions(+), 28 deletions(-) create mode 100644 stats/opentelemetry/experimental/api.go diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index f6b3f8cfc2bf..08bb3ff31595 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -84,12 +84,12 @@ func defaultMetricsOptions(_ *testing.T, methodAttributeFilter func(string) bool } // defaultTraceOptions function to create default trace options -func defaultTraceOptions(_ *testing.T) (*TraceOptions, *tracetest.InMemoryExporter) { +func defaultTraceOptions(_ *testing.T) (*experimental.TraceOptions, *tracetest.InMemoryExporter) { spanExporter := tracetest.NewInMemoryExporter() spanProcessor := trace.NewSimpleSpanProcessor(spanExporter) tracerProvider := trace.NewTracerProvider(trace.WithSpanProcessor(spanProcessor)) textMapPropagator := propagation.NewCompositeTextMapPropagator(experimental.GRPCTraceBinPropagator{}) - traceOptions := &TraceOptions{ + traceOptions := &experimental.TraceOptions{ TracerProvider: tracerProvider, TextMapPropagator: textMapPropagator, } @@ -98,7 +98,7 @@ func defaultTraceOptions(_ *testing.T) (*TraceOptions, *tracetest.InMemoryExport // setupStubServer creates a stub server with OpenTelemetry component configured on client // and server side and returns the server. -func setupStubServer(t *testing.T, metricsOptions *MetricsOptions, traceOptions *TraceOptions) *stubserver.StubServer { +func setupStubServer(t *testing.T, metricsOptions *MetricsOptions, traceOptions *experimental.TraceOptions) *stubserver.StubServer { ss := &stubserver.StubServer{ UnaryCallF: func(_ context.Context, in *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { return &testpb.SimpleResponse{Payload: &testpb.Payload{ diff --git a/stats/opentelemetry/experimental/api.go b/stats/opentelemetry/experimental/api.go new file mode 100644 index 000000000000..9308d874b934 --- /dev/null +++ b/stats/opentelemetry/experimental/api.go @@ -0,0 +1,21 @@ +package experimental + +import ( + "go.opentelemetry.io/otel/propagation" + "go.opentelemetry.io/otel/trace" +) + +// TraceOptions are the tracing options for OpenTelemetry instrumentation. +// +// # Experimental +// +// Notice: This type is EXPERIMENTAL and may be changed or removed in a +// later release. +type TraceOptions struct { + // TracerProvider is the OpenTelemetry tracer which is required to + // record traces/trace spans for instrumentation + TracerProvider trace.TracerProvider + + // TextMapPropagator propagates span context through text map carrier. + TextMapPropagator propagation.TextMapPropagator +} diff --git a/stats/opentelemetry/opentelemetry.go b/stats/opentelemetry/opentelemetry.go index 94e01b268233..ccbe351fe227 100644 --- a/stats/opentelemetry/opentelemetry.go +++ b/stats/opentelemetry/opentelemetry.go @@ -24,42 +24,41 @@ package opentelemetry import ( "context" + "google.golang.org/grpc/stats/opentelemetry/experimental" + otelinternal "google.golang.org/grpc/stats/opentelemetry/internal" "strings" "time" + otelattribute "go.opentelemetry.io/otel/attribute" + otelmetric "go.opentelemetry.io/otel/metric" + "go.opentelemetry.io/otel/metric/noop" + "go.opentelemetry.io/otel/trace" "google.golang.org/grpc" "google.golang.org/grpc/codes" estats "google.golang.org/grpc/experimental/stats" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/internal" "google.golang.org/grpc/stats" - otelinternal "google.golang.org/grpc/stats/opentelemetry/internal" - - otelattribute "go.opentelemetry.io/otel/attribute" - otelmetric "go.opentelemetry.io/otel/metric" - "go.opentelemetry.io/otel/metric/noop" - "go.opentelemetry.io/otel/propagation" - "go.opentelemetry.io/otel/trace" ) -func init() { - otelinternal.SetPluginOption = func(o *Options, po otelinternal.PluginOption) { - o.MetricsOptions.pluginOption = po - } -} - var logger = grpclog.Component("otel-plugin") var canonicalString = internal.CanonicalString.(func(codes.Code) string) var joinDialOptions = internal.JoinDialOptions.(func(...grpc.DialOption) grpc.DialOption) +func init() { + otelinternal.SetPluginOption = func(o *Options, po otelinternal.PluginOption) { + o.MetricsOptions.pluginOption = po + } +} + // Options are the options for OpenTelemetry instrumentation. type Options struct { // MetricsOptions are the metrics options for OpenTelemetry instrumentation. MetricsOptions MetricsOptions // TraceOptions are the tracing options for OpenTelemetry instrumentation. - TraceOptions TraceOptions + TraceOptions experimental.TraceOptions } // MetricsOptions are the metrics options for OpenTelemetry instrumentation. @@ -94,16 +93,6 @@ type MetricsOptions struct { pluginOption otelinternal.PluginOption } -// TraceOptions are the tracing options for OpenTelemetry instrumentation. -type TraceOptions struct { - // TracerProvider is the OpenTelemetry tracer which is required to - // record traces/trace spans for instrumentation - TracerProvider trace.TracerProvider - - // TextMapPropagator propagates span context through text map carrier. - TextMapPropagator propagation.TextMapPropagator -} - // DialOption returns a dial option which enables OpenTelemetry instrumentation // code for a grpc.ClientConn. // @@ -191,7 +180,7 @@ func isMetricsDisabled(mo MetricsOptions) bool { return mo.MeterProvider == nil } -func isTracingDisabled(to TraceOptions) bool { +func isTracingDisabled(to experimental.TraceOptions) bool { return to.TracerProvider == nil || to.TextMapPropagator == nil } From 90ffa2316e0e5d79951c14acabbd0f5ad57870fc Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Mon, 16 Dec 2024 22:10:03 +0530 Subject: [PATCH 43/57] addressed purnesh's comments --- stats/opentelemetry/e2e_test.go | 276 ++++++++++++++++-- stats/opentelemetry/experimental/api.go | 5 - .../experimental/grpc_trace_bin_propagator.go | 5 + stats/opentelemetry/trace.go | 2 +- 4 files changed, 264 insertions(+), 24 deletions(-) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index 08bb3ff31595..de3868a46856 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -14,7 +14,7 @@ * limitations under the License. */ -package opentelemetry +package opentelemetry_test import ( "context" @@ -59,10 +59,9 @@ import ( "google.golang.org/grpc/stats/opentelemetry/internal/testutils" ) -// traceSpanInfo is the information received about the span. This is a subset -// of information that is important to verify that gRPC has knobs over, which -// goes through a stable OpenTelemetry API with well-defined behavior. This keeps -// the robustness of assertions over time. +// traceSpanInfo is the information received about the trace span. It contains +// subset of information that is needed to verify if correct trace is being +// attributed to the rpc. type traceSpanInfo struct { sc trace2.SpanContext spanKind string @@ -615,11 +614,17 @@ func pollForWantMetrics(ctx context.Context, t *testing.T, reader *metric.Manual // OpenTelemetry instrumentation component. It then configures a system with a gRPC // Client and gRPC server with the OpenTelemetry Dial and Server Option configured // specifying all the metrics and traces provided by this package, and makes a Unary -// RPC and a Streaming RPC. These two RPCs should cause certain recording for each -// registered metric observed through a Manual Metrics Reader on the provided -// OpenTelemetry SDK's Meter Provider. It also verifies the traces are recorded -// correctly. -func (s) TestServerWithMetricsAndTraceOptions(t *testing.T) { +// RPC and a Streaming RPC. +// +// Metrics: +// - Verifies that certain metrics are recorded for each registered metric +// observed through a Manual Metrics Reader on the provided OpenTelemetry SDK's +// Meter Provider. +// +// Traces: +// - Confirms that tracing information is correctly recorded, including span creation, +// context propagation, and any relevant attributes associated with the RPC calls. +func TestServerWithMetricsAndTraceOptions(t *testing.T) { // Create default metrics options mo, reader := defaultMetricsOptions(t, nil) // Create default trace options @@ -665,24 +670,251 @@ func (s) TestServerWithMetricsAndTraceOptions(t *testing.T) { }) testutils.CompareMetrics(ctx, t, reader, gotMetrics, wantMetrics) + wantSI := []traceSpanInfo{ + { + name: "grpc.testing.TestService.UnaryCall", + spanKind: trace2.SpanKindServer.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + Value: attribute.IntValue(0), + }, + { + Key: "FailFast", + Value: attribute.IntValue(0), + }, + { + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), + }, + { + Key: "transparent-retry", + Value: attribute.IntValue(0), + }, + }, + events: []trace.Event{ + { + Name: "Inbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + Value: attribute.IntValue(1), + }, + { + Key: "message-size", + Value: attribute.IntValue(10006), + }, + { + Key: "message-size-compressed", + Value: attribute.IntValue(57), + }, + }, + }, + { + Name: "Outbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + Value: attribute.IntValue(1), + }, + { + Key: "message-size", + Value: attribute.IntValue(10006), + }, + { + Key: "message-size-compressed", + Value: attribute.IntValue(57), + }, + }, + }, + }, + }, + { + name: "Attempt.grpc.testing.TestService.UnaryCall", + spanKind: trace2.SpanKindInternal.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + Value: attribute.IntValue(1), + }, + { + Key: "FailFast", + Value: attribute.IntValue(1), + }, + { + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), + }, + { + Key: "transparent-retry", + Value: attribute.IntValue(0), + }, + }, + events: []trace.Event{ + { + Name: "Outbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + Value: attribute.IntValue(1), + }, + { + Key: "message-size", + Value: attribute.IntValue(10006), + }, + { + Key: "message-size-compressed", + Value: attribute.IntValue(57), + }, + }, + }, + { + Name: "Inbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + Value: attribute.IntValue(1), + }, + { + Key: "message-size", + Value: attribute.IntValue(10006), + }, + { + Key: "message-size-compressed", + Value: attribute.IntValue(57), + }, + }, + }, + }, + }, + { + name: "grpc.testing.TestService.UnaryCall", + spanKind: trace2.SpanKindClient.String(), + attributes: []attribute.KeyValue{}, + events: []trace.Event{}, + }, + { + name: "grpc.testing.TestService.FullDuplexCall", + spanKind: trace2.SpanKindServer.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + Value: attribute.IntValue(0), + }, + { + Key: "FailFast", + Value: attribute.IntValue(0), + }, + { + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), + }, + { + Key: "transparent-retry", + Value: attribute.IntValue(0), + }, + }, + events: []trace.Event{}, + }, + { + name: "grpc.testing.TestService.FullDuplexCall", + spanKind: trace2.SpanKindClient.String(), + attributes: []attribute.KeyValue{}, + events: []trace.Event{}, + }, + { + name: "Attempt.grpc.testing.TestService.FullDuplexCall", + spanKind: trace2.SpanKindInternal.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + Value: attribute.IntValue(1), + }, + { + Key: "FailFast", + Value: attribute.IntValue(1), + }, + { + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), + }, + { + Key: "transparent-retry", + Value: attribute.IntValue(0), + }, + }, + events: []trace.Event{}, + }, + } + // Verify traces spans := exporter.GetSpans() if got, want := len(spans), 6; got != want { - t.Fatalf("Got %d spans, want %d", got, want) + t.Fatalf("got %d spans, want %d", got, want) } // Add assertions for specific span attributes and events as needed. // For example, to check if the server span has the correct status: serverSpan := spans[0] if got, want := serverSpan.Status.Code, otelcodes.Ok; got != want { - t.Errorf("Got status code %v, want %v", got, want) + t.Errorf("got status code %v, want %v", got, want) } + + for index, span := range spans { + // Check that the attempt span has the correct status + if got, want := spans[index].Status.Code, otelcodes.Ok; got != want { + t.Errorf("Got status code %v, want %v", got, want) + } + // name + if got, want := span.Name, wantSI[index].name; got != want { + t.Errorf("Span name is %q, want %q", got, want) + } + // spanKind + if got, want := span.SpanKind.String(), wantSI[index].spanKind; got != want { + t.Errorf("Got span kind %q, want %q", got, want) + } + // attributes + if got, want := len(span.Attributes), len(wantSI[index].attributes); got != want { + t.Errorf("Got attributes list of size %q, want %q", got, want) + } + for idx, att := range span.Attributes { + if got, want := att.Key, wantSI[index].attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %v as %v, want %v", span.Name, got, want) + } + } + // events + if got, want := len(span.Events), len(wantSI[index].events); got != want { + t.Errorf("Event length is %q, want %q", got, want) + } + for eventIdx, event := range span.Events { + if got, want := event.Name, wantSI[index].events[eventIdx].Name; got != want { + t.Errorf("Got event name for span name %q as %q, want %q", span.Name, got, want) + } + for idx, att := range event.Attributes { + if got, want := att.Key, wantSI[index].events[eventIdx].Attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %q with event name %v, as %v, want %v", span.Name, event.Name, got, want) + } + if got, want := att.Value, wantSI[index].events[eventIdx].Attributes[idx].Value; got != want { + t.Errorf("Got attribute value for span name %v with event name %v, as %v, want %v", span.Name, event.Name, got, want) + } + } + } + } + } // TestSpan verifies that the gRPC Trace Binary propagator correctly // propagates span context between a client and server using the grpc- // trace-bin header. It sets up a stub server with OpenTelemetry tracing -// enabled, makes a unary RPC. +// enabled, makes a unary RPC, and streaming RPC as well. +// +// Verification: +// - Ensures that the span context is correctly propagated from the client +// to the server, including the trace ID and span ID. +// - Confirms that the server can access the span context and create +// child spans as expected during the RPC calls. +// - Validates that the tracing information is recorded accurately in +// the OpenTelemetry backend. func (s) TestSpan(t *testing.T) { mo, _ := defaultMetricsOptions(t, nil) // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter @@ -714,7 +946,7 @@ func (s) TestSpan(t *testing.T) { // Get the spans from the exporter spans := spanExporter.GetSpans() if got, want := len(spans), 6; got != want { - t.Fatalf("Got %d spans, want %d", got, want) + t.Fatalf("got %d spans, want %d", got, want) } wantSI := []traceSpanInfo{ @@ -894,7 +1126,7 @@ func (s) TestSpan(t *testing.T) { }, } - // Check that same traceID is used in client and server. + // Check that same traceID is used in client and server for unary RPC call. if got, want := spans[0].SpanContext.TraceID(), spans[2].SpanContext.TraceID(); got != want { t.Fatal("TraceID mismatch in client span and server span.") } @@ -904,7 +1136,7 @@ func (s) TestSpan(t *testing.T) { t.Fatal("SpanID mismatch in client span and server span.") } - // Check that same traceID is used in client and server. + // Check that same traceID is used in client and server for streaming RPC call. if got, want := spans[3].SpanContext.TraceID(), spans[4].SpanContext.TraceID(); got != want { t.Fatal("TraceID mismatch in client span and server span.") } @@ -956,9 +1188,17 @@ func (s) TestSpan(t *testing.T) { } } -// TestSpan_WithW3CContextPropagator sets up a stub server with OpenTelemetry tracing -// enabled makes a unary and a streaming RPC, and then, asserts that the correct +// TestSpan_WithW3CContextPropagator sets up a stub server with OpenTelemetry tracing +// enabled, makes a unary and a streaming RPC, and then asserts that the correct // number of spans are created with the expected spans. +// +// Verification: +// - Confirms that the correct number of spans are created for both unary and +// streaming RPCs. +// - Validates that the spans have the expected names and attributes, ensuring +// they accurately reflect the operations performed. +// - Checks that the trace ID and span ID are correctly assigned and accessible +// in the OpenTelemetry backend. func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { mo, _ := defaultMetricsOptions(t, nil) // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter diff --git a/stats/opentelemetry/experimental/api.go b/stats/opentelemetry/experimental/api.go index 9308d874b934..1ac5dfa106ba 100644 --- a/stats/opentelemetry/experimental/api.go +++ b/stats/opentelemetry/experimental/api.go @@ -6,11 +6,6 @@ import ( ) // TraceOptions are the tracing options for OpenTelemetry instrumentation. -// -// # Experimental -// -// Notice: This type is EXPERIMENTAL and may be changed or removed in a -// later release. type TraceOptions struct { // TracerProvider is the OpenTelemetry tracer which is required to // record traces/trace spans for instrumentation diff --git a/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go b/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go index 2b6742c2e61d..b6cd460f231a 100644 --- a/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go +++ b/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go @@ -16,6 +16,11 @@ * */ +// # Experimental +// +// Notice: This package is EXPERIMENTAL and may be changed or removed in a +// later release. + package experimental import ( diff --git a/stats/opentelemetry/trace.go b/stats/opentelemetry/trace.go index 0101fc85f752..0251e5da1408 100644 --- a/stats/opentelemetry/trace.go +++ b/stats/opentelemetry/trace.go @@ -70,7 +70,7 @@ func (h *serverStatsHandler) traceTagRPC(ctx context.Context, rti *stats.RPCTagI tracer := otel.Tracer("grpc-open-telemetry") ctx = otel.GetTextMapPropagator().Extract(ctx, otelinternaltracing.NewIncomingCarrier(ctx)) // If the context.Context provided in `ctx` to tracer.Start(), contains a - // Span then the newly-created Span will be a child of that span, + // span then the newly-created Span will be a child of that span, // otherwise it will be a root span. ctx, span = tracer.Start(ctx, mn, trace.WithSpanKind(trace.SpanKindServer)) return ctx, &attemptInfo{ From 208383099378baa3d7525c232a76a900c816bed6 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Mon, 16 Dec 2024 22:18:01 +0530 Subject: [PATCH 44/57] don't change opentelemetry/e2e_test.go package name from opentelemetry_test to opentelemetry --- stats/opentelemetry/e2e_test.go | 32 ++++++++++++++------- stats/opentelemetry/metricsregistry_test.go | 4 +-- 2 files changed, 24 insertions(+), 12 deletions(-) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index de3868a46856..19fc91caf3c6 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -19,6 +19,8 @@ package opentelemetry_test import ( "context" "fmt" + "google.golang.org/grpc/internal/grpctest" + "google.golang.org/grpc/stats/opentelemetry" "google.golang.org/grpc/stats/opentelemetry/experimental" "io" "testing" @@ -59,6 +61,16 @@ import ( "google.golang.org/grpc/stats/opentelemetry/internal/testutils" ) +var defaultTestTimeout = 5 * time.Second + +type s struct { + grpctest.Tester +} + +func Test(t *testing.T) { + grpctest.RunSubTests(t, s{}) +} + // traceSpanInfo is the information received about the trace span. It contains // subset of information that is needed to verify if correct trace is being // attributed to the rpc. @@ -71,12 +83,12 @@ type traceSpanInfo struct { } // defaultMetricsOptions creates default metrics options -func defaultMetricsOptions(_ *testing.T, methodAttributeFilter func(string) bool) (*MetricsOptions, *metric.ManualReader) { +func defaultMetricsOptions(_ *testing.T, methodAttributeFilter func(string) bool) (*opentelemetry.MetricsOptions, *metric.ManualReader) { reader := metric.NewManualReader() provider := metric.NewMeterProvider(metric.WithReader(reader)) - metricsOptions := &MetricsOptions{ + metricsOptions := &opentelemetry.MetricsOptions{ MeterProvider: provider, - Metrics: DefaultMetrics(), + Metrics: opentelemetry.DefaultMetrics(), MethodAttributeFilter: methodAttributeFilter, } return metricsOptions, reader @@ -97,7 +109,7 @@ func defaultTraceOptions(_ *testing.T) (*experimental.TraceOptions, *tracetest.I // setupStubServer creates a stub server with OpenTelemetry component configured on client // and server side and returns the server. -func setupStubServer(t *testing.T, metricsOptions *MetricsOptions, traceOptions *experimental.TraceOptions) *stubserver.StubServer { +func setupStubServer(t *testing.T, metricsOptions *opentelemetry.MetricsOptions, traceOptions *experimental.TraceOptions) *stubserver.StubServer { ss := &stubserver.StubServer{ UnaryCallF: func(_ context.Context, in *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { return &testpb.SimpleResponse{Payload: &testpb.Payload{ @@ -114,7 +126,7 @@ func setupStubServer(t *testing.T, metricsOptions *MetricsOptions, traceOptions }, } - otelOptions := Options{} + otelOptions := opentelemetry.Options{} if metricsOptions != nil { otelOptions.MetricsOptions = *metricsOptions } @@ -122,8 +134,8 @@ func setupStubServer(t *testing.T, metricsOptions *MetricsOptions, traceOptions otelOptions.TraceOptions = *traceOptions } - if err := ss.Start([]grpc.ServerOption{ServerOption(otelOptions)}, - DialOption(otelOptions)); err != nil { + if err := ss.Start([]grpc.ServerOption{opentelemetry.ServerOption(otelOptions)}, + opentelemetry.DialOption(otelOptions)); err != nil { t.Fatalf("Error starting endpoint server: %v", err) } return ss @@ -479,14 +491,14 @@ func (s) TestWRRMetrics(t *testing.T) { reader := metric.NewManualReader() provider := metric.NewMeterProvider(metric.WithReader(reader)) - mo := MetricsOptions{ + mo := opentelemetry.MetricsOptions{ MeterProvider: provider, - Metrics: DefaultMetrics().Add("grpc.lb.wrr.rr_fallback", "grpc.lb.wrr.endpoint_weight_not_yet_usable", "grpc.lb.wrr.endpoint_weight_stale", "grpc.lb.wrr.endpoint_weights"), + Metrics: opentelemetry.DefaultMetrics().Add("grpc.lb.wrr.rr_fallback", "grpc.lb.wrr.endpoint_weight_not_yet_usable", "grpc.lb.wrr.endpoint_weight_stale", "grpc.lb.wrr.endpoint_weights"), OptionalLabels: []string{"grpc.lb.locality"}, } target := fmt.Sprintf("xds:///%s", serviceName) - cc, err := grpc.NewClient(target, grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(xdsResolver), DialOption(Options{MetricsOptions: mo})) + cc, err := grpc.NewClient(target, grpc.WithTransportCredentials(insecure.NewCredentials()), grpc.WithResolvers(xdsResolver), opentelemetry.DialOption(opentelemetry.Options{MetricsOptions: mo})) if err != nil { t.Fatalf("Failed to dial local test server: %v", err) } diff --git a/stats/opentelemetry/metricsregistry_test.go b/stats/opentelemetry/metricsregistry_test.go index 67a2d7faeadc..7cd979d1a73d 100644 --- a/stats/opentelemetry/metricsregistry_test.go +++ b/stats/opentelemetry/metricsregistry_test.go @@ -31,7 +31,7 @@ import ( "go.opentelemetry.io/otel/sdk/metric/metricdata/metricdatatest" ) -var defaultTestTimeout = 5 * time.Second +var DefaultTestTimeout = 5 * time.Second type s struct { grpctest.Tester @@ -130,7 +130,7 @@ func (s) TestMetricsRegistryMetrics(t *testing.T) { Default: true, }) - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + ctx, cancel := context.WithTimeout(context.Background(), DefaultTestTimeout) defer cancel() // Only float optional labels are configured, so only float optional labels should show up. From e8e9d536c84e51393a85780052bd8d899480248b Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Mon, 16 Dec 2024 22:21:18 +0530 Subject: [PATCH 45/57] make vet happy --- stats/opentelemetry/e2e_test.go | 6 +++--- .../experimental/grpc_trace_bin_propagator_test.go | 2 +- stats/opentelemetry/opentelemetry.go | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index 19fc91caf3c6..84cda8e4eee0 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -19,9 +19,6 @@ package opentelemetry_test import ( "context" "fmt" - "google.golang.org/grpc/internal/grpctest" - "google.golang.org/grpc/stats/opentelemetry" - "google.golang.org/grpc/stats/opentelemetry/experimental" "io" "testing" "time" @@ -51,6 +48,7 @@ import ( "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/encoding/gzip" "google.golang.org/grpc/internal/grpcsync" + "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/stubserver" itestutils "google.golang.org/grpc/internal/testutils" "google.golang.org/grpc/internal/testutils/xds/e2e" @@ -58,6 +56,8 @@ import ( testgrpc "google.golang.org/grpc/interop/grpc_testing" testpb "google.golang.org/grpc/interop/grpc_testing" "google.golang.org/grpc/orca" + "google.golang.org/grpc/stats/opentelemetry" + "google.golang.org/grpc/stats/opentelemetry/experimental" "google.golang.org/grpc/stats/opentelemetry/internal/testutils" ) diff --git a/stats/opentelemetry/experimental/grpc_trace_bin_propagator_test.go b/stats/opentelemetry/experimental/grpc_trace_bin_propagator_test.go index 022743e9b9df..3519ddbd2fcd 100644 --- a/stats/opentelemetry/experimental/grpc_trace_bin_propagator_test.go +++ b/stats/opentelemetry/experimental/grpc_trace_bin_propagator_test.go @@ -20,11 +20,11 @@ package experimental import ( "context" - "google.golang.org/grpc/internal/grpctest" "testing" "github.com/google/go-cmp/cmp" oteltrace "go.opentelemetry.io/otel/trace" + "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/metadata" itracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" ) diff --git a/stats/opentelemetry/opentelemetry.go b/stats/opentelemetry/opentelemetry.go index ccbe351fe227..c200ff86dce7 100644 --- a/stats/opentelemetry/opentelemetry.go +++ b/stats/opentelemetry/opentelemetry.go @@ -24,8 +24,6 @@ package opentelemetry import ( "context" - "google.golang.org/grpc/stats/opentelemetry/experimental" - otelinternal "google.golang.org/grpc/stats/opentelemetry/internal" "strings" "time" @@ -39,6 +37,8 @@ import ( "google.golang.org/grpc/grpclog" "google.golang.org/grpc/internal" "google.golang.org/grpc/stats" + "google.golang.org/grpc/stats/opentelemetry/experimental" + otelinternal "google.golang.org/grpc/stats/opentelemetry/internal" ) var logger = grpclog.Component("otel-plugin") From 57fd38abf9357a4640ed4a1d64a3ccefbdddb9bf Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Tue, 17 Dec 2024 11:31:40 +0530 Subject: [PATCH 46/57] update TestServerWithMetricsAndTraceOptions --- stats/opentelemetry/e2e_test.go | 253 +------------------------------- 1 file changed, 6 insertions(+), 247 deletions(-) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index 84cda8e4eee0..5fc38bdd3d69 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -622,25 +622,16 @@ func pollForWantMetrics(ctx context.Context, t *testing.T, reader *metric.Manual return fmt.Errorf("error waiting for metrics %v: %v", wantMetrics, ctx.Err()) } -// TestServerWithMetricsAndTraceOptions tests emitted metrics and traces from -// OpenTelemetry instrumentation component. It then configures a system with a gRPC -// Client and gRPC server with the OpenTelemetry Dial and Server Option configured -// specifying all the metrics and traces provided by this package, and makes a Unary -// RPC and a Streaming RPC. -// -// Metrics: -// - Verifies that certain metrics are recorded for each registered metric -// observed through a Manual Metrics Reader on the provided OpenTelemetry SDK's -// Meter Provider. -// -// Traces: -// - Confirms that tracing information is correctly recorded, including span creation, -// context propagation, and any relevant attributes associated with the RPC calls. +// TestServerWithMetricsAndTraceOptions verifies the integration of metrics and traces +// emitted by the OpenTelemetry instrumentation in a gRPC environment. It tests the +// correct emission of metrics during a Unary RPC and a Streaming RPC, ensuring that +// the metrics reflect the operations performed, including the size of the compressed +// message. func TestServerWithMetricsAndTraceOptions(t *testing.T) { // Create default metrics options mo, reader := defaultMetricsOptions(t, nil) // Create default trace options - to, exporter := defaultTraceOptions(t) + to, _ := defaultTraceOptions(t) ss := setupStubServer(t, mo, to) defer ss.Stop() @@ -681,238 +672,6 @@ func TestServerWithMetricsAndTraceOptions(t *testing.T) { UnaryCompressedMessageSize: float64(57), }) testutils.CompareMetrics(ctx, t, reader, gotMetrics, wantMetrics) - - wantSI := []traceSpanInfo{ - { - name: "grpc.testing.TestService.UnaryCall", - spanKind: trace2.SpanKindServer.String(), - attributes: []attribute.KeyValue{ - { - Key: "Client", - Value: attribute.IntValue(0), - }, - { - Key: "FailFast", - Value: attribute.IntValue(0), - }, - { - Key: "previous-rpc-attempts", - Value: attribute.IntValue(0), - }, - { - Key: "transparent-retry", - Value: attribute.IntValue(0), - }, - }, - events: []trace.Event{ - { - Name: "Inbound compressed message", - Attributes: []attribute.KeyValue{ - { - Key: "sequence-number", - Value: attribute.IntValue(1), - }, - { - Key: "message-size", - Value: attribute.IntValue(10006), - }, - { - Key: "message-size-compressed", - Value: attribute.IntValue(57), - }, - }, - }, - { - Name: "Outbound compressed message", - Attributes: []attribute.KeyValue{ - { - Key: "sequence-number", - Value: attribute.IntValue(1), - }, - { - Key: "message-size", - Value: attribute.IntValue(10006), - }, - { - Key: "message-size-compressed", - Value: attribute.IntValue(57), - }, - }, - }, - }, - }, - { - name: "Attempt.grpc.testing.TestService.UnaryCall", - spanKind: trace2.SpanKindInternal.String(), - attributes: []attribute.KeyValue{ - { - Key: "Client", - Value: attribute.IntValue(1), - }, - { - Key: "FailFast", - Value: attribute.IntValue(1), - }, - { - Key: "previous-rpc-attempts", - Value: attribute.IntValue(0), - }, - { - Key: "transparent-retry", - Value: attribute.IntValue(0), - }, - }, - events: []trace.Event{ - { - Name: "Outbound compressed message", - Attributes: []attribute.KeyValue{ - { - Key: "sequence-number", - Value: attribute.IntValue(1), - }, - { - Key: "message-size", - Value: attribute.IntValue(10006), - }, - { - Key: "message-size-compressed", - Value: attribute.IntValue(57), - }, - }, - }, - { - Name: "Inbound compressed message", - Attributes: []attribute.KeyValue{ - { - Key: "sequence-number", - Value: attribute.IntValue(1), - }, - { - Key: "message-size", - Value: attribute.IntValue(10006), - }, - { - Key: "message-size-compressed", - Value: attribute.IntValue(57), - }, - }, - }, - }, - }, - { - name: "grpc.testing.TestService.UnaryCall", - spanKind: trace2.SpanKindClient.String(), - attributes: []attribute.KeyValue{}, - events: []trace.Event{}, - }, - { - name: "grpc.testing.TestService.FullDuplexCall", - spanKind: trace2.SpanKindServer.String(), - attributes: []attribute.KeyValue{ - { - Key: "Client", - Value: attribute.IntValue(0), - }, - { - Key: "FailFast", - Value: attribute.IntValue(0), - }, - { - Key: "previous-rpc-attempts", - Value: attribute.IntValue(0), - }, - { - Key: "transparent-retry", - Value: attribute.IntValue(0), - }, - }, - events: []trace.Event{}, - }, - { - name: "grpc.testing.TestService.FullDuplexCall", - spanKind: trace2.SpanKindClient.String(), - attributes: []attribute.KeyValue{}, - events: []trace.Event{}, - }, - { - name: "Attempt.grpc.testing.TestService.FullDuplexCall", - spanKind: trace2.SpanKindInternal.String(), - attributes: []attribute.KeyValue{ - { - Key: "Client", - Value: attribute.IntValue(1), - }, - { - Key: "FailFast", - Value: attribute.IntValue(1), - }, - { - Key: "previous-rpc-attempts", - Value: attribute.IntValue(0), - }, - { - Key: "transparent-retry", - Value: attribute.IntValue(0), - }, - }, - events: []trace.Event{}, - }, - } - - // Verify traces - spans := exporter.GetSpans() - if got, want := len(spans), 6; got != want { - t.Fatalf("got %d spans, want %d", got, want) - } - - // Add assertions for specific span attributes and events as needed. - // For example, to check if the server span has the correct status: - serverSpan := spans[0] - if got, want := serverSpan.Status.Code, otelcodes.Ok; got != want { - t.Errorf("got status code %v, want %v", got, want) - } - - for index, span := range spans { - // Check that the attempt span has the correct status - if got, want := spans[index].Status.Code, otelcodes.Ok; got != want { - t.Errorf("Got status code %v, want %v", got, want) - } - // name - if got, want := span.Name, wantSI[index].name; got != want { - t.Errorf("Span name is %q, want %q", got, want) - } - // spanKind - if got, want := span.SpanKind.String(), wantSI[index].spanKind; got != want { - t.Errorf("Got span kind %q, want %q", got, want) - } - // attributes - if got, want := len(span.Attributes), len(wantSI[index].attributes); got != want { - t.Errorf("Got attributes list of size %q, want %q", got, want) - } - for idx, att := range span.Attributes { - if got, want := att.Key, wantSI[index].attributes[idx].Key; got != want { - t.Errorf("Got attribute key for span name %v as %v, want %v", span.Name, got, want) - } - } - // events - if got, want := len(span.Events), len(wantSI[index].events); got != want { - t.Errorf("Event length is %q, want %q", got, want) - } - for eventIdx, event := range span.Events { - if got, want := event.Name, wantSI[index].events[eventIdx].Name; got != want { - t.Errorf("Got event name for span name %q as %q, want %q", span.Name, got, want) - } - for idx, att := range event.Attributes { - if got, want := att.Key, wantSI[index].events[eventIdx].Attributes[idx].Key; got != want { - t.Errorf("Got attribute key for span name %q with event name %v, as %v, want %v", span.Name, event.Name, got, want) - } - if got, want := att.Value, wantSI[index].events[eventIdx].Attributes[idx].Value; got != want { - t.Errorf("Got attribute value for span name %v with event name %v, as %v, want %v", span.Name, event.Name, got, want) - } - } - } - } - } // TestSpan verifies that the gRPC Trace Binary propagator correctly From b0aad8a84ed901612c91e9b34584cf621f8105e5 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Tue, 17 Dec 2024 14:07:23 +0530 Subject: [PATCH 47/57] fixed nits --- stats/opentelemetry/e2e_test.go | 17 ++++++++++------- stats/opentelemetry/experimental/api.go | 16 ++++++++++++++++ 2 files changed, 26 insertions(+), 7 deletions(-) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index 5fc38bdd3d69..d41eeebdc005 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -75,7 +75,6 @@ func Test(t *testing.T) { // subset of information that is needed to verify if correct trace is being // attributed to the rpc. type traceSpanInfo struct { - sc trace2.SpanContext spanKind string name string events []trace.Event @@ -627,11 +626,11 @@ func pollForWantMetrics(ctx context.Context, t *testing.T, reader *metric.Manual // correct emission of metrics during a Unary RPC and a Streaming RPC, ensuring that // the metrics reflect the operations performed, including the size of the compressed // message. -func TestServerWithMetricsAndTraceOptions(t *testing.T) { +func (s) TestMetricsAndTracesOptionEnabled(t *testing.T) { // Create default metrics options mo, reader := defaultMetricsOptions(t, nil) // Create default trace options - to, _ := defaultTraceOptions(t) + to, exporter := defaultTraceOptions(t) ss := setupStubServer(t, mo, to) defer ss.Stop() @@ -672,6 +671,12 @@ func TestServerWithMetricsAndTraceOptions(t *testing.T) { UnaryCompressedMessageSize: float64(57), }) testutils.CompareMetrics(ctx, t, reader, gotMetrics, wantMetrics) + + // Verify traces + spans := exporter.GetSpans() + if got, want := len(spans), 6; got != want { + t.Fatalf("got %d spans, want %d", got, want) + } } // TestSpan verifies that the gRPC Trace Binary propagator correctly @@ -687,11 +692,10 @@ func TestServerWithMetricsAndTraceOptions(t *testing.T) { // - Validates that the tracing information is recorded accurately in // the OpenTelemetry backend. func (s) TestSpan(t *testing.T) { - mo, _ := defaultMetricsOptions(t, nil) // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter traceOptions, spanExporter := defaultTraceOptions(t) // Start the server with OpenTelemetry options - ss := setupStubServer(t, mo, traceOptions) + ss := setupStubServer(t, nil, traceOptions) defer ss.Stop() ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) @@ -971,13 +975,12 @@ func (s) TestSpan(t *testing.T) { // - Checks that the trace ID and span ID are correctly assigned and accessible // in the OpenTelemetry backend. func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { - mo, _ := defaultMetricsOptions(t, nil) // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter traceOptions, spanExporter := defaultTraceOptions(t) // Set the W3CContextPropagator as part of TracingOptions. traceOptions.TextMapPropagator = propagation.NewCompositeTextMapPropagator(propagation.TraceContext{}) // Start the server with OpenTelemetry options - ss := setupStubServer(t, mo, traceOptions) + ss := setupStubServer(t, nil, traceOptions) defer ss.Stop() ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) diff --git a/stats/opentelemetry/experimental/api.go b/stats/opentelemetry/experimental/api.go index 1ac5dfa106ba..0d5ad32ebbe4 100644 --- a/stats/opentelemetry/experimental/api.go +++ b/stats/opentelemetry/experimental/api.go @@ -1,3 +1,19 @@ +/* + * Copyright 2024 gRPC 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 experimental import ( From 8680ae8e10192ef73243293870c2be67af5241c1 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Tue, 17 Dec 2024 17:05:45 +0530 Subject: [PATCH 48/57] fixed nits --- stats/opentelemetry/e2e_test.go | 40 ++++++++++++++++----------------- stream.go | 2 +- 2 files changed, 21 insertions(+), 21 deletions(-) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index d41eeebdc005..dcd383d8e56f 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -24,7 +24,7 @@ import ( "time" otelcodes "go.opentelemetry.io/otel/codes" - trace2 "go.opentelemetry.io/otel/trace" + oteltrace "go.opentelemetry.io/otel/trace" v3clusterpb "github.com/envoyproxy/go-control-plane/envoy/config/cluster/v3" v3corepb "github.com/envoyproxy/go-control-plane/envoy/config/core/v3" @@ -685,11 +685,11 @@ func (s) TestMetricsAndTracesOptionEnabled(t *testing.T) { // enabled, makes a unary RPC, and streaming RPC as well. // // Verification: -// - Ensures that the span context is correctly propagated from the client +// - Verifies that the span context is correctly propagated from the client // to the server, including the trace ID and span ID. -// - Confirms that the server can access the span context and create +// - Verifies that the server can access the span context and create // child spans as expected during the RPC calls. -// - Validates that the tracing information is recorded accurately in +// - Verifies that the tracing information is recorded accurately in // the OpenTelemetry backend. func (s) TestSpan(t *testing.T) { // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter @@ -713,11 +713,11 @@ func (s) TestSpan(t *testing.T) { if err != nil { t.Fatalf("ss.Client.FullDuplexCall failed: %f", err) } - stream.CloseSend() if _, err = stream.Recv(); err != io.EOF { t.Fatalf("stream.Recv received an unexpected error: %v, expected an EOF error", err) } + // Get the spans from the exporter spans := spanExporter.GetSpans() if got, want := len(spans), 6; got != want { @@ -727,7 +727,7 @@ func (s) TestSpan(t *testing.T) { wantSI := []traceSpanInfo{ { name: "grpc.testing.TestService.UnaryCall", - spanKind: trace2.SpanKindServer.String(), + spanKind: oteltrace.SpanKindServer.String(), attributes: []attribute.KeyValue{ { Key: "Client", @@ -785,7 +785,7 @@ func (s) TestSpan(t *testing.T) { }, { name: "Attempt.grpc.testing.TestService.UnaryCall", - spanKind: trace2.SpanKindInternal.String(), + spanKind: oteltrace.SpanKindInternal.String(), attributes: []attribute.KeyValue{ { Key: "Client", @@ -843,13 +843,13 @@ func (s) TestSpan(t *testing.T) { }, { name: "grpc.testing.TestService.UnaryCall", - spanKind: trace2.SpanKindClient.String(), + spanKind: oteltrace.SpanKindClient.String(), attributes: []attribute.KeyValue{}, events: []trace.Event{}, }, { name: "grpc.testing.TestService.FullDuplexCall", - spanKind: trace2.SpanKindServer.String(), + spanKind: oteltrace.SpanKindServer.String(), attributes: []attribute.KeyValue{ { Key: "Client", @@ -872,13 +872,13 @@ func (s) TestSpan(t *testing.T) { }, { name: "grpc.testing.TestService.FullDuplexCall", - spanKind: trace2.SpanKindClient.String(), + spanKind: oteltrace.SpanKindClient.String(), attributes: []attribute.KeyValue{}, events: []trace.Event{}, }, { name: "Attempt.grpc.testing.TestService.FullDuplexCall", - spanKind: trace2.SpanKindInternal.String(), + spanKind: oteltrace.SpanKindInternal.String(), attributes: []attribute.KeyValue{ { Key: "Client", @@ -968,11 +968,11 @@ func (s) TestSpan(t *testing.T) { // number of spans are created with the expected spans. // // Verification: -// - Confirms that the correct number of spans are created for both unary and +// - Verifies that the correct number of spans are created for both unary and // streaming RPCs. -// - Validates that the spans have the expected names and attributes, ensuring +// - Verifies that the spans have the expected names and attributes, ensuring // they accurately reflect the operations performed. -// - Checks that the trace ID and span ID are correctly assigned and accessible +// - Verifies that the trace ID and span ID are correctly assigned and accessible // in the OpenTelemetry backend. func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter @@ -1012,7 +1012,7 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { wantSI := []traceSpanInfo{ { name: "grpc.testing.TestService.UnaryCall", - spanKind: trace2.SpanKindServer.String(), + spanKind: oteltrace.SpanKindServer.String(), attributes: []attribute.KeyValue{ { Key: "Client", @@ -1070,7 +1070,7 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { }, { name: "Attempt.grpc.testing.TestService.UnaryCall", - spanKind: trace2.SpanKindInternal.String(), + spanKind: oteltrace.SpanKindInternal.String(), attributes: []attribute.KeyValue{ { Key: "Client", @@ -1128,13 +1128,13 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { }, { name: "grpc.testing.TestService.UnaryCall", - spanKind: trace2.SpanKindClient.String(), + spanKind: oteltrace.SpanKindClient.String(), attributes: []attribute.KeyValue{}, events: []trace.Event{}, }, { name: "grpc.testing.TestService.FullDuplexCall", - spanKind: trace2.SpanKindServer.String(), + spanKind: oteltrace.SpanKindServer.String(), attributes: []attribute.KeyValue{ { Key: "Client", @@ -1157,13 +1157,13 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { }, { name: "grpc.testing.TestService.FullDuplexCall", - spanKind: trace2.SpanKindClient.String(), + spanKind: oteltrace.SpanKindClient.String(), attributes: []attribute.KeyValue{}, events: []trace.Event{}, }, { name: "Attempt.grpc.testing.TestService.FullDuplexCall", - spanKind: trace2.SpanKindInternal.String(), + spanKind: oteltrace.SpanKindInternal.String(), attributes: []attribute.KeyValue{ { Key: "Client", diff --git a/stream.go b/stream.go index 7309e1eb6d07..3406a7d9744f 100644 --- a/stream.go +++ b/stream.go @@ -218,7 +218,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth } var mc serviceconfig.MethodConfig var onCommit func() - var newStream = func(ctx context.Context, done func()) (iresolver.ClientStream, error) { + newStream := func(ctx context.Context, done func()) (iresolver.ClientStream, error) { return newClientStreamWithParams(ctx, desc, cc, method, mc, onCommit, done, cc.nameResolutionDelayed, opts...) } From 98d11b7fb5875dd2bc6552bfc0f4aaca86ba8e88 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Wed, 18 Dec 2024 15:33:59 +0530 Subject: [PATCH 49/57] fix: small nits --- stats/opentelemetry/e2e_test.go | 265 +++++++++++++++++- .../experimental/{api.go => trace_options.go} | 0 2 files changed, 252 insertions(+), 13 deletions(-) rename stats/opentelemetry/experimental/{api.go => trace_options.go} (100%) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index dcd383d8e56f..69e657071fd6 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -240,8 +240,8 @@ func (s) TestAllMetricsOneFunction(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() // Make two RPC's, a unary RPC and a streaming RPC. These should cause - // certain metrics to be emitted, which should be able to be observed - // through the Metric Reader. + // certain metrics to be emitted, which should be observed through the + // Metric Reader. if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ Body: make([]byte, 10000), }}, grpc.UseCompressor(gzip.Name)); err != nil { // Deterministic compression. @@ -621,11 +621,12 @@ func pollForWantMetrics(ctx context.Context, t *testing.T, reader *metric.Manual return fmt.Errorf("error waiting for metrics %v: %v", wantMetrics, ctx.Err()) } -// TestServerWithMetricsAndTraceOptions verifies the integration of metrics and traces -// emitted by the OpenTelemetry instrumentation in a gRPC environment. It tests the -// correct emission of metrics during a Unary RPC and a Streaming RPC, ensuring that -// the metrics reflect the operations performed, including the size of the compressed -// message. +// TestMetricsAndTracesOptionEnabled verifies the integration of metrics and traces +// emitted by the OpenTelemetry instrumentation in a gRPC environment. It sets up a +// stub server with both metrics and traces enabled, and tests the correct emission +// of metrics during a Unary RPC and a Streaming RPC. The test ensures that the +// emitted metrics reflect the operations performed, including the size of the +// compressed message, and verifies that tracing information is correctly recorded. func (s) TestMetricsAndTracesOptionEnabled(t *testing.T) { // Create default metrics options mo, reader := defaultMetricsOptions(t, nil) @@ -677,6 +678,244 @@ func (s) TestMetricsAndTracesOptionEnabled(t *testing.T) { if got, want := len(spans), 6; got != want { t.Fatalf("got %d spans, want %d", got, want) } + + wantSI := []traceSpanInfo{ + { + name: "grpc.testing.TestService.UnaryCall", + spanKind: oteltrace.SpanKindServer.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + Value: attribute.IntValue(0), + }, + { + Key: "FailFast", + Value: attribute.IntValue(0), + }, + { + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), + }, + { + Key: "transparent-retry", + Value: attribute.IntValue(0), + }, + }, + events: []trace.Event{ + { + Name: "Inbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + Value: attribute.IntValue(1), + }, + { + Key: "message-size", + Value: attribute.IntValue(10006), + }, + { + Key: "message-size-compressed", + Value: attribute.IntValue(57), + }, + }, + }, + { + Name: "Outbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + Value: attribute.IntValue(1), + }, + { + Key: "message-size", + Value: attribute.IntValue(10006), + }, + { + Key: "message-size-compressed", + Value: attribute.IntValue(57), + }, + }, + }, + }, + }, + { + name: "Attempt.grpc.testing.TestService.UnaryCall", + spanKind: oteltrace.SpanKindInternal.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + Value: attribute.IntValue(1), + }, + { + Key: "FailFast", + Value: attribute.IntValue(1), + }, + { + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), + }, + { + Key: "transparent-retry", + Value: attribute.IntValue(0), + }, + }, + events: []trace.Event{ + { + Name: "Outbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + Value: attribute.IntValue(1), + }, + { + Key: "message-size", + Value: attribute.IntValue(10006), + }, + { + Key: "message-size-compressed", + Value: attribute.IntValue(57), + }, + }, + }, + { + Name: "Inbound compressed message", + Attributes: []attribute.KeyValue{ + { + Key: "sequence-number", + Value: attribute.IntValue(1), + }, + { + Key: "message-size", + Value: attribute.IntValue(10006), + }, + { + Key: "message-size-compressed", + Value: attribute.IntValue(57), + }, + }, + }, + }, + }, + { + name: "grpc.testing.TestService.UnaryCall", + spanKind: oteltrace.SpanKindClient.String(), + attributes: []attribute.KeyValue{}, + events: []trace.Event{}, + }, + { + name: "grpc.testing.TestService.FullDuplexCall", + spanKind: oteltrace.SpanKindServer.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + Value: attribute.IntValue(0), + }, + { + Key: "FailFast", + Value: attribute.IntValue(0), + }, + { + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), + }, + { + Key: "transparent-retry", + Value: attribute.IntValue(0), + }, + }, + events: []trace.Event{}, + }, + { + name: "grpc.testing.TestService.FullDuplexCall", + spanKind: oteltrace.SpanKindClient.String(), + attributes: []attribute.KeyValue{}, + events: []trace.Event{}, + }, + { + name: "Attempt.grpc.testing.TestService.FullDuplexCall", + spanKind: oteltrace.SpanKindInternal.String(), + attributes: []attribute.KeyValue{ + { + Key: "Client", + Value: attribute.IntValue(1), + }, + { + Key: "FailFast", + Value: attribute.IntValue(1), + }, + { + Key: "previous-rpc-attempts", + Value: attribute.IntValue(0), + }, + { + Key: "transparent-retry", + Value: attribute.IntValue(0), + }, + }, + events: []trace.Event{}, + }, + } + + // Check that same traceID is used in client and server for unary RPC call. + if got, want := spans[0].SpanContext.TraceID(), spans[2].SpanContext.TraceID(); got != want { + t.Fatal("TraceID mismatch in client span and server span.") + } + // Check that the attempt span id of client matches the span id of server + // SpanContext. + if got, want := spans[0].Parent.SpanID(), spans[1].SpanContext.SpanID(); got != want { + t.Fatal("SpanID mismatch in client span and server span.") + } + + // Check that same traceID is used in client and server for streaming RPC call. + if got, want := spans[3].SpanContext.TraceID(), spans[4].SpanContext.TraceID(); got != want { + t.Fatal("TraceID mismatch in client span and server span.") + } + // Check that the attempt span id of client matches the span id of server + // SpanContext. + if got, want := spans[3].Parent.SpanID(), spans[5].SpanContext.SpanID(); got != want { + t.Fatal("SpanID mismatch in client span and server span.") + } + + for index, span := range spans { + // Check that the attempt span has the correct status + if got, want := spans[index].Status.Code, otelcodes.Ok; got != want { + t.Errorf("Got status code %v, want %v", got, want) + } + // name + if got, want := span.Name, wantSI[index].name; got != want { + t.Errorf("Span name is %q, want %q", got, want) + } + // spanKind + if got, want := span.SpanKind.String(), wantSI[index].spanKind; got != want { + t.Errorf("Got span kind %q, want %q", got, want) + } + // attributes + if got, want := len(span.Attributes), len(wantSI[index].attributes); got != want { + t.Errorf("Got attributes list of size %q, want %q", got, want) + } + for idx, att := range span.Attributes { + if got, want := att.Key, wantSI[index].attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %v as %v, want %v", span.Name, got, want) + } + } + // events + if got, want := len(span.Events), len(wantSI[index].events); got != want { + t.Errorf("Event length is %q, want %q", got, want) + } + for eventIdx, event := range span.Events { + if got, want := event.Name, wantSI[index].events[eventIdx].Name; got != want { + t.Errorf("Got event name for span name %q as %q, want %q", span.Name, got, want) + } + for idx, att := range event.Attributes { + if got, want := att.Key, wantSI[index].events[eventIdx].Attributes[idx].Key; got != want { + t.Errorf("Got attribute key for span name %q with event name %v, as %v, want %v", span.Name, event.Name, got, want) + } + if got, want := att.Value, wantSI[index].events[eventIdx].Attributes[idx].Value; got != want { + t.Errorf("Got attribute value for span name %v with event name %v, as %v, want %v", span.Name, event.Name, got, want) + } + } + } + } } // TestSpan verifies that the gRPC Trace Binary propagator correctly @@ -692,9 +931,9 @@ func (s) TestMetricsAndTracesOptionEnabled(t *testing.T) { // - Verifies that the tracing information is recorded accurately in // the OpenTelemetry backend. func (s) TestSpan(t *testing.T) { - // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter + // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter. traceOptions, spanExporter := defaultTraceOptions(t) - // Start the server with OpenTelemetry options + // Start the server with trace options. ss := setupStubServer(t, nil, traceOptions) defer ss.Stop() @@ -702,8 +941,8 @@ func (s) TestSpan(t *testing.T) { defer cancel() // Make two RPC's, a unary RPC and a streaming RPC. These should cause - // certain metrics to be emitted, which should be able to be observed - // through the Metric Reader. + // certain metrics to be emitted, which should be observed through the + // Metric Reader. if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ Body: make([]byte, 10000), }}); err != nil { @@ -987,8 +1226,8 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { defer cancel() // Make two RPC's, a unary RPC and a streaming RPC. These should cause - // certain metrics to be emitted, which should be able to be observed - // through the Metric Reader. + // certain metrics to be emitted, which should be observed through the + // Metric Reader. if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ Body: make([]byte, 10000), }}); err != nil { diff --git a/stats/opentelemetry/experimental/api.go b/stats/opentelemetry/experimental/trace_options.go similarity index 100% rename from stats/opentelemetry/experimental/api.go rename to stats/opentelemetry/experimental/trace_options.go From d0e1a0ac6dcaebb9cfa807e7ca2dc14ce0fc1a59 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Thu, 19 Dec 2024 09:42:29 +0530 Subject: [PATCH 50/57] Add test with metrics and traces disabled. --- stats/opentelemetry/e2e_test.go | 73 +++++++++++++++++++++++++++------ 1 file changed, 60 insertions(+), 13 deletions(-) diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index 69e657071fd6..db394b6b2427 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -624,9 +624,9 @@ func pollForWantMetrics(ctx context.Context, t *testing.T, reader *metric.Manual // TestMetricsAndTracesOptionEnabled verifies the integration of metrics and traces // emitted by the OpenTelemetry instrumentation in a gRPC environment. It sets up a // stub server with both metrics and traces enabled, and tests the correct emission -// of metrics during a Unary RPC and a Streaming RPC. The test ensures that the -// emitted metrics reflect the operations performed, including the size of the -// compressed message, and verifies that tracing information is correctly recorded. +// of metrics and traces during a Unary RPC and a Streaming RPC. The test ensures +// that the emitted metrics reflect the operations performed, including the size of +// the compressed message, and verifies that tracing information is correctly recorded. func (s) TestMetricsAndTracesOptionEnabled(t *testing.T) { // Create default metrics options mo, reader := defaultMetricsOptions(t, nil) @@ -640,7 +640,8 @@ func (s) TestMetricsAndTracesOptionEnabled(t *testing.T) { defer cancel() // Make two RPC's, a unary RPC and a streaming RPC. These should cause - // certain metrics and traces to be emitted. + // certain metrics and traces to be emitted which should be observed + // through metrics reader and span exporter respectively. if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ Body: make([]byte, 10000), }}, grpc.UseCompressor(gzip.Name)); err != nil { // Deterministic compression. @@ -931,18 +932,19 @@ func (s) TestMetricsAndTracesOptionEnabled(t *testing.T) { // - Verifies that the tracing information is recorded accurately in // the OpenTelemetry backend. func (s) TestSpan(t *testing.T) { + mo, _ := defaultMetricsOptions(t, nil) // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter. - traceOptions, spanExporter := defaultTraceOptions(t) + to, spanExporter := defaultTraceOptions(t) // Start the server with trace options. - ss := setupStubServer(t, nil, traceOptions) + ss := setupStubServer(t, mo, to) defer ss.Stop() ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() // Make two RPC's, a unary RPC and a streaming RPC. These should cause - // certain metrics to be emitted, which should be observed through the - // Metric Reader. + // certain traces to be emitted, which should be observed through the + // span exporter. if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ Body: make([]byte, 10000), }}); err != nil { @@ -1214,20 +1216,21 @@ func (s) TestSpan(t *testing.T) { // - Verifies that the trace ID and span ID are correctly assigned and accessible // in the OpenTelemetry backend. func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { + mo, _ := defaultMetricsOptions(t, nil) // Using defaultTraceOptions to set up OpenTelemetry with an in-memory exporter - traceOptions, spanExporter := defaultTraceOptions(t) + to, spanExporter := defaultTraceOptions(t) // Set the W3CContextPropagator as part of TracingOptions. - traceOptions.TextMapPropagator = propagation.NewCompositeTextMapPropagator(propagation.TraceContext{}) + to.TextMapPropagator = propagation.NewCompositeTextMapPropagator(propagation.TraceContext{}) // Start the server with OpenTelemetry options - ss := setupStubServer(t, nil, traceOptions) + ss := setupStubServer(t, mo, to) defer ss.Stop() ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() // Make two RPC's, a unary RPC and a streaming RPC. These should cause - // certain metrics to be emitted, which should be observed through the - // Metric Reader. + // certain traces to be emitted, which should be observed through the + // span exporter. if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ Body: make([]byte, 10000), }}); err != nil { @@ -1485,3 +1488,47 @@ func (s) TestSpan_WithW3CContextPropagator(t *testing.T) { } } } + +// TestMetricsAndTracesDisabled verifies that RPCs call succeed as expected +// when metrics and traces are disabled in the OpenTelemetry instrumentation. +func (s) TestMetricsAndTracesDisabled(t *testing.T) { + ss := &stubserver.StubServer{ + UnaryCallF: func(_ context.Context, in *testpb.SimpleRequest) (*testpb.SimpleResponse, error) { + return &testpb.SimpleResponse{Payload: &testpb.Payload{ + Body: make([]byte, len(in.GetPayload().GetBody())), + }}, nil + }, + FullDuplexCallF: func(stream testgrpc.TestService_FullDuplexCallServer) error { + for { + _, err := stream.Recv() + if err == io.EOF { + return nil + } + } + }, + } + + if err := ss.Start(nil); err != nil { + t.Fatalf("Error starting endpoint server: %v", err) + } + defer ss.Stop() + + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + defer cancel() + + // Make two RPCs, a unary RPC and a streaming RPC. + if _, err := ss.Client.UnaryCall(ctx, &testpb.SimpleRequest{Payload: &testpb.Payload{ + Body: make([]byte, 10000), + }}); err != nil { + t.Fatalf("Unexpected error from UnaryCall: %v", err) + } + stream, err := ss.Client.FullDuplexCall(ctx) + if err != nil { + t.Fatalf("ss.Client.FullDuplexCall failed: %v", err) + } + + stream.CloseSend() + if _, err = stream.Recv(); err != io.EOF { + t.Fatalf("stream.Recv received an unexpected error: %v, expected an EOF error", err) + } +} From b6503f781397cc3e04cdbd2fbe7ac033595421f2 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Thu, 19 Dec 2024 10:52:11 +0530 Subject: [PATCH 51/57] make vet happy --- stats/opentelemetry/experimental/grpc_trace_bin_propagator.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go b/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go index b6cd460f231a..4385184239d2 100644 --- a/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go +++ b/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go @@ -18,7 +18,7 @@ // # Experimental // -// Notice: This package is EXPERIMENTAL and may be changed or removed in a +// Notice: package experimental is EXPERIMENTAL and may be changed or removed in a // later release. package experimental From b2831f1b962dc944799bc6a367ad4f3e35ce1d2b Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Fri, 20 Dec 2024 23:16:36 +0530 Subject: [PATCH 52/57] pull out logic to find name resolution delay --- clientconn.go | 11 +++----- stats/handlers.go | 5 ---- stats/opentelemetry/client_metrics.go | 6 +---- stream.go | 36 ++++++++++++--------------- 4 files changed, 20 insertions(+), 38 deletions(-) diff --git a/clientconn.go b/clientconn.go index 5f326647e11f..4f57b55434f9 100644 --- a/clientconn.go +++ b/clientconn.go @@ -130,10 +130,9 @@ func (dcs *defaultConfigSelector) SelectConfig(rpcInfo iresolver.RPCInfo) (*ires // function. func NewClient(target string, opts ...DialOption) (conn *ClientConn, err error) { cc := &ClientConn{ - target: target, - conns: make(map[*addrConn]struct{}), - dopts: defaultDialOptions(), - nameResolutionDelayed: false, + target: target, + conns: make(map[*addrConn]struct{}), + dopts: defaultDialOptions(), } cc.retryThrottler.Store((*retryThrottler)(nil)) @@ -605,10 +604,6 @@ type ClientConn struct { idlenessMgr *idle.Manager metricsRecorderList *stats.MetricsRecorderList - // To track if there was a delay in name resolution, helping to track - // latency issues in gRPC connection setup. - nameResolutionDelayed bool - // The following provide their own synchronization, and therefore don't // require cc.mu to be held to access them. csMgr *connectivityStateManager diff --git a/stats/handlers.go b/stats/handlers.go index c1dd190c1609..dc03731e45ef 100644 --- a/stats/handlers.go +++ b/stats/handlers.go @@ -38,11 +38,6 @@ type RPCTagInfo struct { // FailFast indicates if this RPC is failfast. // This field is only valid on client side, it's always false on server side. FailFast bool - // NameResolutionDelay indicates whether there was a delay in name - // resolution. - // - // This field is only valid on client side, it's always false on server side. - NameResolutionDelay bool } // Handler defines the interface for the related stats handling (e.g., RPCs, connections). diff --git a/stats/opentelemetry/client_metrics.go b/stats/opentelemetry/client_metrics.go index cb13182d7b9a..fc17c9be55be 100644 --- a/stats/opentelemetry/client_metrics.go +++ b/stats/opentelemetry/client_metrics.go @@ -213,11 +213,7 @@ func (h *clientStatsHandler) TagRPC(ctx context.Context, info *stats.RPCTagInfo) ai := &attemptInfo{} startTime := time.Now() if !isTracingDisabled(h.options.TraceOptions) { - callSpan := trace.SpanFromContext(ctx) - if info.NameResolutionDelay { - callSpan.AddEvent("Delayed name resolution complete") - } - ctx, ai = h.traceTagRPC(trace.ContextWithSpan(ctx, callSpan), info) + ctx, ai = h.traceTagRPC(ctx, info) } ai.startTime = startTime ai.xdsLabels = labels.TelemetryLabels diff --git a/stream.go b/stream.go index 3406a7d9744f..521472925a78 100644 --- a/stream.go +++ b/stream.go @@ -213,13 +213,12 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth // Provide an opportunity for the first RPC to see the first service config // provided by the resolver. if err := cc.waitForResolvedAddrs(ctx); err != nil { - cc.nameResolutionDelayed = true return nil, err } var mc serviceconfig.MethodConfig var onCommit func() newStream := func(ctx context.Context, done func()) (iresolver.ClientStream, error) { - return newClientStreamWithParams(ctx, desc, cc, method, mc, onCommit, done, cc.nameResolutionDelayed, opts...) + return newClientStreamWithParams(ctx, desc, cc, method, mc, onCommit, done, opts...) } rpcInfo := iresolver.RPCInfo{Context: ctx, Method: method} @@ -257,7 +256,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth return newStream(ctx, func() {}) } -func newClientStreamWithParams(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, mc serviceconfig.MethodConfig, onCommit, doneFunc func(), nameResolutionDelayed bool, opts ...CallOption) (_ iresolver.ClientStream, err error) { +func newClientStreamWithParams(ctx context.Context, desc *StreamDesc, cc *ClientConn, method string, mc serviceconfig.MethodConfig, onCommit, doneFunc func(), opts ...CallOption) (_ iresolver.ClientStream, err error) { c := defaultCallInfo() if mc.WaitForReady != nil { c.failFast = !*mc.WaitForReady @@ -321,20 +320,19 @@ func newClientStreamWithParams(ctx context.Context, desc *StreamDesc, cc *Client } cs := &clientStream{ - callHdr: callHdr, - ctx: ctx, - methodConfig: &mc, - opts: opts, - callInfo: c, - cc: cc, - desc: desc, - codec: c.codec, - cp: cp, - comp: comp, - cancel: cancel, - firstAttempt: true, - onCommit: onCommit, - nameResolutionDelayed: nameResolutionDelayed, + callHdr: callHdr, + ctx: ctx, + methodConfig: &mc, + opts: opts, + callInfo: c, + cc: cc, + desc: desc, + codec: c.codec, + cp: cp, + comp: comp, + cancel: cancel, + firstAttempt: true, + onCommit: onCommit, } if !cc.dopts.disableRetry { cs.retryThrottler = cc.retryThrottler.Load().(*retryThrottler) @@ -418,7 +416,7 @@ func (cs *clientStream) newAttemptLocked(isTransparent bool) (*csAttempt, error) var beginTime time.Time shs := cs.cc.dopts.copts.StatsHandlers for _, sh := range shs { - ctx = sh.TagRPC(ctx, &stats.RPCTagInfo{FullMethodName: method, FailFast: cs.callInfo.failFast, NameResolutionDelay: cs.nameResolutionDelayed}) + ctx = sh.TagRPC(ctx, &stats.RPCTagInfo{FullMethodName: method, FailFast: cs.callInfo.failFast}) beginTime = time.Now() begin := &stats.Begin{ Client: true, @@ -556,8 +554,6 @@ type clientStream struct { // synchronized. serverHeaderBinlogged bool - nameResolutionDelayed bool - mu sync.Mutex firstAttempt bool // if true, transparent retry is valid numRetries int // exclusive of transparent retry attempt(s) From 1cb43967a135ad6a0add5cb37a8b665680ed550a Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Fri, 20 Dec 2024 23:39:47 +0530 Subject: [PATCH 53/57] remove experimental notice from grpc_trace_bin_propagator --- .../opentelemetry/experimental/grpc_trace_bin_propagator.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go b/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go index 4385184239d2..2b6742c2e61d 100644 --- a/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go +++ b/stats/opentelemetry/experimental/grpc_trace_bin_propagator.go @@ -16,11 +16,6 @@ * */ -// # Experimental -// -// Notice: package experimental is EXPERIMENTAL and may be changed or removed in a -// later release. - package experimental import ( From b8fe8db99bc61497f4bc944866b3cb1396ba5a99 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Mon, 23 Dec 2024 22:50:02 +0530 Subject: [PATCH 54/57] refactor and addressed comments from doug --- .../opentelemetry}/trace_options.go | 0 stats/opentelemetry/client_metrics.go | 35 +++++------- stats/opentelemetry/client_tracing.go | 38 +++++++++++++ stats/opentelemetry/e2e_test.go | 7 +-- stats/opentelemetry/metricsregistry_test.go | 4 +- stats/opentelemetry/opentelemetry.go | 30 +++++------ stats/opentelemetry/server_metrics.go | 26 +++------ stats/opentelemetry/server_tracing.go | 44 +++++++++++++++ stats/opentelemetry/trace.go | 53 ------------------- stream.go | 1 + 10 files changed, 125 insertions(+), 113 deletions(-) rename {stats/opentelemetry/experimental => experimental/opentelemetry}/trace_options.go (100%) create mode 100644 stats/opentelemetry/client_tracing.go create mode 100644 stats/opentelemetry/server_tracing.go diff --git a/stats/opentelemetry/experimental/trace_options.go b/experimental/opentelemetry/trace_options.go similarity index 100% rename from stats/opentelemetry/experimental/trace_options.go rename to experimental/opentelemetry/trace_options.go diff --git a/stats/opentelemetry/client_metrics.go b/stats/opentelemetry/client_metrics.go index fc17c9be55be..6a7ca9749cd7 100644 --- a/stats/opentelemetry/client_metrics.go +++ b/stats/opentelemetry/client_metrics.go @@ -74,15 +74,6 @@ func (h *clientStatsHandler) initializeMetrics() { rm.registerMetrics(metrics, meter) } -func (h *clientStatsHandler) initializeTracing() { - if isTracingDisabled(h.options.TraceOptions) { - return - } - - otel.SetTextMapPropagator(h.options.TraceOptions.TextMapPropagator) - otel.SetTracerProvider(h.options.TraceOptions.TracerProvider) -} - func (h *clientStatsHandler) unaryInterceptor(ctx context.Context, method string, req, reply any, cc *grpc.ClientConn, invoker grpc.UnaryInvoker, opts ...grpc.CallOption) error { ci := &callInfo{ target: cc.CanonicalTarget(), @@ -101,7 +92,7 @@ func (h *clientStatsHandler) unaryInterceptor(ctx context.Context, method string startTime := time.Now() var span *trace.Span - if !isTracingDisabled(h.options.TraceOptions) { + if h.options.isTracingEnabled() { ctx, span = h.createCallTraceSpan(ctx, method) } err := invoker(ctx, method, req, reply, cc, opts...) @@ -139,7 +130,7 @@ func (h *clientStatsHandler) streamInterceptor(ctx context.Context, desc *grpc.S startTime := time.Now() var span *trace.Span - if !isTracingDisabled(h.options.TraceOptions) { + if h.options.isTracingEnabled() { ctx, span = h.createCallTraceSpan(ctx, method) } callback := func(err error) { @@ -151,7 +142,7 @@ func (h *clientStatsHandler) streamInterceptor(ctx context.Context, desc *grpc.S // perCallTracesAndMetrics records per call trace spans and metrics. func (h *clientStatsHandler) perCallTracesAndMetrics(ctx context.Context, err error, startTime time.Time, ci *callInfo, ts *trace.Span) { - if !isTracingDisabled(h.options.TraceOptions) && ts != nil { + if h.options.isTracingEnabled() && ts != nil { s := status.Convert(err) if s.Code() == grpccodes.OK { (*ts).SetStatus(otelcodes.Ok, s.Message()) @@ -160,7 +151,7 @@ func (h *clientStatsHandler) perCallTracesAndMetrics(ctx context.Context, err er } (*ts).End() } - if !isMetricsDisabled(h.options.MetricsOptions) { + if h.options.isMetricsEnabled() { callLatency := float64(time.Since(startTime)) / float64(time.Second) attrs := otelmetric.WithAttributeSet(otelattribute.NewSet( otelattribute.String("grpc.method", ci.method), @@ -210,14 +201,14 @@ func (h *clientStatsHandler) TagRPC(ctx context.Context, info *stats.RPCTagInfo) } ctx = istats.SetLabels(ctx, labels) } - ai := &attemptInfo{} - startTime := time.Now() - if !isTracingDisabled(h.options.TraceOptions) { - ctx, ai = h.traceTagRPC(ctx, info) + ai := &attemptInfo{ + startTime: time.Now(), + xdsLabels: labels.TelemetryLabels, + method: info.FullMethodName, + } + if h.options.isTracingEnabled() { + ctx, ai = h.traceTagRPC(ctx, info, ai) } - ai.startTime = startTime - ai.xdsLabels = labels.TelemetryLabels - ai.method = info.FullMethodName return setRPCInfo(ctx, &rpcInfo{ ai: ai, }) @@ -229,10 +220,10 @@ func (h *clientStatsHandler) HandleRPC(ctx context.Context, rs stats.RPCStats) { logger.Error("ctx passed into client side stats handler metrics event handling has no client attempt data present") return } - if !isMetricsDisabled(h.options.MetricsOptions) { + if h.options.isMetricsEnabled() { h.processRPCEvent(ctx, rs, ri.ai) } - if !isTracingDisabled(h.options.TraceOptions) { + if h.options.isTracingEnabled() { h.populateSpan(ctx, rs, ri.ai) } } diff --git a/stats/opentelemetry/client_tracing.go b/stats/opentelemetry/client_tracing.go new file mode 100644 index 000000000000..c4c54685e0a9 --- /dev/null +++ b/stats/opentelemetry/client_tracing.go @@ -0,0 +1,38 @@ +package opentelemetry + +import ( + "context" + "strings" + + "go.opentelemetry.io/otel" + "google.golang.org/grpc/stats" + otelinternaltracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" +) + +func (h *clientStatsHandler) initializeTracing() { + if !h.options.isTracingEnabled() { + return + } + + otel.SetTextMapPropagator(h.options.TraceOptions.TextMapPropagator) + otel.SetTracerProvider(h.options.TraceOptions.TracerProvider) +} + +// traceTagRPC populates provided context with a new span using the +// TextMapPropagator supplied in trace options and internal itracing.carrier. +// It creates a new outgoing carrier which serializes information about this +// span into gRPC Metadata, if TextMapPropagator is provided in the trace +// options. if TextMapPropagator is not provided, it returns the context as is. +func (h *clientStatsHandler) traceTagRPC(ctx context.Context, rti *stats.RPCTagInfo, ai *attemptInfo) (context.Context, *attemptInfo) { + if h.options.TraceOptions.TextMapPropagator == nil { + return ctx, nil + } + + mn := "Attempt." + strings.Replace(removeLeadingSlash(rti.FullMethodName), "/", ".", -1) + tracer := otel.Tracer("grpc-open-telemetry") + ctx, span := tracer.Start(ctx, mn) + carrier := otelinternaltracing.NewOutgoingCarrier(ctx) + otel.GetTextMapPropagator().Inject(ctx, carrier) + ai.traceSpan = span + return carrier.Context(), ai +} diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index db394b6b2427..d5d21ef8cd27 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -47,6 +47,7 @@ import ( "google.golang.org/grpc" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/encoding/gzip" + experimental "google.golang.org/grpc/experimental/opentelemetry" "google.golang.org/grpc/internal/grpcsync" "google.golang.org/grpc/internal/grpctest" "google.golang.org/grpc/internal/stubserver" @@ -57,7 +58,7 @@ import ( testpb "google.golang.org/grpc/interop/grpc_testing" "google.golang.org/grpc/orca" "google.golang.org/grpc/stats/opentelemetry" - "google.golang.org/grpc/stats/opentelemetry/experimental" + expstats "google.golang.org/grpc/stats/opentelemetry/experimental" "google.golang.org/grpc/stats/opentelemetry/internal/testutils" ) @@ -98,7 +99,7 @@ func defaultTraceOptions(_ *testing.T) (*experimental.TraceOptions, *tracetest.I spanExporter := tracetest.NewInMemoryExporter() spanProcessor := trace.NewSimpleSpanProcessor(spanExporter) tracerProvider := trace.NewTracerProvider(trace.WithSpanProcessor(spanProcessor)) - textMapPropagator := propagation.NewCompositeTextMapPropagator(experimental.GRPCTraceBinPropagator{}) + textMapPropagator := propagation.NewCompositeTextMapPropagator(expstats.GRPCTraceBinPropagator{}) traceOptions := &experimental.TraceOptions{ TracerProvider: tracerProvider, TextMapPropagator: textMapPropagator, @@ -636,7 +637,7 @@ func (s) TestMetricsAndTracesOptionEnabled(t *testing.T) { ss := setupStubServer(t, mo, to) defer ss.Stop() - ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout*2) defer cancel() // Make two RPC's, a unary RPC and a streaming RPC. These should cause diff --git a/stats/opentelemetry/metricsregistry_test.go b/stats/opentelemetry/metricsregistry_test.go index 7cd979d1a73d..67a2d7faeadc 100644 --- a/stats/opentelemetry/metricsregistry_test.go +++ b/stats/opentelemetry/metricsregistry_test.go @@ -31,7 +31,7 @@ import ( "go.opentelemetry.io/otel/sdk/metric/metricdata/metricdatatest" ) -var DefaultTestTimeout = 5 * time.Second +var defaultTestTimeout = 5 * time.Second type s struct { grpctest.Tester @@ -130,7 +130,7 @@ func (s) TestMetricsRegistryMetrics(t *testing.T) { Default: true, }) - ctx, cancel := context.WithTimeout(context.Background(), DefaultTestTimeout) + ctx, cancel := context.WithTimeout(context.Background(), defaultTestTimeout) defer cancel() // Only float optional labels are configured, so only float optional labels should show up. diff --git a/stats/opentelemetry/opentelemetry.go b/stats/opentelemetry/opentelemetry.go index c200ff86dce7..4a7a4b24b14b 100644 --- a/stats/opentelemetry/opentelemetry.go +++ b/stats/opentelemetry/opentelemetry.go @@ -33,26 +33,26 @@ import ( "go.opentelemetry.io/otel/trace" "google.golang.org/grpc" "google.golang.org/grpc/codes" + experimental "google.golang.org/grpc/experimental/opentelemetry" estats "google.golang.org/grpc/experimental/stats" "google.golang.org/grpc/grpclog" "google.golang.org/grpc/internal" "google.golang.org/grpc/stats" - "google.golang.org/grpc/stats/opentelemetry/experimental" otelinternal "google.golang.org/grpc/stats/opentelemetry/internal" ) -var logger = grpclog.Component("otel-plugin") - -var canonicalString = internal.CanonicalString.(func(codes.Code) string) - -var joinDialOptions = internal.JoinDialOptions.(func(...grpc.DialOption) grpc.DialOption) - func init() { otelinternal.SetPluginOption = func(o *Options, po otelinternal.PluginOption) { o.MetricsOptions.pluginOption = po } } +var ( + logger = grpclog.Component("otel-plugin") + canonicalString = internal.CanonicalString.(func(codes.Code) string) + joinDialOptions = internal.JoinDialOptions.(func(...grpc.DialOption) grpc.DialOption) +) + // Options are the options for OpenTelemetry instrumentation. type Options struct { // MetricsOptions are the metrics options for OpenTelemetry instrumentation. @@ -61,6 +61,14 @@ type Options struct { TraceOptions experimental.TraceOptions } +func (o *Options) isMetricsEnabled() bool { + return o.MetricsOptions.MeterProvider != nil +} + +func (o *Options) isTracingEnabled() bool { + return !(o.TraceOptions.TracerProvider == nil || o.TraceOptions.TextMapPropagator == nil) +} + // MetricsOptions are the metrics options for OpenTelemetry instrumentation. type MetricsOptions struct { // MeterProvider is the MeterProvider instance that will be used to create @@ -176,14 +184,6 @@ func removeLeadingSlash(mn string) string { return strings.TrimLeft(mn, "/") } -func isMetricsDisabled(mo MetricsOptions) bool { - return mo.MeterProvider == nil -} - -func isTracingDisabled(to experimental.TraceOptions) bool { - return to.TracerProvider == nil || to.TextMapPropagator == nil -} - // attemptInfo is RPC information scoped to the RPC attempt life span client // side, and the RPC life span server side. type attemptInfo struct { diff --git a/stats/opentelemetry/server_metrics.go b/stats/opentelemetry/server_metrics.go index f61c13571415..a543030b70ac 100644 --- a/stats/opentelemetry/server_metrics.go +++ b/stats/opentelemetry/server_metrics.go @@ -28,7 +28,6 @@ import ( "google.golang.org/grpc/stats" "google.golang.org/grpc/status" - "go.opentelemetry.io/otel" otelattribute "go.opentelemetry.io/otel/attribute" otelmetric "go.opentelemetry.io/otel/metric" ) @@ -68,15 +67,6 @@ func (h *serverStatsHandler) initializeMetrics() { rm.registerMetrics(metrics, meter) } -func (h *serverStatsHandler) initializeTracing() { - if !isTracingDisabled(h.options.TraceOptions) { - return - } - - otel.SetTextMapPropagator(h.options.TraceOptions.TextMapPropagator) - otel.SetTracerProvider(h.options.TraceOptions.TracerProvider) -} - // attachLabelsTransportStream intercepts SetHeader and SendHeader calls of the // underlying ServerTransportStream to attach metadataExchangeLabels. type attachLabelsTransportStream struct { @@ -208,13 +198,13 @@ func (h *serverStatsHandler) TagRPC(ctx context.Context, info *stats.RPCTagInfo) } } - ai := &attemptInfo{} - startTime := time.Now() - if !isTracingDisabled(h.options.TraceOptions) { - ctx, ai = h.traceTagRPC(ctx, info) + ai := &attemptInfo{ + startTime: time.Now(), + method: removeLeadingSlash(method), + } + if h.options.isTracingEnabled() { + ctx, ai = h.traceTagRPC(ctx, info, ai) } - ai.startTime = startTime - ai.method = removeLeadingSlash(method) return setRPCInfo(ctx, &rpcInfo{ ai: ai, }) @@ -227,10 +217,10 @@ func (h *serverStatsHandler) HandleRPC(ctx context.Context, rs stats.RPCStats) { logger.Error("ctx passed into server side stats handler metrics event handling has no server call data present") return } - if !isTracingDisabled(h.options.TraceOptions) { + if h.options.isTracingEnabled() { h.populateSpan(ctx, rs, ri.ai) } - if !isMetricsDisabled(h.options.MetricsOptions) { + if h.options.isMetricsEnabled() { h.processRPCData(ctx, rs, ri.ai) } } diff --git a/stats/opentelemetry/server_tracing.go b/stats/opentelemetry/server_tracing.go new file mode 100644 index 000000000000..e979c3176959 --- /dev/null +++ b/stats/opentelemetry/server_tracing.go @@ -0,0 +1,44 @@ +package opentelemetry + +import ( + "context" + "strings" + + "go.opentelemetry.io/otel" + "go.opentelemetry.io/otel/trace" + "google.golang.org/grpc/stats" + otelinternaltracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" +) + +func (h *serverStatsHandler) initializeTracing() { + if !h.options.isTracingEnabled() { + return + } + + otel.SetTextMapPropagator(h.options.TraceOptions.TextMapPropagator) + otel.SetTracerProvider(h.options.TraceOptions.TracerProvider) +} + +// traceTagRPC populates context with new span data using the TextMapPropagator +// supplied in trace options and internal itracing.Carrier. It creates a new +// incoming carrier which extracts an existing span context (if present) by +// deserializing from provided context. If valid span context is extracted, it +// is set as parent of the new span otherwise new span remains the root span. +// If TextMapPropagator is not provided in the trace options, it returns context +// as is. +func (h *serverStatsHandler) traceTagRPC(ctx context.Context, rti *stats.RPCTagInfo, ai *attemptInfo) (context.Context, *attemptInfo) { + if h.options.TraceOptions.TextMapPropagator == nil { + return ctx, nil + } + + mn := strings.Replace(removeLeadingSlash(rti.FullMethodName), "/", ".", -1) + var span trace.Span + tracer := otel.Tracer("grpc-open-telemetry") + ctx = otel.GetTextMapPropagator().Extract(ctx, otelinternaltracing.NewIncomingCarrier(ctx)) + // If the context.Context provided in `ctx` to tracer.Start(), contains a + // span then the newly-created Span will be a child of that span, + // otherwise it will be a root span. + ctx, span = tracer.Start(ctx, mn, trace.WithSpanKind(trace.SpanKindServer)) + ai.traceSpan = span + return ctx, ai +} diff --git a/stats/opentelemetry/trace.go b/stats/opentelemetry/trace.go index 0251e5da1408..d46cf1b1deae 100644 --- a/stats/opentelemetry/trace.go +++ b/stats/opentelemetry/trace.go @@ -18,68 +18,15 @@ package opentelemetry import ( "context" - "strings" "sync/atomic" - "go.opentelemetry.io/otel" "go.opentelemetry.io/otel/attribute" otelcodes "go.opentelemetry.io/otel/codes" "go.opentelemetry.io/otel/trace" "google.golang.org/grpc/stats" - otelinternaltracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" "google.golang.org/grpc/status" ) -// traceTagRPC populates provided context with a new span using the -// TextMapPropagator supplied in trace options and internal itracing.carrier. -// It creates a new outgoing carrier which serializes information about this -// span into gRPC Metadata, if TextMapPropagator is provided in the trace -// options. if TextMapPropagator is not provided, it returns the context as is. -func (h *clientStatsHandler) traceTagRPC(ctx context.Context, rti *stats.RPCTagInfo) (context.Context, *attemptInfo) { - if h.options.TraceOptions.TextMapPropagator == nil { - return ctx, nil - } - - mn := "Attempt." + strings.Replace(removeLeadingSlash(rti.FullMethodName), "/", ".", -1) - tracer := otel.Tracer("grpc-open-telemetry") - ctx, span := tracer.Start(ctx, mn) - carrier := otelinternaltracing.NewOutgoingCarrier(ctx) - otel.GetTextMapPropagator().Inject(ctx, carrier) - - return carrier.Context(), &attemptInfo{ - traceSpan: span, - countSentMsg: 0, // msg events scoped to scope of context, per attempt client side - countRecvMsg: 0, - } -} - -// traceTagRPC populates context with new span data using the TextMapPropagator -// supplied in trace options and internal itracing.Carrier. It creates a new -// incoming carrier which extracts an existing span context (if present) by -// deserializing from provided context. If valid span context is extracted, it -// is set as parent of the new span otherwise new span remains the root span. -// If TextMapPropagator is not provided in the trace options, it returns context -// as is. -func (h *serverStatsHandler) traceTagRPC(ctx context.Context, rti *stats.RPCTagInfo) (context.Context, *attemptInfo) { - if h.options.TraceOptions.TextMapPropagator == nil { - return ctx, nil - } - - mn := strings.Replace(removeLeadingSlash(rti.FullMethodName), "/", ".", -1) - var span trace.Span - tracer := otel.Tracer("grpc-open-telemetry") - ctx = otel.GetTextMapPropagator().Extract(ctx, otelinternaltracing.NewIncomingCarrier(ctx)) - // If the context.Context provided in `ctx` to tracer.Start(), contains a - // span then the newly-created Span will be a child of that span, - // otherwise it will be a root span. - ctx, span = tracer.Start(ctx, mn, trace.WithSpanKind(trace.SpanKindServer)) - return ctx, &attemptInfo{ - traceSpan: span, - countSentMsg: 0, - countRecvMsg: 0, - } -} - // statsHandler holds common functionality for both client and server stats // handler. type statsHandler struct{} diff --git a/stream.go b/stream.go index 521472925a78..17e2267b3320 100644 --- a/stream.go +++ b/stream.go @@ -215,6 +215,7 @@ func newClientStream(ctx context.Context, desc *StreamDesc, cc *ClientConn, meth if err := cc.waitForResolvedAddrs(ctx); err != nil { return nil, err } + var mc serviceconfig.MethodConfig var onCommit func() newStream := func(ctx context.Context, done func()) (iresolver.ClientStream, error) { From d4ae3abfe8bc808fa90fb712114d9e15be84d50e Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Mon, 23 Dec 2024 22:55:33 +0530 Subject: [PATCH 55/57] Add copyright notice to client_tracing.go --- stats/opentelemetry/client_tracing.go | 16 ++++++++++++++++ stats/opentelemetry/server_tracing.go | 16 ++++++++++++++++ 2 files changed, 32 insertions(+) diff --git a/stats/opentelemetry/client_tracing.go b/stats/opentelemetry/client_tracing.go index c4c54685e0a9..527f9a07b56c 100644 --- a/stats/opentelemetry/client_tracing.go +++ b/stats/opentelemetry/client_tracing.go @@ -1,3 +1,19 @@ +/* + * Copyright 2024 gRPC 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 opentelemetry import ( diff --git a/stats/opentelemetry/server_tracing.go b/stats/opentelemetry/server_tracing.go index e979c3176959..25ac03a2d180 100644 --- a/stats/opentelemetry/server_tracing.go +++ b/stats/opentelemetry/server_tracing.go @@ -1,3 +1,19 @@ +/* + * Copyright 2024 gRPC 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 opentelemetry import ( From c705b978ba2dbbe1d71437d86258b712d3520563 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Tue, 24 Dec 2024 09:41:27 +0530 Subject: [PATCH 56/57] let client set the propagator and trace provider --- stats/opentelemetry/client_tracing.go | 9 --------- stats/opentelemetry/e2e_test.go | 3 +++ stats/opentelemetry/opentelemetry.go | 4 ---- stats/opentelemetry/server_tracing.go | 9 --------- 4 files changed, 3 insertions(+), 22 deletions(-) diff --git a/stats/opentelemetry/client_tracing.go b/stats/opentelemetry/client_tracing.go index 527f9a07b56c..ee0a285e201b 100644 --- a/stats/opentelemetry/client_tracing.go +++ b/stats/opentelemetry/client_tracing.go @@ -25,15 +25,6 @@ import ( otelinternaltracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" ) -func (h *clientStatsHandler) initializeTracing() { - if !h.options.isTracingEnabled() { - return - } - - otel.SetTextMapPropagator(h.options.TraceOptions.TextMapPropagator) - otel.SetTracerProvider(h.options.TraceOptions.TracerProvider) -} - // traceTagRPC populates provided context with a new span using the // TextMapPropagator supplied in trace options and internal itracing.carrier. // It creates a new outgoing carrier which serializes information about this diff --git a/stats/opentelemetry/e2e_test.go b/stats/opentelemetry/e2e_test.go index d5d21ef8cd27..331b3474d7e5 100644 --- a/stats/opentelemetry/e2e_test.go +++ b/stats/opentelemetry/e2e_test.go @@ -23,6 +23,7 @@ import ( "testing" "time" + "go.opentelemetry.io/otel" otelcodes "go.opentelemetry.io/otel/codes" oteltrace "go.opentelemetry.io/otel/trace" @@ -100,6 +101,8 @@ func defaultTraceOptions(_ *testing.T) (*experimental.TraceOptions, *tracetest.I spanProcessor := trace.NewSimpleSpanProcessor(spanExporter) tracerProvider := trace.NewTracerProvider(trace.WithSpanProcessor(spanProcessor)) textMapPropagator := propagation.NewCompositeTextMapPropagator(expstats.GRPCTraceBinPropagator{}) + otel.SetTextMapPropagator(textMapPropagator) + otel.SetTracerProvider(tracerProvider) traceOptions := &experimental.TraceOptions{ TracerProvider: tracerProvider, TextMapPropagator: textMapPropagator, diff --git a/stats/opentelemetry/opentelemetry.go b/stats/opentelemetry/opentelemetry.go index 4a7a4b24b14b..4a001aad800c 100644 --- a/stats/opentelemetry/opentelemetry.go +++ b/stats/opentelemetry/opentelemetry.go @@ -115,8 +115,6 @@ type MetricsOptions struct { // will create a default view for that metric. func DialOption(o Options) grpc.DialOption { csh := &clientStatsHandler{options: o} - csh.initializeMetrics() - csh.initializeTracing() return joinDialOptions(grpc.WithChainUnaryInterceptor(csh.unaryInterceptor), grpc.WithChainStreamInterceptor(csh.streamInterceptor), grpc.WithStatsHandler(csh)) } @@ -136,8 +134,6 @@ var joinServerOptions = internal.JoinServerOptions.(func(...grpc.ServerOption) g // will create a default view for that metric. func ServerOption(o Options) grpc.ServerOption { ssh := &serverStatsHandler{options: o} - ssh.initializeMetrics() - ssh.initializeTracing() return joinServerOptions(grpc.ChainUnaryInterceptor(ssh.unaryInterceptor), grpc.ChainStreamInterceptor(ssh.streamInterceptor), grpc.StatsHandler(ssh)) } diff --git a/stats/opentelemetry/server_tracing.go b/stats/opentelemetry/server_tracing.go index 25ac03a2d180..2b796bfd6a53 100644 --- a/stats/opentelemetry/server_tracing.go +++ b/stats/opentelemetry/server_tracing.go @@ -26,15 +26,6 @@ import ( otelinternaltracing "google.golang.org/grpc/stats/opentelemetry/internal/tracing" ) -func (h *serverStatsHandler) initializeTracing() { - if !h.options.isTracingEnabled() { - return - } - - otel.SetTextMapPropagator(h.options.TraceOptions.TextMapPropagator) - otel.SetTracerProvider(h.options.TraceOptions.TracerProvider) -} - // traceTagRPC populates context with new span data using the TextMapPropagator // supplied in trace options and internal itracing.Carrier. It creates a new // incoming carrier which extracts an existing span context (if present) by From 5571e3bac8683c6ebaf5b1a257f8dd0fce78bde8 Mon Sep 17 00:00:00 2001 From: Abhishek Ranjan Date: Tue, 24 Dec 2024 09:47:47 +0530 Subject: [PATCH 57/57] fix breaking tests --- stats/opentelemetry/opentelemetry.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/stats/opentelemetry/opentelemetry.go b/stats/opentelemetry/opentelemetry.go index 4a001aad800c..afdb9808f0c4 100644 --- a/stats/opentelemetry/opentelemetry.go +++ b/stats/opentelemetry/opentelemetry.go @@ -115,6 +115,7 @@ type MetricsOptions struct { // will create a default view for that metric. func DialOption(o Options) grpc.DialOption { csh := &clientStatsHandler{options: o} + csh.initializeMetrics() return joinDialOptions(grpc.WithChainUnaryInterceptor(csh.unaryInterceptor), grpc.WithChainStreamInterceptor(csh.streamInterceptor), grpc.WithStatsHandler(csh)) } @@ -134,6 +135,7 @@ var joinServerOptions = internal.JoinServerOptions.(func(...grpc.ServerOption) g // will create a default view for that metric. func ServerOption(o Options) grpc.ServerOption { ssh := &serverStatsHandler{options: o} + ssh.initializeMetrics() return joinServerOptions(grpc.ChainUnaryInterceptor(ssh.unaryInterceptor), grpc.ChainStreamInterceptor(ssh.streamInterceptor), grpc.StatsHandler(ssh)) }