diff --git a/internal/quic/cmd/interop/main.go b/internal/quic/cmd/interop/main.go index cc5292e9e..2ca5d652a 100644 --- a/internal/quic/cmd/interop/main.go +++ b/internal/quic/cmd/interop/main.go @@ -18,6 +18,7 @@ import ( "fmt" "io" "log" + "log/slog" "net" "net/url" "os" @@ -25,14 +26,16 @@ import ( "sync" "golang.org/x/net/internal/quic" + "golang.org/x/net/internal/quic/qlog" ) var ( - listen = flag.String("listen", "", "listen address") - cert = flag.String("cert", "", "certificate") - pkey = flag.String("key", "", "private key") - root = flag.String("root", "", "serve files from this root") - output = flag.String("output", "", "directory to write files to") + listen = flag.String("listen", "", "listen address") + cert = flag.String("cert", "", "certificate") + pkey = flag.String("key", "", "private key") + root = flag.String("root", "", "serve files from this root") + output = flag.String("output", "", "directory to write files to") + qlogdir = flag.String("qlog", "", "directory to write qlog output to") ) func main() { @@ -48,6 +51,10 @@ func main() { }, MaxBidiRemoteStreams: -1, MaxUniRemoteStreams: -1, + QLogLogger: slog.New(qlog.NewJSONHandler(qlog.HandlerOptions{ + Level: quic.QLogLevelFrame, + Dir: *qlogdir, + })), } if *cert != "" { c, err := tls.LoadX509KeyPair(*cert, *pkey) diff --git a/internal/quic/cmd/interop/run_endpoint.sh b/internal/quic/cmd/interop/run_endpoint.sh index d72335d8e..442039bc0 100644 --- a/internal/quic/cmd/interop/run_endpoint.sh +++ b/internal/quic/cmd/interop/run_endpoint.sh @@ -11,7 +11,7 @@ if [ "$ROLE" == "client" ]; then # Wait for the simulator to start up. /wait-for-it.sh sim:57832 -s -t 30 - ./interop -output=/downloads $CLIENT_PARAMS $REQUESTS + ./interop -output=/downloads -qlog=$QLOGDIR $CLIENT_PARAMS $REQUESTS elif [ "$ROLE" == "server" ]; then - ./interop -cert=/certs/cert.pem -key=/certs/priv.key -listen=:443 -root=/www "$@" $SERVER_PARAMS + ./interop -cert=/certs/cert.pem -key=/certs/priv.key -qlog=$QLOGDIR -listen=:443 -root=/www "$@" $SERVER_PARAMS fi diff --git a/internal/quic/config.go b/internal/quic/config.go index 6278bf89c..b10ecc79e 100644 --- a/internal/quic/config.go +++ b/internal/quic/config.go @@ -8,6 +8,7 @@ package quic import ( "crypto/tls" + "log/slog" ) // A Config structure configures a QUIC endpoint. @@ -72,6 +73,16 @@ type Config struct { // // If this field is left as zero, stateless reset is disabled. StatelessResetKey [32]byte + + // QLogLogger receives qlog events. + // + // Events currently correspond to the definitions in draft-ietf-qlog-quic-events-03. + // This is not the latest version of the draft, but is the latest version supported + // by common event log viewers as of the time this paragraph was written. + // + // The qlog package contains a slog.Handler which serializes qlog events + // to a standard JSON representation. + QLogLogger *slog.Logger } func configDefault(v, def, limit int64) int64 { diff --git a/internal/quic/conn.go b/internal/quic/conn.go index 1292f2b20..cca11166c 100644 --- a/internal/quic/conn.go +++ b/internal/quic/conn.go @@ -11,6 +11,7 @@ import ( "crypto/tls" "errors" "fmt" + "log/slog" "net/netip" "time" ) @@ -60,6 +61,8 @@ type Conn struct { // Tests only: Send a PING in a specific number space. testSendPingSpace numberSpace testSendPing sentVal + + log *slog.Logger } // connTestHooks override conn behavior in tests. @@ -94,7 +97,7 @@ type newServerConnIDs struct { retrySrcConnID []byte // source from server's Retry } -func newConn(now time.Time, side connSide, cids newServerConnIDs, peerAddr netip.AddrPort, config *Config, l *Listener) (*Conn, error) { +func newConn(now time.Time, side connSide, cids newServerConnIDs, peerAddr netip.AddrPort, config *Config, l *Listener) (conn *Conn, _ error) { c := &Conn{ side: side, listener: l, @@ -106,6 +109,14 @@ func newConn(now time.Time, side connSide, cids newServerConnIDs, peerAddr netip idleTimeout: now.Add(defaultMaxIdleTimeout), peerAckDelayExponent: -1, } + defer func() { + // If we hit an error in newConn, close donec so tests don't get stuck waiting for it. + // This is only relevant if we've got a bug, but it makes tracking that bug down + // much easier. + if conn == nil { + close(c.donec) + } + }() // A one-element buffer allows us to wake a Conn's event loop as a // non-blocking operation. @@ -135,6 +146,7 @@ func newConn(now time.Time, side connSide, cids newServerConnIDs, peerAddr netip // The smallest allowed maximum QUIC datagram size is 1200 bytes. // TODO: PMTU discovery. const maxDatagramSize = 1200 + c.logConnectionStarted(cids.originalDstConnID, peerAddr) c.keysAppData.init() c.loss.init(c.side, maxDatagramSize, now) c.streamsInit() @@ -259,6 +271,7 @@ func (c *Conn) loop(now time.Time) { defer close(c.donec) defer c.tls.Close() defer c.listener.connDrained(c) + defer c.logConnectionClosed() // The connection timer sends a message to the connection loop on expiry. // We need to give it an expiry when creating it, so set the initial timeout to diff --git a/internal/quic/conn_close_test.go b/internal/quic/conn_close_test.go index d583ae92a..0dd46dd20 100644 --- a/internal/quic/conn_close_test.go +++ b/internal/quic/conn_close_test.go @@ -70,7 +70,8 @@ func TestConnCloseResponseBackoff(t *testing.T) { } func TestConnCloseWithPeerResponse(t *testing.T) { - tc := newTestConn(t, clientSide) + qr := &qlogRecord{} + tc := newTestConn(t, clientSide, qr.config) tc.handshake() tc.conn.Abort(nil) @@ -99,10 +100,19 @@ func TestConnCloseWithPeerResponse(t *testing.T) { if err := tc.conn.Wait(canceledContext()); !errors.Is(err, wantErr) { t.Errorf("non-blocking conn.Wait() = %v, want %v", err, wantErr) } + + tc.advance(1 * time.Second) // long enough to exit the draining state + qr.wantEvents(t, jsonEvent{ + "name": "connectivity:connection_closed", + "data": map[string]any{ + "trigger": "application", + }, + }) } func TestConnClosePeerCloses(t *testing.T) { - tc := newTestConn(t, clientSide) + qr := &qlogRecord{} + tc := newTestConn(t, clientSide, qr.config) tc.handshake() wantErr := &ApplicationError{ @@ -128,6 +138,14 @@ func TestConnClosePeerCloses(t *testing.T) { code: 9, reason: "because", }) + + tc.advance(1 * time.Second) // long enough to exit the draining state + qr.wantEvents(t, jsonEvent{ + "name": "connectivity:connection_closed", + "data": map[string]any{ + "trigger": "application", + }, + }) } func TestConnCloseReceiveInInitial(t *testing.T) { diff --git a/internal/quic/conn_test.go b/internal/quic/conn_test.go index c70c58ef0..514a8775e 100644 --- a/internal/quic/conn_test.go +++ b/internal/quic/conn_test.go @@ -198,6 +198,7 @@ func newTestConn(t *testing.T, side connSide, opts ...any) *testConn { // The initial connection ID for the server is chosen by the client. cids.srcConnID = testPeerConnID(0) cids.dstConnID = testPeerConnID(-1) + cids.originalDstConnID = cids.dstConnID } var configTransportParams []func(*transportParameters) var configTestConn []func(*testConn) diff --git a/internal/quic/qlog.go b/internal/quic/qlog.go new file mode 100644 index 000000000..29875693e --- /dev/null +++ b/internal/quic/qlog.go @@ -0,0 +1,141 @@ +// Copyright 2023 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +//go:build go1.21 + +package quic + +import ( + "context" + "encoding/hex" + "log/slog" + "net/netip" +) + +// Log levels for qlog events. +const ( + // QLogLevelFrame includes per-frame information. + // When this level is enabled, packet_sent and packet_received events will + // contain information on individual frames sent/received. + QLogLevelFrame = slog.Level(-6) + + // QLogLevelPacket events occur at most once per packet sent or received. + // + // For example: packet_sent, packet_received. + QLogLevelPacket = slog.Level(-4) + + // QLogLevelConn events occur multiple times over a connection's lifetime, + // but less often than the frequency of individual packets. + // + // For example: connection_state_updated. + QLogLevelConn = slog.Level(-2) + + // QLogLevelEndpoint events occur at most once per connection. + // + // For example: connection_started, connection_closed. + QLogLevelEndpoint = slog.Level(0) +) + +func (c *Conn) logEnabled(level slog.Level) bool { + return c.log != nil && c.log.Enabled(context.Background(), level) +} + +// slogHexstring returns a slog.Attr for a value of the hexstring type. +// +// https://www.ietf.org/archive/id/draft-ietf-quic-qlog-main-schema-04.html#section-1.1.1 +func slogHexstring(key string, value []byte) slog.Attr { + return slog.String(key, hex.EncodeToString(value)) +} + +func slogAddr(key string, value netip.Addr) slog.Attr { + return slog.String(key, value.String()) +} + +func (c *Conn) logConnectionStarted(originalDstConnID []byte, peerAddr netip.AddrPort) { + if c.config.QLogLogger == nil || + !c.config.QLogLogger.Enabled(context.Background(), QLogLevelEndpoint) { + return + } + var vantage string + if c.side == clientSide { + vantage = "client" + originalDstConnID = c.connIDState.originalDstConnID + } else { + vantage = "server" + } + // A qlog Trace container includes some metadata (title, description, vantage_point) + // and a list of Events. The Trace also includes a common_fields field setting field + // values common to all events in the trace. + // + // Trace = { + // ? title: text + // ? description: text + // ? configuration: Configuration + // ? common_fields: CommonFields + // ? vantage_point: VantagePoint + // events: [* Event] + // } + // + // To map this into slog's data model, we start each per-connection trace with a With + // call that includes both the trace metadata and the common fields. + // + // This means that in slog's model, each trace event will also include + // the Trace metadata fields (vantage_point), which is a divergence from the qlog model. + c.log = c.config.QLogLogger.With( + // The group_id permits associating traces taken from different vantage points + // for the same connection. + // + // We use the original destination connection ID as the group ID. + // + // https://www.ietf.org/archive/id/draft-ietf-quic-qlog-main-schema-04.html#section-3.4.6 + slogHexstring("group_id", originalDstConnID), + slog.Group("vantage_point", + slog.String("name", "go quic"), + slog.String("type", vantage), + ), + ) + localAddr := c.listener.LocalAddr() + // https://www.ietf.org/archive/id/draft-ietf-quic-qlog-quic-events-03.html#section-4.2 + c.log.LogAttrs(context.Background(), QLogLevelEndpoint, + "connectivity:connection_started", + slogAddr("src_ip", localAddr.Addr()), + slog.Int("src_port", int(localAddr.Port())), + slogHexstring("src_cid", c.connIDState.local[0].cid), + slogAddr("dst_ip", peerAddr.Addr()), + slog.Int("dst_port", int(peerAddr.Port())), + slogHexstring("dst_cid", c.connIDState.remote[0].cid), + ) +} + +func (c *Conn) logConnectionClosed() { + if !c.logEnabled(QLogLevelEndpoint) { + return + } + err := c.lifetime.finalErr + trigger := "error" + switch e := err.(type) { + case *ApplicationError: + // TODO: Distinguish between peer and locally-initiated close. + trigger = "application" + case localTransportError: + if e.code == errNo { + trigger = "clean" + } + case peerTransportError: + if e.code == errNo { + trigger = "clean" + } + default: + switch err { + case errStatelessReset: + trigger = "stateless_reset" + } + // TODO: idle_timeout, handshake_timeout + } + // https://www.ietf.org/archive/id/draft-ietf-quic-qlog-quic-events-03.html#section-4.3 + c.log.LogAttrs(context.Background(), QLogLevelEndpoint, + "connectivity:connection_closed", + slog.String("trigger", trigger), + ) +} diff --git a/internal/quic/qlog/handler.go b/internal/quic/qlog/handler.go new file mode 100644 index 000000000..35a66cf8b --- /dev/null +++ b/internal/quic/qlog/handler.go @@ -0,0 +1,76 @@ +// Copyright 2023 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +//go:build go1.21 + +package qlog + +import ( + "context" + "log/slog" +) + +type withAttrsHandler struct { + attrs []slog.Attr + h slog.Handler +} + +func withAttrs(h slog.Handler, attrs []slog.Attr) slog.Handler { + if len(attrs) == 0 { + return h + } + return &withAttrsHandler{attrs: attrs, h: h} +} + +func (h *withAttrsHandler) Enabled(ctx context.Context, level slog.Level) bool { + return h.h.Enabled(ctx, level) +} + +func (h *withAttrsHandler) Handle(ctx context.Context, r slog.Record) error { + r.AddAttrs(h.attrs...) + return h.h.Handle(ctx, r) +} + +func (h *withAttrsHandler) WithAttrs(attrs []slog.Attr) slog.Handler { + return withAttrs(h, attrs) +} + +func (h *withAttrsHandler) WithGroup(name string) slog.Handler { + return withGroup(h, name) +} + +type withGroupHandler struct { + name string + h slog.Handler +} + +func withGroup(h slog.Handler, name string) slog.Handler { + if name == "" { + return h + } + return &withGroupHandler{name: name, h: h} +} + +func (h *withGroupHandler) Enabled(ctx context.Context, level slog.Level) bool { + return h.h.Enabled(ctx, level) +} + +func (h *withGroupHandler) Handle(ctx context.Context, r slog.Record) error { + var attrs []slog.Attr + r.Attrs(func(a slog.Attr) bool { + attrs = append(attrs, a) + return true + }) + nr := slog.NewRecord(r.Time, r.Level, r.Message, r.PC) + nr.Add(slog.Any(h.name, slog.GroupValue(attrs...))) + return h.h.Handle(ctx, nr) +} + +func (h *withGroupHandler) WithAttrs(attrs []slog.Attr) slog.Handler { + return withAttrs(h, attrs) +} + +func (h *withGroupHandler) WithGroup(name string) slog.Handler { + return withGroup(h, name) +} diff --git a/internal/quic/qlog/json_writer.go b/internal/quic/qlog/json_writer.go new file mode 100644 index 000000000..50cf33bc5 --- /dev/null +++ b/internal/quic/qlog/json_writer.go @@ -0,0 +1,194 @@ +// Copyright 2023 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +//go:build go1.21 + +package qlog + +import ( + "bytes" + "fmt" + "io" + "log/slog" + "strconv" + "sync" + "time" +) + +// A jsonWriter writes JSON-SEQ (RFC 7464). +// +// A JSON-SEQ file consists of a series of JSON text records, +// each beginning with an RS (0x1e) character and ending with LF (0x0a). +type jsonWriter struct { + mu sync.Mutex + w io.WriteCloser + buf bytes.Buffer +} + +// writeRecordStart writes the start of a JSON-SEQ record. +func (w *jsonWriter) writeRecordStart() { + w.mu.Lock() + w.buf.WriteByte(0x1e) + w.buf.WriteByte('{') +} + +// writeRecordEnd finishes writing a JSON-SEQ record. +func (w *jsonWriter) writeRecordEnd() { + w.buf.WriteByte('}') + w.buf.WriteByte('\n') + w.w.Write(w.buf.Bytes()) + w.buf.Reset() + w.mu.Unlock() +} + +// writeAttrsField writes a []slog.Attr as an object field. +func (w *jsonWriter) writeAttrsField(name string, attrs []slog.Attr) { + w.writeName(name) + w.buf.WriteByte('{') + for _, a := range attrs { + w.writeAttr(a) + } + w.buf.WriteByte('}') +} + +// writeAttr writes a slog.Attr as an object field. +func (w *jsonWriter) writeAttr(a slog.Attr) { + v := a.Value.Resolve() + switch v.Kind() { + case slog.KindAny: + w.writeStringField(a.Key, fmt.Sprint(v.Any())) + case slog.KindBool: + w.writeBoolField(a.Key, v.Bool()) + case slog.KindDuration: + w.writeDurationField(a.Key, v.Duration()) + case slog.KindFloat64: + w.writeFloat64Field(a.Key, v.Float64()) + case slog.KindInt64: + w.writeInt64Field(a.Key, v.Int64()) + case slog.KindString: + w.writeStringField(a.Key, v.String()) + case slog.KindTime: + w.writeTimeField(a.Key, v.Time()) + case slog.KindUint64: + w.writeUint64Field(a.Key, v.Uint64()) + case slog.KindGroup: + w.writeAttrsField(a.Key, v.Group()) + default: + w.writeString("unhandled kind") + } +} + +// writeName writes an object field name followed by a colon. +func (w *jsonWriter) writeName(name string) { + if b := w.buf.Bytes(); len(b) > 0 && b[len(b)-1] != '{' { + // Add the comma separating this from the previous field. + w.buf.WriteByte(',') + } + w.writeString(name) + w.buf.WriteByte(':') +} + +// writeObject writes an object-valued object field. +// The function f is called to write the contents. +func (w *jsonWriter) writeObjectField(name string, f func()) { + w.writeName(name) + w.buf.WriteByte('{') + f() + w.buf.WriteByte('}') +} + +// writeRawField writes an field with a raw JSON value. +func (w *jsonWriter) writeRawField(name, v string) { + w.writeName(name) + w.buf.WriteString(v) +} + +// writeBoolField writes a bool-valued object field. +func (w *jsonWriter) writeBoolField(name string, v bool) { + w.writeName(name) + if v { + w.buf.WriteString("true") + } else { + w.buf.WriteString("false") + } +} + +// writeDurationField writes a millisecond duration-valued object field. +func (w *jsonWriter) writeDurationField(name string, v time.Duration) { + w.writeName(name) + fmt.Fprintf(&w.buf, "%d.%06d", v.Milliseconds(), v%time.Millisecond) +} + +// writeFloat64Field writes an float64-valued object field. +func (w *jsonWriter) writeFloat64Field(name string, v float64) { + w.writeName(name) + w.buf.Write(strconv.AppendFloat(w.buf.AvailableBuffer(), v, 'f', -1, 64)) +} + +// writeInt64Field writes an int64-valued object field. +func (w *jsonWriter) writeInt64Field(name string, v int64) { + w.writeName(name) + w.buf.Write(strconv.AppendInt(w.buf.AvailableBuffer(), v, 10)) +} + +// writeUint64Field writes a uint64-valued object field. +func (w *jsonWriter) writeUint64Field(name string, v uint64) { + w.writeName(name) + w.buf.Write(strconv.AppendUint(w.buf.AvailableBuffer(), v, 10)) +} + +// writeStringField writes a string-valued object field. +func (w *jsonWriter) writeStringField(name, v string) { + w.writeName(name) + w.writeString(v) +} + +// writeTimeField writes a time-valued object field. +func (w *jsonWriter) writeTimeField(name string, v time.Time) { + w.writeName(name) + fmt.Fprintf(&w.buf, "%d.%06d", v.UnixMilli(), v.Nanosecond()%int(time.Millisecond)) +} + +func jsonSafeSet(c byte) bool { + // mask is a 128-bit bitmap with 1s for allowed bytes, + // so that the byte c can be tested with a shift and an and. + // If c > 128, then 1<>64)) != 0 +} + +func jsonNeedsEscape(s string) bool { + for i := range s { + if !jsonSafeSet(s[i]) { + return true + } + } + return false +} + +// writeString writes an ASCII string. +// +// qlog fields should never contain anything that isn't ASCII, +// so we do the bare minimum to avoid producing invalid output if we +// do write something unexpected. +func (w *jsonWriter) writeString(v string) { + w.buf.WriteByte('"') + if !jsonNeedsEscape(v) { + w.buf.WriteString(v) + } else { + for i := range v { + if jsonSafeSet(v[i]) { + w.buf.WriteByte(v[i]) + } else { + fmt.Fprintf(&w.buf, `\u%04x`, v[i]) + } + } + } + w.buf.WriteByte('"') +} diff --git a/internal/quic/qlog/json_writer_test.go b/internal/quic/qlog/json_writer_test.go new file mode 100644 index 000000000..7ba5e1737 --- /dev/null +++ b/internal/quic/qlog/json_writer_test.go @@ -0,0 +1,186 @@ +// Copyright 2023 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +//go:build go1.21 + +package qlog + +import ( + "bytes" + "errors" + "fmt" + "log/slog" + "strings" + "sync" + "testing" + "time" +) + +type testJSONOut struct { + bytes.Buffer +} + +func (o *testJSONOut) Close() error { return nil } + +func newTestJSONWriter() *jsonWriter { + return &jsonWriter{w: &testJSONOut{}} +} + +func wantJSONRecord(t *testing.T, w *jsonWriter, want string) { + t.Helper() + want = "\x1e" + want + "\n" + got := w.w.(*testJSONOut).String() + if got != want { + t.Errorf("jsonWriter contains unexpected output\ngot: %q\nwant: %q", got, want) + } +} + +func TestJSONWriterWriteConcurrentRecords(t *testing.T) { + w := newTestJSONWriter() + var wg sync.WaitGroup + for i := 0; i < 3; i++ { + wg.Add(1) + go func() { + defer wg.Done() + w.writeRecordStart() + w.writeInt64Field("field", 0) + w.writeRecordEnd() + }() + } + wg.Wait() + wantJSONRecord(t, w, strings.Join([]string{ + `{"field":0}`, + `{"field":0}`, + `{"field":0}`, + }, "\n\x1e")) +} + +func TestJSONWriterAttrs(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeAttrsField("field", []slog.Attr{ + slog.Any("any", errors.New("value")), + slog.Bool("bool", true), + slog.Duration("duration", 1*time.Second), + slog.Float64("float64", 1), + slog.Int64("int64", 1), + slog.String("string", "value"), + slog.Time("time", time.Date(2000, 1, 1, 0, 0, 0, 0, time.UTC)), + slog.Uint64("uint64", 1), + slog.Group("group", "a", 1), + }) + w.writeRecordEnd() + wantJSONRecord(t, w, + `{"field":{`+ + `"any":"value",`+ + `"bool":true,`+ + `"duration":1000.000000,`+ + `"float64":1,`+ + `"int64":1,`+ + `"string":"value",`+ + `"time":946684800000.000000,`+ + `"uint64":1,`+ + `"group":{"a":1}`+ + `}}`) +} + +func TestJSONWriterObjectEmpty(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeObjectField("field", func() {}) + w.writeRecordEnd() + wantJSONRecord(t, w, `{"field":{}}`) +} + +func TestJSONWriterObjectFields(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeObjectField("field", func() { + w.writeStringField("a", "value") + w.writeInt64Field("b", 10) + }) + w.writeRecordEnd() + wantJSONRecord(t, w, `{"field":{"a":"value","b":10}}`) +} + +func TestJSONWriterRawField(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeRawField("field", `[1]`) + w.writeRecordEnd() + wantJSONRecord(t, w, `{"field":[1]}`) +} + +func TestJSONWriterBoolField(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeBoolField("true", true) + w.writeBoolField("false", false) + w.writeRecordEnd() + wantJSONRecord(t, w, `{"true":true,"false":false}`) +} + +func TestJSONWriterDurationField(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeDurationField("field", (10*time.Millisecond)+(2*time.Nanosecond)) + w.writeRecordEnd() + wantJSONRecord(t, w, `{"field":10.000002}`) +} + +func TestJSONWriterFloat64Field(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeFloat64Field("field", 1.1) + w.writeRecordEnd() + wantJSONRecord(t, w, `{"field":1.1}`) +} + +func TestJSONWriterInt64Field(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeInt64Field("field", 1234) + w.writeRecordEnd() + wantJSONRecord(t, w, `{"field":1234}`) +} + +func TestJSONWriterUint64Field(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeUint64Field("field", 1234) + w.writeRecordEnd() + wantJSONRecord(t, w, `{"field":1234}`) +} + +func TestJSONWriterStringField(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeStringField("field", "value") + w.writeRecordEnd() + wantJSONRecord(t, w, `{"field":"value"}`) +} + +func TestJSONWriterStringFieldEscaped(t *testing.T) { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeStringField("field", "va\x00ue") + w.writeRecordEnd() + wantJSONRecord(t, w, `{"field":"va\u0000ue"}`) +} + +func TestJSONWriterStringEscaping(t *testing.T) { + for c := 0; c <= 0xff; c++ { + w := newTestJSONWriter() + w.writeRecordStart() + w.writeStringField("field", string([]byte{byte(c)})) + w.writeRecordEnd() + var want string + if (c >= 0x20 && c <= 0x21) || (c >= 0x23 && c <= 0x5b) || (c >= 0x5d && c <= 0x7e) { + want = fmt.Sprintf(`%c`, c) + } else { + want = fmt.Sprintf(`\u%04x`, c) + } + wantJSONRecord(t, w, `{"field":"`+want+`"}`) + } +} diff --git a/internal/quic/qlog/qlog.go b/internal/quic/qlog/qlog.go new file mode 100644 index 000000000..0e71d71aa --- /dev/null +++ b/internal/quic/qlog/qlog.go @@ -0,0 +1,267 @@ +// Copyright 2023 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +//go:build go1.21 + +// Package qlog serializes qlog events. +package qlog + +import ( + "bytes" + "context" + "errors" + "io" + "log/slog" + "os" + "path/filepath" + "sync" + "time" +) + +// Vantage is the vantage point of a trace. +type Vantage string + +const ( + // VantageEndpoint traces contain events not specific to a single connection. + VantageEndpoint = Vantage("endpoint") + + // VantageClient traces follow a connection from the client's perspective. + VantageClient = Vantage("client") + + // VantageClient traces follow a connection from the server's perspective. + VantageServer = Vantage("server") +) + +// TraceInfo contains information about a trace. +type TraceInfo struct { + // Vantage is the vantage point of the trace. + Vantage Vantage + + // GroupID identifies the logical group the trace belongs to. + // For a connection trace, the group will be the same for + // both the client and server vantage points. + GroupID string +} + +// HandlerOptions are options for a JSONHandler. +type HandlerOptions struct { + // Level reports the minimum record level that will be logged. + // If Level is nil, the handler assumes QLogLevelEndpoint. + Level slog.Leveler + + // Dir is the directory in which to create trace files. + // The handler will create one file per connection. + // If NewTrace is non-nil or Dir is "", the handler will not create files. + Dir string + + // NewTrace is called to create a new trace. + // If NewTrace is nil and Dir is set, + // the handler will create a new file in Dir for each trace. + NewTrace func(TraceInfo) (io.WriteCloser, error) +} + +type endpointHandler struct { + opts HandlerOptions + + traceOnce sync.Once + trace *jsonTraceHandler +} + +// NewJSONHandler returns a handler which serializes qlog events to JSON. +// +// The handler will write an endpoint-wide trace, +// and a separate trace for each connection. +// The HandlerOptions control the location traces are written. +// +// It uses the streamable JSON Text Sequences mapping (JSON-SEQ) +// defined in draft-ietf-quic-qlog-main-schema-04, Section 6.2. +// +// A JSONHandler may be used as the handler for a quic.Config.QLogLogger. +// It is not a general-purpose slog handler, +// and may not properly handle events from other sources. +func NewJSONHandler(opts HandlerOptions) slog.Handler { + if opts.Dir == "" && opts.NewTrace == nil { + return slogDiscard{} + } + return &endpointHandler{ + opts: opts, + } +} + +func (h *endpointHandler) Enabled(ctx context.Context, level slog.Level) bool { + return enabled(h.opts.Level, level) +} + +func (h *endpointHandler) Handle(ctx context.Context, r slog.Record) error { + h.traceOnce.Do(func() { + h.trace, _ = newJSONTraceHandler(h.opts, nil) + }) + if h.trace != nil { + h.trace.Handle(ctx, r) + } + return nil +} + +func (h *endpointHandler) WithAttrs(attrs []slog.Attr) slog.Handler { + // Create a new trace output file for each top-level WithAttrs. + tr, err := newJSONTraceHandler(h.opts, attrs) + if err != nil { + return withAttrs(h, attrs) + } + return tr +} + +func (h *endpointHandler) WithGroup(name string) slog.Handler { + return withGroup(h, name) +} + +type jsonTraceHandler struct { + level slog.Leveler + w jsonWriter + start time.Time + buf bytes.Buffer +} + +func newJSONTraceHandler(opts HandlerOptions, attrs []slog.Attr) (*jsonTraceHandler, error) { + w, err := newTraceWriter(opts, traceInfoFromAttrs(attrs)) + if err != nil { + return nil, err + } + + // For testing, it might be nice to set the start time used for relative timestamps + // to the time of the first event. + // + // At the expense of some additional complexity here, we could defer writing + // the reference_time header field until the first event is processed. + // + // Just use the current time for now. + start := time.Now() + + h := &jsonTraceHandler{ + w: jsonWriter{w: w}, + level: opts.Level, + start: start, + } + h.writeHeader(attrs) + return h, nil +} + +func traceInfoFromAttrs(attrs []slog.Attr) TraceInfo { + info := TraceInfo{ + Vantage: VantageEndpoint, // default if not specified + } + for _, a := range attrs { + if a.Key == "group_id" && a.Value.Kind() == slog.KindString { + info.GroupID = a.Value.String() + } + if a.Key == "vantage_point" && a.Value.Kind() == slog.KindGroup { + for _, aa := range a.Value.Group() { + if aa.Key == "type" && aa.Value.Kind() == slog.KindString { + info.Vantage = Vantage(aa.Value.String()) + } + } + } + } + return info +} + +func newTraceWriter(opts HandlerOptions, info TraceInfo) (io.WriteCloser, error) { + var w io.WriteCloser + var err error + if opts.NewTrace != nil { + w, err = opts.NewTrace(info) + } else if opts.Dir != "" { + var filename string + if info.GroupID != "" { + filename = info.GroupID + "_" + } + filename += string(info.Vantage) + ".sqlog" + if !filepath.IsLocal(filename) { + return nil, errors.New("invalid trace filename") + } + w, err = os.Create(filepath.Join(opts.Dir, filename)) + } else { + err = errors.New("no log destination") + } + return w, err +} + +func (h *jsonTraceHandler) writeHeader(attrs []slog.Attr) { + h.w.writeRecordStart() + defer h.w.writeRecordEnd() + + // At the time of writing this comment the most recent version is 0.4, + // but qvis only supports up to 0.3. + h.w.writeStringField("qlog_version", "0.3") + h.w.writeStringField("qlog_format", "JSON-SEQ") + + // The attrs flatten both common trace event fields and Trace fields. + // This identifies the fields that belong to the Trace. + isTraceSeqField := func(s string) bool { + switch s { + case "title", "description", "configuration", "vantage_point": + return true + } + return false + } + + h.w.writeObjectField("trace", func() { + h.w.writeObjectField("common_fields", func() { + h.w.writeRawField("protocol_type", `["QUIC"]`) + h.w.writeStringField("time_format", "relative") + h.w.writeTimeField("reference_time", h.start) + for _, a := range attrs { + if !isTraceSeqField(a.Key) { + h.w.writeAttr(a) + } + } + }) + for _, a := range attrs { + if isTraceSeqField(a.Key) { + h.w.writeAttr(a) + } + } + }) +} + +func (h *jsonTraceHandler) Enabled(ctx context.Context, level slog.Level) bool { + return enabled(h.level, level) +} + +func (h *jsonTraceHandler) Handle(ctx context.Context, r slog.Record) error { + h.w.writeRecordStart() + defer h.w.writeRecordEnd() + h.w.writeDurationField("time", r.Time.Sub(h.start)) + h.w.writeStringField("name", r.Message) + h.w.writeObjectField("data", func() { + r.Attrs(func(a slog.Attr) bool { + h.w.writeAttr(a) + return true + }) + }) + return nil +} + +func (h *jsonTraceHandler) WithAttrs(attrs []slog.Attr) slog.Handler { + return withAttrs(h, attrs) +} + +func (h *jsonTraceHandler) WithGroup(name string) slog.Handler { + return withGroup(h, name) +} + +func enabled(leveler slog.Leveler, level slog.Level) bool { + var minLevel slog.Level + if leveler != nil { + minLevel = leveler.Level() + } + return level >= minLevel +} + +type slogDiscard struct{} + +func (slogDiscard) Enabled(context.Context, slog.Level) bool { return false } +func (slogDiscard) Handle(ctx context.Context, r slog.Record) error { return nil } +func (slogDiscard) WithAttrs(attrs []slog.Attr) slog.Handler { return slogDiscard{} } +func (slogDiscard) WithGroup(name string) slog.Handler { return slogDiscard{} } diff --git a/internal/quic/qlog/qlog_test.go b/internal/quic/qlog/qlog_test.go new file mode 100644 index 000000000..7575cd890 --- /dev/null +++ b/internal/quic/qlog/qlog_test.go @@ -0,0 +1,151 @@ +// Copyright 2023 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +//go:build go1.21 + +package qlog + +import ( + "bytes" + "encoding/json" + "fmt" + "io" + "log/slog" + "reflect" + "testing" + "time" +) + +// QLog tests are mostly in the quic package, where we can test event generation +// and serialization together. + +func TestQLogHandlerEvents(t *testing.T) { + for _, test := range []struct { + name string + f func(*slog.Logger) + want []map[string]any // events, not counting the trace header + }{{ + name: "various types", + f: func(log *slog.Logger) { + log.Info("message", + "bool", true, + "duration", time.Duration(1*time.Second), + "float", 0.0, + "int", 0, + "string", "value", + "uint", uint64(0), + slog.Group("group", + "a", 0, + ), + ) + }, + want: []map[string]any{{ + "name": "message", + "data": map[string]any{ + "bool": true, + "duration": float64(1000), + "float": float64(0.0), + "int": float64(0), + "string": "value", + "uint": float64(0), + "group": map[string]any{ + "a": float64(0), + }, + }, + }}, + }, { + name: "WithAttrs", + f: func(log *slog.Logger) { + log = log.With( + "with_a", "a", + "with_b", "b", + ) + log.Info("m1", "field", "1") + log.Info("m2", "field", "2") + }, + want: []map[string]any{{ + "name": "m1", + "data": map[string]any{ + "with_a": "a", + "with_b": "b", + "field": "1", + }, + }, { + "name": "m2", + "data": map[string]any{ + "with_a": "a", + "with_b": "b", + "field": "2", + }, + }}, + }, { + name: "WithGroup", + f: func(log *slog.Logger) { + log = log.With( + "with_a", "a", + "with_b", "b", + ) + log.Info("m1", "field", "1") + log.Info("m2", "field", "2") + }, + want: []map[string]any{{ + "name": "m1", + "data": map[string]any{ + "with_a": "a", + "with_b": "b", + "field": "1", + }, + }, { + "name": "m2", + "data": map[string]any{ + "with_a": "a", + "with_b": "b", + "field": "2", + }, + }}, + }} { + var out bytes.Buffer + opts := HandlerOptions{ + Level: slog.LevelDebug, + NewTrace: func(TraceInfo) (io.WriteCloser, error) { + return nopCloseWriter{&out}, nil + }, + } + h, err := newJSONTraceHandler(opts, []slog.Attr{ + slog.String("group_id", "group"), + slog.Group("vantage_point", + slog.String("type", "client"), + ), + }) + if err != nil { + t.Fatal(err) + } + log := slog.New(h) + test.f(log) + got := []map[string]any{} + for i, e := range bytes.Split(out.Bytes(), []byte{0x1e}) { + // i==0: empty string before the initial record separator + // i==1: trace header; not part of this test + if i < 2 { + continue + } + var val map[string]any + if err := json.Unmarshal(e, &val); err != nil { + panic(fmt.Errorf("log unmarshal failure: %v\n%q", err, string(e))) + } + delete(val, "time") + got = append(got, val) + } + if !reflect.DeepEqual(got, test.want) { + t.Errorf("event mismatch\ngot: %v\nwant: %v", got, test.want) + } + } + +} + +type nopCloseWriter struct { + io.Writer +} + +func (nopCloseWriter) Close() error { return nil } diff --git a/internal/quic/qlog_test.go b/internal/quic/qlog_test.go new file mode 100644 index 000000000..5a2858b8b --- /dev/null +++ b/internal/quic/qlog_test.go @@ -0,0 +1,132 @@ +// Copyright 2023 The Go Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSE file. + +//go:build go1.21 + +package quic + +import ( + "encoding/hex" + "encoding/json" + "fmt" + "io" + "log/slog" + "reflect" + "testing" + + "golang.org/x/net/internal/quic/qlog" +) + +func TestQLogHandshake(t *testing.T) { + testSides(t, "", func(t *testing.T, side connSide) { + qr := &qlogRecord{} + tc := newTestConn(t, side, qr.config) + tc.handshake() + tc.conn.Abort(nil) + tc.wantFrame("aborting connection generates CONN_CLOSE", + packetType1RTT, debugFrameConnectionCloseTransport{ + code: errNo, + }) + tc.writeFrames(packetType1RTT, debugFrameConnectionCloseTransport{}) + tc.advanceToTimer() // let the conn finish draining + + var src, dst []byte + if side == clientSide { + src = testLocalConnID(0) + dst = testLocalConnID(-1) + } else { + src = testPeerConnID(-1) + dst = testPeerConnID(0) + } + qr.wantEvents(t, jsonEvent{ + "name": "connectivity:connection_started", + "data": map[string]any{ + "src_cid": hex.EncodeToString(src), + "dst_cid": hex.EncodeToString(dst), + }, + }, jsonEvent{ + "name": "connectivity:connection_closed", + "data": map[string]any{ + "trigger": "clean", + }, + }) + }) +} + +type nopCloseWriter struct { + io.Writer +} + +func (nopCloseWriter) Close() error { return nil } + +type jsonEvent map[string]any + +func (j jsonEvent) String() string { + b, _ := json.MarshalIndent(j, "", " ") + return string(b) +} + +// eventPartialEqual verifies that every field set in want matches the corresponding field in got. +// It ignores additional fields in got. +func eventPartialEqual(got, want jsonEvent) bool { + for k := range want { + ge, gok := got[k].(map[string]any) + we, wok := want[k].(map[string]any) + if gok && wok { + if !eventPartialEqual(ge, we) { + return false + } + } else { + if !reflect.DeepEqual(got[k], want[k]) { + return false + } + } + } + return true +} + +// A qlogRecord records events. +type qlogRecord struct { + ev []jsonEvent +} + +func (q *qlogRecord) Write(b []byte) (int, error) { + // This relies on the property that the Handler always makes one Write call per event. + if len(b) < 1 || b[0] != 0x1e { + panic(fmt.Errorf("trace Write should start with record separator, got %q", string(b))) + } + var val map[string]any + if err := json.Unmarshal(b[1:], &val); err != nil { + panic(fmt.Errorf("log unmarshal failure: %v\n%v", err, string(b))) + } + q.ev = append(q.ev, val) + return len(b), nil +} + +func (q *qlogRecord) Close() error { return nil } + +// config may be passed to newTestConn to configure the conn to use this logger. +func (q *qlogRecord) config(c *Config) { + c.QLogLogger = slog.New(qlog.NewJSONHandler(qlog.HandlerOptions{ + NewTrace: func(info qlog.TraceInfo) (io.WriteCloser, error) { + return q, nil + }, + })) +} + +// wantEvents checks that every event in want occurs in the order specified. +func (q *qlogRecord) wantEvents(t *testing.T, want ...jsonEvent) { + t.Helper() + got := q.ev + unseen := want + for _, g := range got { + if eventPartialEqual(g, unseen[0]) { + unseen = unseen[1:] + if len(unseen) == 0 { + return + } + } + } + t.Fatalf("got events:\n%v\n\nwant events:\n%v", got, want) +} diff --git a/internal/quic/stateless_reset_test.go b/internal/quic/stateless_reset_test.go index 8a16597c4..c01375fbd 100644 --- a/internal/quic/stateless_reset_test.go +++ b/internal/quic/stateless_reset_test.go @@ -130,7 +130,8 @@ func TestStatelessResetSentSizes(t *testing.T) { func TestStatelessResetSuccessfulNewConnectionID(t *testing.T) { // "[...] Stateless Reset Token field values from [...] NEW_CONNECTION_ID frames [...]" // https://www.rfc-editor.org/rfc/rfc9000#section-10.3.1-1 - tc := newTestConn(t, clientSide) + qr := &qlogRecord{} + tc := newTestConn(t, clientSide, qr.config) tc.handshake() tc.ignoreFrame(frameTypeAck) @@ -158,6 +159,13 @@ func TestStatelessResetSuccessfulNewConnectionID(t *testing.T) { tc.wantIdle("closed connection is idle in draining") tc.advance(1 * time.Second) // long enough to exit the draining state tc.wantIdle("closed connection is idle after draining") + + qr.wantEvents(t, jsonEvent{ + "name": "connectivity:connection_closed", + "data": map[string]any{ + "trigger": "stateless_reset", + }, + }) } func TestStatelessResetSuccessfulTransportParameter(t *testing.T) {