diff --git a/Makefile b/Makefile index d4fcd2d0a1c4..d69d151af04e 100644 --- a/Makefile +++ b/Makefile @@ -1549,6 +1549,7 @@ EVENTLOG_PROTOS = \ pkg/util/log/eventpb/privilege_events.proto \ pkg/util/log/eventpb/role_events.proto \ pkg/util/log/eventpb/zone_events.proto \ + pkg/util/log/eventpb/session_events.proto \ pkg/util/log/eventpb/cluster_events.proto docs/generated/eventlog.md: pkg/util/log/eventpb/gen.go $(EVENTLOG_PROTOS) | bin/.go_protobuf_sources diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index 8cb9a42f700b..c71edfaad871 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -1183,6 +1183,172 @@ removed from a user for a type object. | `GrantedPrivileges` | The privileges being granted to the grantee. | no | | `RevokedPrivileges` | The privileges being revoked from the grantee. | no | +## SQL Session events + +Events in this category report SQL client connections +and sessions. + +They are relative to a particular SQL tenant. +In a multi-tenant setup, copies of these miscellaneous events are +preserved in each tenant's own system.eventlog table. + +Events in this category are logged to channel SESSIONS. + + +### `client_authentication_failed` + +An event of type `client_authentication_failed` is reported when a client session +did not authenticate successfully. + +Events of this type are only emitted when the cluster setting +`server.auth_log.sql_sessions.enabled` is set. + + +| Field | Description | Sensitive | +|--|--|--| +| `Reason` | The reason for the authentication failure. | yes | +| `Detail` | The detailed error for the authentication failure. | yes | +| `Method` | The authentication method used. | yes | + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `InstanceID` | The instance ID (not tenant ID) of the SQL server where the event was originated. | no | +| `Network` | The network protocol for this connection: tcp4, tcp6, unix, etc. | yes | +| `RemoteAddress` | The remote address of the SQL client. Note that when using a proxy or other intermediate server, this field will contain the address of the intermediate server. | yes | +| `Transport` | The connection type after transport negotiation. | yes | +| `User` | The username the session is for. This is the username passed by the client, after case-folding and Unicode normalization. | yes | + +### `client_authentication_info` + +An event of type `client_authentication_info` is reported for intermediate +steps during the authentication process. + +Events of this type are only emitted when the cluster setting +`server.auth_log.sql_sessions.enabled` is set. + + +| Field | Description | Sensitive | +|--|--|--| +| `Method` | The authentication method used, once known. | yes | +| `Info` | The authentication progress message. | yes | + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `InstanceID` | The instance ID (not tenant ID) of the SQL server where the event was originated. | no | +| `Network` | The network protocol for this connection: tcp4, tcp6, unix, etc. | yes | +| `RemoteAddress` | The remote address of the SQL client. Note that when using a proxy or other intermediate server, this field will contain the address of the intermediate server. | yes | +| `Transport` | The connection type after transport negotiation. | yes | +| `User` | The username the session is for. This is the username passed by the client, after case-folding and Unicode normalization. | yes | + +### `client_authentication_ok` + +An event of type `client_authentication_ok` is reported when a client session +was authenticated successfully. + +Events of this type are only emitted when the cluster setting +`server.auth_log.sql_sessions.enabled` is set. + + +| Field | Description | Sensitive | +|--|--|--| +| `Method` | The authentication method used. | yes | + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `InstanceID` | The instance ID (not tenant ID) of the SQL server where the event was originated. | no | +| `Network` | The network protocol for this connection: tcp4, tcp6, unix, etc. | yes | +| `RemoteAddress` | The remote address of the SQL client. Note that when using a proxy or other intermediate server, this field will contain the address of the intermediate server. | yes | +| `Transport` | The connection type after transport negotiation. | yes | +| `User` | The username the session is for. This is the username passed by the client, after case-folding and Unicode normalization. | yes | + +### `client_connection_end` + +An event of type `client_connection_end` is reported when a client connection +is closed. This is reported even when authentication +fails, and even for simple cancellation messages. + +Events of this type are only emitted when the cluster setting +`server.auth_log.sql_connections.enabled` is set. + + +| Field | Description | Sensitive | +|--|--|--| +| `Duration` | The duration of the connection in nanoseconds. | no | + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `InstanceID` | The instance ID (not tenant ID) of the SQL server where the event was originated. | no | +| `Network` | The network protocol for this connection: tcp4, tcp6, unix, etc. | yes | +| `RemoteAddress` | The remote address of the SQL client. Note that when using a proxy or other intermediate server, this field will contain the address of the intermediate server. | yes | + +### `client_connection_start` + +An event of type `client_connection_start` is reported when a client connection +is established. This is reported even when authentication +fails, and even for simple cancellation messages. + +Events of this type are only emitted when the cluster setting +`server.auth_log.sql_connections.enabled` is set. + + + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `InstanceID` | The instance ID (not tenant ID) of the SQL server where the event was originated. | no | +| `Network` | The network protocol for this connection: tcp4, tcp6, unix, etc. | yes | +| `RemoteAddress` | The remote address of the SQL client. Note that when using a proxy or other intermediate server, this field will contain the address of the intermediate server. | yes | + +### `client_session_end` + +An event of type `client_session_end` is reported when a client session +is completed. + +Events of this type are only emitted when the cluster setting +`server.auth_log.sql_sessions.enabled` is set. + + +| Field | Description | Sensitive | +|--|--|--| +| `Duration` | The duration of the connection in nanoseconds. | no | + + +#### Common fields + +| Field | Description | Sensitive | +|--|--|--| +| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no | +| `EventType` | The type of the event. | no | +| `InstanceID` | The instance ID (not tenant ID) of the SQL server where the event was originated. | no | +| `Network` | The network protocol for this connection: tcp4, tcp6, unix, etc. | yes | +| `RemoteAddress` | The remote address of the SQL client. Note that when using a proxy or other intermediate server, this field will contain the address of the intermediate server. | yes | +| `Transport` | The connection type after transport negotiation. | yes | +| `User` | The username the session is for. This is the username passed by the client, after case-folding and Unicode normalization. | yes | + ## SQL User and Role operations Events in this category pertain to SQL statements that modify the diff --git a/pkg/sql/pgwire/BUILD.bazel b/pkg/sql/pgwire/BUILD.bazel index ffcd2656804a..401f1248f46e 100644 --- a/pkg/sql/pgwire/BUILD.bazel +++ b/pkg/sql/pgwire/BUILD.bazel @@ -43,6 +43,7 @@ go_library( "//pkg/util/humanizeutil", "//pkg/util/ipaddr", "//pkg/util/log", + "//pkg/util/log/eventpb", "//pkg/util/metric", "//pkg/util/mon", "//pkg/util/stop", @@ -82,6 +83,7 @@ go_test( "//pkg/sql", "//pkg/sql/catalog/colinfo", "//pkg/sql/parser", + "//pkg/sql/pgwire/hba", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", "//pkg/sql/pgwire/pgwirebase", @@ -112,6 +114,7 @@ go_test( "@com_github_cockroachdb_datadriven//:datadriven", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_errors//stdstrings", + "@com_github_cockroachdb_redact//:redact", "@com_github_jackc_pgproto3_v2//:pgproto3", "@com_github_jackc_pgx//:pgx", "@com_github_lib_pq//:pq", diff --git a/pkg/sql/pgwire/auth.go b/pkg/sql/pgwire/auth.go index dfee602713a7..53d774489b43 100644 --- a/pkg/sql/pgwire/auth.go +++ b/pkg/sql/pgwire/auth.go @@ -13,6 +13,7 @@ package pgwire import ( "context" "crypto/tls" + "fmt" "net" "github.com/cockroachdb/cockroach/pkg/security" @@ -20,6 +21,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/errors" ) @@ -40,6 +42,9 @@ type authOptions struct { // connType is the actual type of client connection (e.g. local, // hostssl, hostnossl). connType hba.ConnType + // connDetails is the event payload common to all auth/session events. + connDetails eventpb.CommonConnectionDetails + // auth is the current HBA configuration as returned by // (*Server).GetAuthenticationConfiguration(). auth *hba.Conf @@ -87,17 +92,18 @@ func (c *conn) handleAuthentication( ctx, authOpt.ie, c.sessionArgs.User, ) if err != nil { - ac.Logf(ctx, "user retrieval failed for user=%q: %v", c.sessionArgs.User, err) + log.Warningf(ctx, "user retrieval failed for user=%q: %+v", c.sessionArgs.User, err) + ac.LogAuthFailed(ctx, "user retrieval error", err) return nil, sendError(err) } if !exists { - ac.Logf(ctx, "user does not exist: %q", c.sessionArgs.User) + ac.LogAuthFailed(ctx, "user does not exist", nil) return nil, sendError(errors.Errorf(security.ErrPasswordUserAuthFailed, c.sessionArgs.User)) } if !canLogin { - ac.Logf(ctx, "%q does not have login privilege", c.sessionArgs.User) + ac.LogAuthFailed(ctx, "user does not have privilege to log in", nil) return nil, sendError(errors.Errorf( "%s does not have login privilege", c.sessionArgs.User)) } @@ -105,26 +111,28 @@ func (c *conn) handleAuthentication( // Retrieve the authentication method. tlsState, hbaEntry, methodFn, err := c.findAuthenticationMethod(authOpt) if err != nil { - ac.Logf(ctx, "auth method lookup failed: %v", err) + ac.LogAuthFailed(ctx, "authentication method lookup error", err) return nil, sendError(err) } - ac.Logf(ctx, "connection matches HBA rule: %s", hbaEntry.Input) + + ac.SetAuthMethod(hbaEntry.Method.String()) + ac.LogAuthInfof(ctx, "HBA rule: %s", hbaEntry.Input) // Ask the method to authenticate. authenticationHook, err := methodFn(ctx, ac, tlsState, pwRetrievalFn, validUntilFn, execCfg, hbaEntry) if err != nil { - ac.Logf(ctx, "authentication pre-hook failed: %v", err) + ac.LogAuthFailed(ctx, "authentication pre-hook failed", err) return nil, sendError(err) } + if connClose, err = authenticationHook(c.sessionArgs.User, true /* public */); err != nil { - ac.Logf(ctx, "authentication failed: %v", err) + ac.LogAuthFailed(ctx, "authentication failed", err) return connClose, sendError(err) } - ac.Logf(ctx, "authentication succeeded") - + ac.LogAuthOK(ctx) c.msgBuilder.initMsg(pgwirebase.ServerMsgAuth) c.msgBuilder.putInt32(authOK) return connClose, c.msgBuilder.finishMsg(c.conn) @@ -247,15 +255,23 @@ type AuthConn interface { // AuthOK declares that authentication succeeded and provides a // unqualifiedIntSizer, to be returned by authenticator.authResult(). Future // authenticator.sendPwdData() calls fail. - AuthOK(unqualifiedIntSizer) + AuthOK(context.Context, unqualifiedIntSizer) // AuthFail declares that authentication has failed and provides an error to // be returned by authenticator.authResult(). Future // authenticator.sendPwdData() calls fail. The error has already been written // to the client connection. AuthFail(err error) - // Logf logs a message on the authentication log, if auth logs - // are enabled. - Logf(ctx context.Context, format string, args ...interface{}) + + // SetAuthMethod sets the authentication method for subsequent + // logging messages. + SetAuthMethod(method string) + // LogAuthInfof logs details about the progress of the + // authentication. + LogAuthInfof(ctx context.Context, format string, args ...interface{}) + // LogAuthFailed logs details about an authentication failure. + LogAuthFailed(ctx context.Context, reason string, err error) + // LogAuthOK logs when the authentication handshake has completed. + LogAuthOK(ctx context.Context) } // authPipe is the implementation for the authenticator and AuthConn interfaces. @@ -265,6 +281,10 @@ type authPipe struct { c *conn // Only used for writing, not for reading. log bool + connDetails eventpb.CommonConnectionDetails + authDetails eventpb.CommonSessionDetails + authMethod string + ch chan []byte // writerDone is a channel closed by noMorePwdData(). // Nil if noMorePwdData(). @@ -277,10 +297,15 @@ type authRes struct { err error } -func newAuthPipe(c *conn, logAuthn bool) *authPipe { +func newAuthPipe(c *conn, logAuthn bool, authOpt authOptions, user security.SQLUsername) *authPipe { ap := &authPipe{ - c: c, - log: logAuthn, + c: c, + log: logAuthn, + connDetails: authOpt.connDetails, + authDetails: eventpb.CommonSessionDetails{ + Transport: authOpt.connType.String(), + User: user.Normalized(), + }, ch: make(chan []byte), writerDone: make(chan struct{}), readerDone: make(chan authRes, 1), @@ -320,7 +345,7 @@ func (p *authPipe) GetPwdData() ([]byte, error) { } // AuthOK is part of the AuthConn interface. -func (p *authPipe) AuthOK(intSizer unqualifiedIntSizer) { +func (p *authPipe) AuthOK(ctx context.Context, intSizer unqualifiedIntSizer) { p.readerDone <- authRes{intSizer: intSizer} } @@ -328,9 +353,47 @@ func (p *authPipe) AuthFail(err error) { p.readerDone <- authRes{err: err} } -func (p *authPipe) Logf(ctx context.Context, format string, args ...interface{}) { +func (p *authPipe) SetAuthMethod(method string) { + p.authMethod = method +} + +func (p *authPipe) LogAuthOK(ctx context.Context) { + if p.log { + ev := &eventpb.ClientAuthenticationOk{ + CommonConnectionDetails: p.connDetails, + CommonSessionDetails: p.authDetails, + Method: p.authMethod, + } + log.StructuredEvent(ctx, ev) + } +} + +func (p *authPipe) LogAuthInfof(ctx context.Context, format string, args ...interface{}) { + if p.log { + ev := &eventpb.ClientAuthenticationInfo{ + CommonConnectionDetails: p.connDetails, + CommonSessionDetails: p.authDetails, + Info: fmt.Sprintf(format, args...), + Method: p.authMethod, + } + log.StructuredEvent(ctx, ev) + } +} + +func (p *authPipe) LogAuthFailed(ctx context.Context, reason string, detailedErr error) { if p.log { - log.Sessions.Infof(ctx, format, args...) + var errStr string + if detailedErr != nil { + errStr = detailedErr.Error() + } + ev := &eventpb.ClientAuthenticationFailed{ + CommonConnectionDetails: p.connDetails, + CommonSessionDetails: p.authDetails, + Reason: reason, + Detail: errStr, + Method: p.authMethod, + } + log.StructuredEvent(ctx, ev) } } diff --git a/pkg/sql/pgwire/auth_methods.go b/pkg/sql/pgwire/auth_methods.go index 6cbc28f95f8d..bf3a658c9d64 100644 --- a/pkg/sql/pgwire/auth_methods.go +++ b/pkg/sql/pgwire/auth_methods.go @@ -106,7 +106,7 @@ func authPassword( return nil, err } if len(hashedPassword) == 0 { - c.Logf(ctx, "user has no password defined") + c.LogAuthInfof(ctx, "user has no password defined") } validUntil, err := pwValidUntilFn(ctx) @@ -115,7 +115,7 @@ func authPassword( } if validUntil != nil { if validUntil.Sub(timeutil.Now()) < 0 { - c.Logf(ctx, "password is expired") + c.LogAuthFailed(ctx, "password is expired", nil) return nil, errors.New("password is expired") } } @@ -163,10 +163,10 @@ func authCertPassword( ) (security.UserAuthHook, error) { var fn AuthMethod if len(tlsState.PeerCertificates) == 0 { - c.Logf(ctx, "no client certificate, proceeding with password authentication") + c.LogAuthInfof(ctx, "no client certificate, proceeding with password authentication") fn = authPassword } else { - c.Logf(ctx, "client presented certificate, proceeding with certificate validation") + c.LogAuthInfof(ctx, "client presented certificate, proceeding with certificate validation") fn = authCert } return fn(ctx, c, tlsState, pwRetrieveFn, pwValidUntilFn, execCfg, entry) diff --git a/pkg/sql/pgwire/auth_test.go b/pkg/sql/pgwire/auth_test.go index 9f4b6c1990f6..d304f734b5b2 100644 --- a/pkg/sql/pgwire/auth_test.go +++ b/pkg/sql/pgwire/auth_test.go @@ -13,6 +13,7 @@ package pgwire_test import ( "context" gosql "database/sql" + "encoding/json" "fmt" "io/ioutil" "math" @@ -42,6 +43,7 @@ import ( "github.com/cockroachdb/datadriven" "github.com/cockroachdb/errors" "github.com/cockroachdb/errors/stdstrings" + "github.com/cockroachdb/redact" "github.com/lib/pq" ) @@ -308,17 +310,32 @@ func hbaRunTest(t *testing.T, insecure bool) { entry := &entries[i] // t.Logf("found log entry: %+v", *entry) - // The tag part is going to contain a client address, with a random port number. - // To make the test deterministic, erase the random part. - tags := addrRe.ReplaceAllString(entry.Tags, ",client=XXX") - var maybeTags string - if len(tags) > 0 { - maybeTags = "[" + tags + "] " + parts := strings.SplitN(entry.Message, " ", 4) + if len(parts) < 4 || parts[1] != "Structured" || parts[2] != "entry:" { + // TODO(knz): Enhance this when the log file + // contains proper markers for structured entries. + t.Errorf("malformed structured message: %q", entry.Message) } - // Ditto with the duration. - msg := durationRe.ReplaceAllString(entry.Message, "duration: XXX") - fmt.Fprintf(&buf, "%c: %s%s\n", entry.Severity.String()[0], maybeTags, msg) + jsonPayload := []byte(parts[3]) + if entry.Redactable { + jsonPayload = redact.RedactableBytes(jsonPayload).StripMarkers() + } + var info map[string]interface{} + if err := json.Unmarshal(jsonPayload, &info); err != nil { + t.Fatalf("unable to decode json: %q: %v", jsonPayload, err) + } + // Erase non-deterministic fields. + info["Timestamp"] = "XXX" + info["RemoteAddress"] = "XXX" + if _, ok := info["Duration"]; ok { + info["Duration"] = "NNN" + } + msg, err := json.Marshal(info) + if err != nil { + t.Fatal(err) + } + fmt.Fprintf(&buf, "%s %s\n", parts[0], msg) } lastLogMsg := entries[0].Message if !re.MatchString(lastLogMsg) { @@ -435,9 +452,7 @@ func hbaRunTest(t *testing.T, insecure bool) { }) } -var authLogFileRe = regexp.MustCompile(`pgwire/(auth|conn|server)\.go`) -var addrRe = regexp.MustCompile(`,client(=[^\],]*)?`) -var durationRe = regexp.MustCompile(`duration: \d.*s`) +var authLogFileRe = regexp.MustCompile(`"EventType":"client_`) // fmtErr formats an error into an expected output. func fmtErr(err error) string { diff --git a/pkg/sql/pgwire/conn.go b/pkg/sql/pgwire/conn.go index cc2c09319d3a..3175cb7a7974 100644 --- a/pkg/sql/pgwire/conn.go +++ b/pkg/sql/pgwire/conn.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" "github.com/cockroachdb/cockroach/pkg/sql/types" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" @@ -223,7 +224,13 @@ func (c *conn) serveImpl( sessionStart := timeutil.Now() defer func() { if c.authLogEnabled() { - log.Sessions.Infof(ctx, "session terminated; duration: %s", timeutil.Now().Sub(sessionStart)) + endTime := timeutil.Now() + ev := &eventpb.ClientSessionEnd{ + CommonEventDetails: eventpb.CommonEventDetails{Timestamp: endTime.UnixNano()}, + CommonConnectionDetails: authOpt.connDetails, + Duration: endTime.Sub(sessionStart).Nanoseconds(), + } + log.StructuredEvent(ctx, ev) } }() } @@ -259,7 +266,7 @@ func (c *conn) serveImpl( logAuthn := !inTestWithoutSQL && c.authLogEnabled() // We'll build an authPipe to communicate with the authentication process. - authPipe := newAuthPipe(c, logAuthn) + authPipe := newAuthPipe(c, logAuthn, authOpt, c.sessionArgs.User) var authenticator authenticatorIO = authPipe // procCh is the channel on which we'll receive the termination signal from @@ -287,7 +294,7 @@ func (c *conn) serveImpl( } var ac AuthConn = authPipe // Simulate auth succeeding. - ac.AuthOK(fixedIntSizer{size: types.Int}) + ac.AuthOK(ctx, fixedIntSizer{size: types.Int}) dummyCh := make(chan error) close(dummyCh) procCh = dummyCh @@ -609,7 +616,7 @@ func (c *conn) processCommandsAsync( return } // Signal the connection was established to the authenticator. - ac.AuthOK(connHandler) + ac.AuthOK(ctx, connHandler) // Mark the authentication as succeeded in case a panic // is thrown below and we need to report to the client // using the defer above. diff --git a/pkg/sql/pgwire/conn_test.go b/pkg/sql/pgwire/conn_test.go index 056ac846ae06..d831cd4eb811 100644 --- a/pkg/sql/pgwire/conn_test.go +++ b/pkg/sql/pgwire/conn_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/hba" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase" @@ -111,7 +112,7 @@ func TestConn(t *testing.T) { // sqlServer - nil means don't create a command processor and a write side of the conn nil, mon.BoundAccount{}, /* reserved */ - authOptions{testingSkipAuth: true}, + authOptions{testingSkipAuth: true, connType: hba.ConnHostAny}, ) return nil }) @@ -1056,7 +1057,7 @@ func TestMaliciousInputs(t *testing.T) { func() bool { return false }, /* draining */ nil, /* sqlServer */ mon.BoundAccount{}, /* reserved */ - authOptions{testingSkipAuth: true}, + authOptions{testingSkipAuth: true, connType: hba.ConnHostAny}, ) if err := <-errChan; err != nil { t.Fatal(err) diff --git a/pkg/sql/pgwire/hba/hba.go b/pkg/sql/pgwire/hba/hba.go index c56d55009705..f802b6f1ba04 100644 --- a/pkg/sql/pgwire/hba/hba.go +++ b/pkg/sql/pgwire/hba/hba.go @@ -91,7 +91,7 @@ func (t ConnType) String() string { case ConnHostAny: return "host" default: - panic("unimplemented") + panic(errors.Newf("unimplemented conn type: %v", int(t))) } } diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go index b3be8a3c3fb8..461fd28db6a6 100644 --- a/pkg/sql/pgwire/server.go +++ b/pkg/sql/pgwire/server.go @@ -35,6 +35,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/envutil" "github.com/cockroachdb/cockroach/pkg/util/humanizeutil" "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/log/eventpb" "github.com/cockroachdb/cockroach/pkg/util/metric" "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -490,11 +491,21 @@ func (s *Server) ServeConn(ctx context.Context, conn net.Conn, socketType Socket ctx, draining, onCloseFn := s.registerConn(ctx) defer onCloseFn() + connDetails := eventpb.CommonConnectionDetails{ + InstanceID: int32(s.execCfg.NodeID.SQLInstanceID()), + Network: conn.RemoteAddr().Network(), + RemoteAddress: conn.RemoteAddr().String(), + } + // Some bookkeeping, for security-minded administrators. // This registers the connection to the authentication log. connStart := timeutil.Now() if s.connLogEnabled() { - log.Sessions.Infof(ctx, "received connection") + ev := &eventpb.ClientConnectionStart{ + CommonEventDetails: eventpb.CommonEventDetails{Timestamp: connStart.UnixNano()}, + CommonConnectionDetails: connDetails, + } + log.StructuredEvent(ctx, ev) } defer func() { // The duration of the session is logged at the end so that the @@ -502,7 +513,13 @@ func (s *Server) ServeConn(ctx context.Context, conn net.Conn, socketType Socket // to find when the connection was opened. This is important // because the log files may have been rotated since. if s.connLogEnabled() { - log.Sessions.Infof(ctx, "disconnected; duration: %s", timeutil.Now().Sub(connStart)) + endTime := timeutil.Now() + ev := &eventpb.ClientConnectionEnd{ + CommonEventDetails: eventpb.CommonEventDetails{Timestamp: endTime.UnixNano()}, + CommonConnectionDetails: connDetails, + Duration: endTime.Sub(connStart).Nanoseconds(), + } + log.StructuredEvent(ctx, ev) } }() @@ -591,6 +608,7 @@ func (s *Server) ServeConn(ctx context.Context, conn net.Conn, socketType Socket reserved, authOptions{ connType: connType, + connDetails: connDetails, insecure: s.cfg.Insecure, ie: s.execCfg.InternalExecutor, auth: s.GetAuthenticationConfiguration(), diff --git a/pkg/sql/pgwire/testdata/auth/conn_log b/pkg/sql/pgwire/testdata/auth/conn_log index 3353a0419f39..de6a64781e3d 100644 --- a/pkg/sql/pgwire/testdata/auth/conn_log +++ b/pkg/sql/pgwire/testdata/auth/conn_log @@ -44,42 +44,42 @@ ok defaultdb authlog 6 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 5 received connection -I: [n1,client=XXX,hostssl,user=root] 6 connection matches HBA rule: ‹host all root all cert-password # CockroachDB mandatory rule› -I: [n1,client=XXX,hostssl,user=root] 7 client presented certificate, proceeding with certificate validation -I: [n1,client=XXX,hostssl,user=root] 8 authentication succeeded -I: [n1,client=XXX,hostssl,user=root] 9 session terminated; duration: XXX -I: [n1,client=XXX,hostssl] 10 disconnected; duration: XXX +5 {"EventType":"client_connection_start","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +6 {"EventType":"client_authentication_info","Info":"HBA rule: host all root all cert-password # CockroachDB mandatory rule","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"root"} +7 {"EventType":"client_authentication_info","Info":"client presented certificate, proceeding with certificate validation","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"root"} +8 {"EventType":"client_authentication_ok","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"root"} +9 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +10 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} connect user=root password=secureabc sslmode=require sslcert= sslkey= ---- ok defaultdb authlog 6 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 11 received connection -I: [n1,client=XXX,hostssl,user=root] 12 connection matches HBA rule: ‹host all root all cert-password # CockroachDB mandatory rule› -I: [n1,client=XXX,hostssl,user=root] 13 no client certificate, proceeding with password authentication -I: [n1,client=XXX,hostssl,user=root] 14 authentication succeeded -I: [n1,client=XXX,hostssl,user=root] 15 session terminated; duration: XXX -I: [n1,client=XXX,hostssl] 16 disconnected; duration: XXX +11 {"EventType":"client_connection_start","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +12 {"EventType":"client_authentication_info","Info":"HBA rule: host all root all cert-password # CockroachDB mandatory rule","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"root"} +13 {"EventType":"client_authentication_info","Info":"no client certificate, proceeding with password authentication","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"root"} +14 {"EventType":"client_authentication_ok","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"root"} +15 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +16 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} connect user=root password=badpass sslmode=require sslcert= sslkey= ---- ERROR: password authentication failed for user root authlog 6 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 17 received connection -I: [n1,client=XXX,hostssl,user=root] 18 connection matches HBA rule: ‹host all root all cert-password # CockroachDB mandatory rule› -I: [n1,client=XXX,hostssl,user=root] 19 no client certificate, proceeding with password authentication -I: [n1,client=XXX,hostssl,user=root] 20 authentication failed: password authentication failed for user ‹root› -I: [n1,client=XXX,hostssl,user=root] 21 session terminated; duration: XXX -I: [n1,client=XXX,hostssl] 22 disconnected; duration: XXX +17 {"EventType":"client_connection_start","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +18 {"EventType":"client_authentication_info","Info":"HBA rule: host all root all cert-password # CockroachDB mandatory rule","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"root"} +19 {"EventType":"client_authentication_info","Info":"no client certificate, proceeding with password authentication","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"root"} +20 {"Detail":"password authentication failed for user root","EventType":"client_authentication_failed","InstanceID":1,"Method":"cert-password","Network":"tcp","Reason":"authentication failed","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"root"} +21 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +22 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} subtest end @@ -91,13 +91,13 @@ connect user=trusted ok defaultdb authlog 5 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 23 received connection -I: [n1,client=XXX,hostssl,user=‹trusted›] 24 connection matches HBA rule: ‹host all trusted all trust # custom› -I: [n1,client=XXX,hostssl,user=‹trusted›] 25 authentication succeeded -I: [n1,client=XXX,hostssl,user=‹trusted›] 26 session terminated; duration: XXX -I: [n1,client=XXX,hostssl] 27 disconnected; duration: XXX +23 {"EventType":"client_connection_start","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +24 {"EventType":"client_authentication_info","Info":"HBA rule: host all trusted all trust # custom","InstanceID":1,"Method":"trust","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"trusted"} +25 {"EventType":"client_authentication_ok","InstanceID":1,"Method":"trust","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"trusted"} +26 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +27 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} subtest end @@ -108,28 +108,28 @@ connect user=userpw password=pass ok defaultdb authlog 6 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 28 received connection -I: [n1,client=XXX,hostssl,user=‹userpw›] 29 connection matches HBA rule: ‹host all all all cert-password # built-in CockroachDB default› -I: [n1,client=XXX,hostssl,user=‹userpw›] 30 no client certificate, proceeding with password authentication -I: [n1,client=XXX,hostssl,user=‹userpw›] 31 authentication succeeded -I: [n1,client=XXX,hostssl,user=‹userpw›] 32 session terminated; duration: XXX -I: [n1,client=XXX,hostssl] 33 disconnected; duration: XXX +28 {"EventType":"client_connection_start","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +29 {"EventType":"client_authentication_info","Info":"HBA rule: host all all all cert-password # built-in CockroachDB default","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"userpw"} +30 {"EventType":"client_authentication_info","Info":"no client certificate, proceeding with password authentication","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"userpw"} +31 {"EventType":"client_authentication_ok","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"userpw"} +32 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +33 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} connect user=userpw password=badpass ---- ERROR: password authentication failed for user userpw authlog 6 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 34 received connection -I: [n1,client=XXX,hostssl,user=‹userpw›] 35 connection matches HBA rule: ‹host all all all cert-password # built-in CockroachDB default› -I: [n1,client=XXX,hostssl,user=‹userpw›] 36 no client certificate, proceeding with password authentication -I: [n1,client=XXX,hostssl,user=‹userpw›] 37 authentication failed: password authentication failed for user ‹userpw› -I: [n1,client=XXX,hostssl,user=‹userpw›] 38 session terminated; duration: XXX -I: [n1,client=XXX,hostssl] 39 disconnected; duration: XXX +34 {"EventType":"client_connection_start","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +35 {"EventType":"client_authentication_info","Info":"HBA rule: host all all all cert-password # built-in CockroachDB default","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"userpw"} +36 {"EventType":"client_authentication_info","Info":"no client certificate, proceeding with password authentication","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"userpw"} +37 {"Detail":"password authentication failed for user userpw","EventType":"client_authentication_failed","InstanceID":1,"Method":"cert-password","Network":"tcp","Reason":"authentication failed","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"userpw"} +38 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +39 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} subtest end @@ -140,15 +140,15 @@ connect user=usernopw ERROR: password authentication failed for user usernopw authlog 7 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 40 received connection -I: [n1,client=XXX,hostssl,user=‹usernopw›] 41 connection matches HBA rule: ‹host all all all cert-password # built-in CockroachDB default› -I: [n1,client=XXX,hostssl,user=‹usernopw›] 42 no client certificate, proceeding with password authentication -I: [n1,client=XXX,hostssl,user=‹usernopw›] 43 user has no password defined -I: [n1,client=XXX,hostssl,user=‹usernopw›] 44 authentication failed: password authentication failed for user ‹usernopw› -I: [n1,client=XXX,hostssl,user=‹usernopw›] 45 session terminated; duration: XXX -I: [n1,client=XXX,hostssl] 46 disconnected; duration: XXX +40 {"EventType":"client_connection_start","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +41 {"EventType":"client_authentication_info","Info":"HBA rule: host all all all cert-password # built-in CockroachDB default","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"usernopw"} +42 {"EventType":"client_authentication_info","Info":"no client certificate, proceeding with password authentication","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"usernopw"} +43 {"EventType":"client_authentication_info","Info":"user has no password defined","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"usernopw"} +44 {"Detail":"password authentication failed for user usernopw","EventType":"client_authentication_failed","InstanceID":1,"Method":"cert-password","Network":"tcp","Reason":"authentication failed","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"hostssl","User":"usernopw"} +45 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} +46 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"} subtest end @@ -165,26 +165,26 @@ connect_unix user=root password=secureabc ok defaultdb authlog 5 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 47 received connection -I: [n1,client=XXX,local,user=root] 48 connection matches HBA rule: ‹local all all password # built-in CockroachDB default› -I: [n1,client=XXX,local,user=root] 49 authentication succeeded -I: [n1,client=XXX,local,user=root] 50 session terminated; duration: XXX -I: [n1,client=XXX,local] 51 disconnected; duration: XXX +47 {"EventType":"client_connection_start","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +48 {"EventType":"client_authentication_info","Info":"HBA rule: local all all password # built-in CockroachDB default","InstanceID":1,"Method":"password","Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"root"} +49 {"EventType":"client_authentication_ok","InstanceID":1,"Method":"password","Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"root"} +50 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +51 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} connect_unix user=root password=badpass ---- ERROR: password authentication failed for user root authlog 5 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 52 received connection -I: [n1,client=XXX,local,user=root] 53 connection matches HBA rule: ‹local all all password # built-in CockroachDB default› -I: [n1,client=XXX,local,user=root] 54 authentication failed: password authentication failed for user ‹root› -I: [n1,client=XXX,local,user=root] 55 session terminated; duration: XXX -I: [n1,client=XXX,local] 56 disconnected; duration: XXX +52 {"EventType":"client_connection_start","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +53 {"EventType":"client_authentication_info","Info":"HBA rule: local all all password # built-in CockroachDB default","InstanceID":1,"Method":"password","Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"root"} +54 {"Detail":"password authentication failed for user root","EventType":"client_authentication_failed","InstanceID":1,"Method":"password","Network":"unix","Reason":"authentication failed","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"root"} +55 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +56 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} subtest end @@ -196,13 +196,13 @@ connect_unix user=trusted ERROR: authentication rejected by configuration authlog 5 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 57 received connection -I: [n1,client=XXX,local,user=‹trusted›] 58 connection matches HBA rule: ‹local all trusted reject # custom› -I: [n1,client=XXX,local,user=‹trusted›] 59 authentication failed: authentication rejected by configuration -I: [n1,client=XXX,local,user=‹trusted›] 60 session terminated; duration: XXX -I: [n1,client=XXX,local] 61 disconnected; duration: XXX +57 {"EventType":"client_connection_start","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +58 {"EventType":"client_authentication_info","Info":"HBA rule: local all trusted reject # custom","InstanceID":1,"Method":"reject","Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"trusted"} +59 {"Detail":"authentication rejected by configuration","EventType":"client_authentication_failed","InstanceID":1,"Method":"reject","Network":"unix","Reason":"authentication failed","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"trusted"} +60 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +61 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} subtest end @@ -213,26 +213,26 @@ connect_unix user=userpw password=pass ok defaultdb authlog 5 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 62 received connection -I: [n1,client=XXX,local,user=‹userpw›] 63 connection matches HBA rule: ‹local all all password # built-in CockroachDB default› -I: [n1,client=XXX,local,user=‹userpw›] 64 authentication succeeded -I: [n1,client=XXX,local,user=‹userpw›] 65 session terminated; duration: XXX -I: [n1,client=XXX,local] 66 disconnected; duration: XXX +62 {"EventType":"client_connection_start","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +63 {"EventType":"client_authentication_info","Info":"HBA rule: local all all password # built-in CockroachDB default","InstanceID":1,"Method":"password","Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"userpw"} +64 {"EventType":"client_authentication_ok","InstanceID":1,"Method":"password","Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"userpw"} +65 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +66 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} connect_unix user=userpw password=badpass ---- ERROR: password authentication failed for user userpw authlog 5 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 67 received connection -I: [n1,client=XXX,local,user=‹userpw›] 68 connection matches HBA rule: ‹local all all password # built-in CockroachDB default› -I: [n1,client=XXX,local,user=‹userpw›] 69 authentication failed: password authentication failed for user ‹userpw› -I: [n1,client=XXX,local,user=‹userpw›] 70 session terminated; duration: XXX -I: [n1,client=XXX,local] 71 disconnected; duration: XXX +67 {"EventType":"client_connection_start","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +68 {"EventType":"client_authentication_info","Info":"HBA rule: local all all password # built-in CockroachDB default","InstanceID":1,"Method":"password","Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"userpw"} +69 {"Detail":"password authentication failed for user userpw","EventType":"client_authentication_failed","InstanceID":1,"Method":"password","Network":"unix","Reason":"authentication failed","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"userpw"} +70 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +71 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} subtest end @@ -243,12 +243,12 @@ connect_unix user=usernologin password=123 ERROR: usernologin does not have login privilege authlog 4 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 72 received connection -I: [n1,client=XXX,local,user=‹usernologin›] 73 ‹"usernologin"› does not have login privilege -I: [n1,client=XXX,local,user=‹usernologin›] 74 session terminated; duration: XXX -I: [n1,client=XXX,local] 75 disconnected; duration: XXX +72 {"EventType":"client_connection_start","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +73 {"EventType":"client_authentication_failed","InstanceID":1,"Network":"unix","Reason":"user does not have privilege to log in","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"usernologin"} +74 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +75 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} connect_unix user=userexpired password=123 @@ -256,14 +256,14 @@ connect_unix user=userexpired password=123 ERROR: password is expired authlog 6 -.*disconnected +.*client_connection_end ---- -I: [n1,client=XXX] 76 received connection -I: [n1,client=XXX,local,user=‹userexpired›] 77 connection matches HBA rule: ‹local all all password # built-in CockroachDB default› -I: [n1,client=XXX,local,user=‹userexpired›] 78 password is expired -I: [n1,client=XXX,local,user=‹userexpired›] 79 authentication pre-hook failed: password is expired -I: [n1,client=XXX,local,user=‹userexpired›] 80 session terminated; duration: XXX -I: [n1,client=XXX,local] 81 disconnected; duration: XXX +76 {"EventType":"client_connection_start","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +77 {"EventType":"client_authentication_info","Info":"HBA rule: local all all password # built-in CockroachDB default","InstanceID":1,"Method":"password","Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"userexpired"} +78 {"EventType":"client_authentication_failed","InstanceID":1,"Method":"password","Network":"unix","Reason":"password is expired","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"userexpired"} +79 {"Detail":"password is expired","EventType":"client_authentication_failed","InstanceID":1,"Method":"password","Network":"unix","Reason":"authentication pre-hook failed","RemoteAddress":"XXX","Timestamp":"XXX","Transport":"local","User":"userexpired"} +80 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} +81 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"unix","RemoteAddress":"XXX","Timestamp":"XXX"} subtest end diff --git a/pkg/util/log/eventpb/BUILD.bazel b/pkg/util/log/eventpb/BUILD.bazel index 9e69b9d87fa8..888d6b22a00f 100644 --- a/pkg/util/log/eventpb/BUILD.bazel +++ b/pkg/util/log/eventpb/BUILD.bazel @@ -13,6 +13,7 @@ go_library( "misc_sql_events.pb.go", "privilege_events.pb.go", "role_events.pb.go", + "session_events.pb.go", "zone_events.pb.go", ], importpath = "github.com/cockroachdb/cockroach/pkg/util/log/eventpb", diff --git a/pkg/util/log/eventpb/eventlog_channels_generated.go b/pkg/util/log/eventpb/eventlog_channels_generated.go index d7eefa82e236..8da43d14e5f1 100644 --- a/pkg/util/log/eventpb/eventlog_channels_generated.go +++ b/pkg/util/log/eventpb/eventlog_channels_generated.go @@ -154,6 +154,24 @@ func (m *ChangeTablePrivilege) LoggingChannel() logpb.Channel { return logpb.Cha // LoggingChannel implements the EventPayload interface. func (m *ChangeTypePrivilege) LoggingChannel() logpb.Channel { return logpb.Channel_PRIVILEGES } +// LoggingChannel implements the EventPayload interface. +func (m *ClientAuthenticationFailed) LoggingChannel() logpb.Channel { return logpb.Channel_SESSIONS } + +// LoggingChannel implements the EventPayload interface. +func (m *ClientAuthenticationInfo) LoggingChannel() logpb.Channel { return logpb.Channel_SESSIONS } + +// LoggingChannel implements the EventPayload interface. +func (m *ClientAuthenticationOk) LoggingChannel() logpb.Channel { return logpb.Channel_SESSIONS } + +// LoggingChannel implements the EventPayload interface. +func (m *ClientConnectionEnd) LoggingChannel() logpb.Channel { return logpb.Channel_SESSIONS } + +// LoggingChannel implements the EventPayload interface. +func (m *ClientConnectionStart) LoggingChannel() logpb.Channel { return logpb.Channel_SESSIONS } + +// LoggingChannel implements the EventPayload interface. +func (m *ClientSessionEnd) LoggingChannel() logpb.Channel { return logpb.Channel_SESSIONS } + // LoggingChannel implements the EventPayload interface. func (m *AlterRole) LoggingChannel() logpb.Channel { return logpb.Channel_USER_ADMIN } diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index 0dfd5d410d41..f607b638c8dc 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -417,6 +417,164 @@ func (m *ChangeTypePrivilege) AppendJSONFields(printComma bool, b redact.Redacta return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *ClientAuthenticationFailed) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonConnectionDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSessionDetails.AppendJSONFields(printComma, b) + + if m.Reason != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Reason\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.Reason))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.Detail != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Detail\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.Detail))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.Method != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Method\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.Method))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + return printComma, b +} + +// AppendJSONFields implements the EventPayload interface. +func (m *ClientAuthenticationInfo) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonConnectionDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSessionDetails.AppendJSONFields(printComma, b) + + if m.Method != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Method\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.Method))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.Info != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Info\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.Info))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + return printComma, b +} + +// AppendJSONFields implements the EventPayload interface. +func (m *ClientAuthenticationOk) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonConnectionDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSessionDetails.AppendJSONFields(printComma, b) + + if m.Method != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Method\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.Method))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + return printComma, b +} + +// AppendJSONFields implements the EventPayload interface. +func (m *ClientConnectionEnd) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonConnectionDetails.AppendJSONFields(printComma, b) + + if m.Duration != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Duration\":"...) + b = strconv.AppendInt(b, int64(m.Duration), 10) + } + + return printComma, b +} + +// AppendJSONFields implements the EventPayload interface. +func (m *ClientConnectionStart) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonConnectionDetails.AppendJSONFields(printComma, b) + + return printComma, b +} + +// AppendJSONFields implements the EventPayload interface. +func (m *ClientSessionEnd) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + printComma, b = m.CommonEventDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonConnectionDetails.AppendJSONFields(printComma, b) + + printComma, b = m.CommonSessionDetails.AppendJSONFields(printComma, b) + + if m.Duration != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Duration\":"...) + b = strconv.AppendInt(b, int64(m.Duration), 10) + } + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *CommentOnColumn) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { @@ -609,6 +767,45 @@ func (m *CommentOnTable) AppendJSONFields(printComma bool, b redact.RedactableBy return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *CommonConnectionDetails) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + if m.InstanceID != 0 { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"InstanceID\":"...) + b = strconv.AppendInt(b, int64(m.InstanceID), 10) + } + + if m.Network != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Network\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.Network))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.RemoteAddress != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"RemoteAddress\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.RemoteAddress))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *CommonEventDetails) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { @@ -827,6 +1024,36 @@ func (m *CommonSchemaChangeEventDetails) AppendJSONFields(printComma bool, b red return printComma, b } +// AppendJSONFields implements the EventPayload interface. +func (m *CommonSessionDetails) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { + + if m.Transport != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"Transport\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.Transport))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + if m.User != "" { + if printComma { + b = append(b, ',') + } + printComma = true + b = append(b, "\"User\":\""...) + b = append(b, redact.StartMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.User))))) + b = append(b, redact.EndMarker()...) + b = append(b, '"') + } + + return printComma, b +} + // AppendJSONFields implements the EventPayload interface. func (m *CommonZoneConfigDetails) AppendJSONFields(printComma bool, b redact.RedactableBytes) (bool, redact.RedactableBytes) { diff --git a/pkg/util/log/eventpb/session_events.pb.go b/pkg/util/log/eventpb/session_events.pb.go new file mode 100644 index 000000000000..88af12efcfbd --- /dev/null +++ b/pkg/util/log/eventpb/session_events.pb.go @@ -0,0 +1,2272 @@ +// Code generated by protoc-gen-gogo. DO NOT EDIT. +// source: util/log/eventpb/session_events.proto + +package eventpb + +import proto "github.com/gogo/protobuf/proto" +import fmt "fmt" +import math "math" + +import io "io" + +// Reference imports to suppress errors if they are not otherwise used. +var _ = proto.Marshal +var _ = fmt.Errorf +var _ = math.Inf + +// This is a compile-time assertion to ensure that this generated file +// is compatible with the proto package it is being compiled against. +// A compilation error at this line likely means your copy of the +// proto package needs to be updated. +const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package + +// CommonConnectionDetails are payload fields common to all +// connection/session events. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_connections.enabled` is set. +type CommonConnectionDetails struct { + // The instance ID (not tenant ID) of the SQL server where the event was originated. + InstanceID int32 `protobuf:"varint,1,opt,name=instance_id,json=instanceId,proto3" json:",omitempty"` + // The network protocol for this connection: tcp4, tcp6, unix, etc. + Network string `protobuf:"bytes,2,opt,name=network,proto3" json:",omitempty"` + // The remote address of the SQL client. Note that when using a + // proxy or other intermediate server, this field will contain the + // address of the intermediate server. + RemoteAddress string `protobuf:"bytes,3,opt,name=remote_address,json=remoteAddress,proto3" json:",omitempty"` +} + +func (m *CommonConnectionDetails) Reset() { *m = CommonConnectionDetails{} } +func (m *CommonConnectionDetails) String() string { return proto.CompactTextString(m) } +func (*CommonConnectionDetails) ProtoMessage() {} +func (*CommonConnectionDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_session_events_eb3571f04b8bd5df, []int{0} +} +func (m *CommonConnectionDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CommonConnectionDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CommonConnectionDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_CommonConnectionDetails.Merge(dst, src) +} +func (m *CommonConnectionDetails) XXX_Size() int { + return m.Size() +} +func (m *CommonConnectionDetails) XXX_DiscardUnknown() { + xxx_messageInfo_CommonConnectionDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_CommonConnectionDetails proto.InternalMessageInfo + +// CommonConnectionDetails are payload fields common to all +// session events. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_connections.enabled` is set. +type CommonSessionDetails struct { + // The connection type after transport negotiation. + Transport string `protobuf:"bytes,1,opt,name=transport,proto3" json:",omitempty"` + // The username the session is for. This is the username passed by + // the client, after case-folding and Unicode normalization. + User string `protobuf:"bytes,2,opt,name=user,proto3" json:",omitempty"` +} + +func (m *CommonSessionDetails) Reset() { *m = CommonSessionDetails{} } +func (m *CommonSessionDetails) String() string { return proto.CompactTextString(m) } +func (*CommonSessionDetails) ProtoMessage() {} +func (*CommonSessionDetails) Descriptor() ([]byte, []int) { + return fileDescriptor_session_events_eb3571f04b8bd5df, []int{1} +} +func (m *CommonSessionDetails) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *CommonSessionDetails) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *CommonSessionDetails) XXX_Merge(src proto.Message) { + xxx_messageInfo_CommonSessionDetails.Merge(dst, src) +} +func (m *CommonSessionDetails) XXX_Size() int { + return m.Size() +} +func (m *CommonSessionDetails) XXX_DiscardUnknown() { + xxx_messageInfo_CommonSessionDetails.DiscardUnknown(m) +} + +var xxx_messageInfo_CommonSessionDetails proto.InternalMessageInfo + +// ClientConnectionStart is reported when a client connection +// is established. This is reported even when authentication +// fails, and even for simple cancellation messages. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_connections.enabled` is set. +type ClientConnectionStart struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonConnectionDetails `protobuf:"bytes,2,opt,name=conn,proto3,embedded=conn" json:""` +} + +func (m *ClientConnectionStart) Reset() { *m = ClientConnectionStart{} } +func (m *ClientConnectionStart) String() string { return proto.CompactTextString(m) } +func (*ClientConnectionStart) ProtoMessage() {} +func (*ClientConnectionStart) Descriptor() ([]byte, []int) { + return fileDescriptor_session_events_eb3571f04b8bd5df, []int{2} +} +func (m *ClientConnectionStart) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientConnectionStart) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClientConnectionStart) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientConnectionStart.Merge(dst, src) +} +func (m *ClientConnectionStart) XXX_Size() int { + return m.Size() +} +func (m *ClientConnectionStart) XXX_DiscardUnknown() { + xxx_messageInfo_ClientConnectionStart.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientConnectionStart proto.InternalMessageInfo + +// ClientConnectionEnd is reported when a client connection +// is closed. This is reported even when authentication +// fails, and even for simple cancellation messages. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_connections.enabled` is set. +type ClientConnectionEnd struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonConnectionDetails `protobuf:"bytes,2,opt,name=conn,proto3,embedded=conn" json:""` + // The duration of the connection in nanoseconds. + Duration int64 `protobuf:"varint,3,opt,name=duration,proto3" json:",omitempty"` +} + +func (m *ClientConnectionEnd) Reset() { *m = ClientConnectionEnd{} } +func (m *ClientConnectionEnd) String() string { return proto.CompactTextString(m) } +func (*ClientConnectionEnd) ProtoMessage() {} +func (*ClientConnectionEnd) Descriptor() ([]byte, []int) { + return fileDescriptor_session_events_eb3571f04b8bd5df, []int{3} +} +func (m *ClientConnectionEnd) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientConnectionEnd) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClientConnectionEnd) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientConnectionEnd.Merge(dst, src) +} +func (m *ClientConnectionEnd) XXX_Size() int { + return m.Size() +} +func (m *ClientConnectionEnd) XXX_DiscardUnknown() { + xxx_messageInfo_ClientConnectionEnd.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientConnectionEnd proto.InternalMessageInfo + +// ClientSessionEnd is reported when a client session +// is completed. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_sessions.enabled` is set. +type ClientSessionEnd struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonConnectionDetails `protobuf:"bytes,2,opt,name=conn,proto3,embedded=conn" json:""` + CommonSessionDetails `protobuf:"bytes,3,opt,name=session,proto3,embedded=session" json:""` + // The duration of the connection in nanoseconds. + Duration int64 `protobuf:"varint,4,opt,name=duration,proto3" json:",omitempty"` +} + +func (m *ClientSessionEnd) Reset() { *m = ClientSessionEnd{} } +func (m *ClientSessionEnd) String() string { return proto.CompactTextString(m) } +func (*ClientSessionEnd) ProtoMessage() {} +func (*ClientSessionEnd) Descriptor() ([]byte, []int) { + return fileDescriptor_session_events_eb3571f04b8bd5df, []int{4} +} +func (m *ClientSessionEnd) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientSessionEnd) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClientSessionEnd) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientSessionEnd.Merge(dst, src) +} +func (m *ClientSessionEnd) XXX_Size() int { + return m.Size() +} +func (m *ClientSessionEnd) XXX_DiscardUnknown() { + xxx_messageInfo_ClientSessionEnd.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientSessionEnd proto.InternalMessageInfo + +// ClientAuthenticationFailed is reported when a client session +// did not authenticate successfully. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_sessions.enabled` is set. +type ClientAuthenticationFailed struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonConnectionDetails `protobuf:"bytes,2,opt,name=conn,proto3,embedded=conn" json:""` + CommonSessionDetails `protobuf:"bytes,3,opt,name=session,proto3,embedded=session" json:""` + // The reason for the authentication failure. + Reason string `protobuf:"bytes,4,opt,name=reason,proto3" json:",omitempty"` + // The detailed error for the authentication failure. + Detail string `protobuf:"bytes,5,opt,name=detail,proto3" json:",omitempty"` + // The authentication method used. + Method string `protobuf:"bytes,6,opt,name=method,proto3" json:",omitempty"` +} + +func (m *ClientAuthenticationFailed) Reset() { *m = ClientAuthenticationFailed{} } +func (m *ClientAuthenticationFailed) String() string { return proto.CompactTextString(m) } +func (*ClientAuthenticationFailed) ProtoMessage() {} +func (*ClientAuthenticationFailed) Descriptor() ([]byte, []int) { + return fileDescriptor_session_events_eb3571f04b8bd5df, []int{5} +} +func (m *ClientAuthenticationFailed) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientAuthenticationFailed) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClientAuthenticationFailed) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientAuthenticationFailed.Merge(dst, src) +} +func (m *ClientAuthenticationFailed) XXX_Size() int { + return m.Size() +} +func (m *ClientAuthenticationFailed) XXX_DiscardUnknown() { + xxx_messageInfo_ClientAuthenticationFailed.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientAuthenticationFailed proto.InternalMessageInfo + +// ClientAuthenticationOk is reported when a client session +// was authenticated successfully. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_sessions.enabled` is set. +type ClientAuthenticationOk struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonConnectionDetails `protobuf:"bytes,2,opt,name=conn,proto3,embedded=conn" json:""` + CommonSessionDetails `protobuf:"bytes,3,opt,name=session,proto3,embedded=session" json:""` + // The authentication method used. + Method string `protobuf:"bytes,4,opt,name=method,proto3" json:",omitempty"` +} + +func (m *ClientAuthenticationOk) Reset() { *m = ClientAuthenticationOk{} } +func (m *ClientAuthenticationOk) String() string { return proto.CompactTextString(m) } +func (*ClientAuthenticationOk) ProtoMessage() {} +func (*ClientAuthenticationOk) Descriptor() ([]byte, []int) { + return fileDescriptor_session_events_eb3571f04b8bd5df, []int{6} +} +func (m *ClientAuthenticationOk) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientAuthenticationOk) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClientAuthenticationOk) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientAuthenticationOk.Merge(dst, src) +} +func (m *ClientAuthenticationOk) XXX_Size() int { + return m.Size() +} +func (m *ClientAuthenticationOk) XXX_DiscardUnknown() { + xxx_messageInfo_ClientAuthenticationOk.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientAuthenticationOk proto.InternalMessageInfo + +// ClientAuthenticationInfo is reported for intermediate +// steps during the authentication process. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_sessions.enabled` is set. +type ClientAuthenticationInfo struct { + CommonEventDetails `protobuf:"bytes,1,opt,name=common,proto3,embedded=common" json:""` + CommonConnectionDetails `protobuf:"bytes,2,opt,name=conn,proto3,embedded=conn" json:""` + CommonSessionDetails `protobuf:"bytes,3,opt,name=session,proto3,embedded=session" json:""` + // The authentication method used, once known. + Method string `protobuf:"bytes,4,opt,name=method,proto3" json:",omitempty"` + // The authentication progress message. + Info string `protobuf:"bytes,5,opt,name=info,proto3" json:",omitempty"` +} + +func (m *ClientAuthenticationInfo) Reset() { *m = ClientAuthenticationInfo{} } +func (m *ClientAuthenticationInfo) String() string { return proto.CompactTextString(m) } +func (*ClientAuthenticationInfo) ProtoMessage() {} +func (*ClientAuthenticationInfo) Descriptor() ([]byte, []int) { + return fileDescriptor_session_events_eb3571f04b8bd5df, []int{7} +} +func (m *ClientAuthenticationInfo) XXX_Unmarshal(b []byte) error { + return m.Unmarshal(b) +} +func (m *ClientAuthenticationInfo) XXX_Marshal(b []byte, deterministic bool) ([]byte, error) { + b = b[:cap(b)] + n, err := m.MarshalTo(b) + if err != nil { + return nil, err + } + return b[:n], nil +} +func (dst *ClientAuthenticationInfo) XXX_Merge(src proto.Message) { + xxx_messageInfo_ClientAuthenticationInfo.Merge(dst, src) +} +func (m *ClientAuthenticationInfo) XXX_Size() int { + return m.Size() +} +func (m *ClientAuthenticationInfo) XXX_DiscardUnknown() { + xxx_messageInfo_ClientAuthenticationInfo.DiscardUnknown(m) +} + +var xxx_messageInfo_ClientAuthenticationInfo proto.InternalMessageInfo + +func init() { + proto.RegisterType((*CommonConnectionDetails)(nil), "cockroach.util.log.eventpb.CommonConnectionDetails") + proto.RegisterType((*CommonSessionDetails)(nil), "cockroach.util.log.eventpb.CommonSessionDetails") + proto.RegisterType((*ClientConnectionStart)(nil), "cockroach.util.log.eventpb.ClientConnectionStart") + proto.RegisterType((*ClientConnectionEnd)(nil), "cockroach.util.log.eventpb.ClientConnectionEnd") + proto.RegisterType((*ClientSessionEnd)(nil), "cockroach.util.log.eventpb.ClientSessionEnd") + proto.RegisterType((*ClientAuthenticationFailed)(nil), "cockroach.util.log.eventpb.ClientAuthenticationFailed") + proto.RegisterType((*ClientAuthenticationOk)(nil), "cockroach.util.log.eventpb.ClientAuthenticationOk") + proto.RegisterType((*ClientAuthenticationInfo)(nil), "cockroach.util.log.eventpb.ClientAuthenticationInfo") +} +func (m *CommonConnectionDetails) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CommonConnectionDetails) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if m.InstanceID != 0 { + dAtA[i] = 0x8 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.InstanceID)) + } + if len(m.Network) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.Network))) + i += copy(dAtA[i:], m.Network) + } + if len(m.RemoteAddress) > 0 { + dAtA[i] = 0x1a + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.RemoteAddress))) + i += copy(dAtA[i:], m.RemoteAddress) + } + return i, nil +} + +func (m *CommonSessionDetails) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *CommonSessionDetails) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + if len(m.Transport) > 0 { + dAtA[i] = 0xa + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.Transport))) + i += copy(dAtA[i:], m.Transport) + } + if len(m.User) > 0 { + dAtA[i] = 0x12 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.User))) + i += copy(dAtA[i:], m.User) + } + return i, nil +} + +func (m *ClientConnectionStart) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientConnectionStart) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonEventDetails.Size())) + n1, err := m.CommonEventDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n1 + dAtA[i] = 0x12 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonConnectionDetails.Size())) + n2, err := m.CommonConnectionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n2 + return i, nil +} + +func (m *ClientConnectionEnd) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientConnectionEnd) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonEventDetails.Size())) + n3, err := m.CommonEventDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n3 + dAtA[i] = 0x12 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonConnectionDetails.Size())) + n4, err := m.CommonConnectionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n4 + if m.Duration != 0 { + dAtA[i] = 0x18 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.Duration)) + } + return i, nil +} + +func (m *ClientSessionEnd) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientSessionEnd) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonEventDetails.Size())) + n5, err := m.CommonEventDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n5 + dAtA[i] = 0x12 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonConnectionDetails.Size())) + n6, err := m.CommonConnectionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n6 + dAtA[i] = 0x1a + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonSessionDetails.Size())) + n7, err := m.CommonSessionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n7 + if m.Duration != 0 { + dAtA[i] = 0x20 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.Duration)) + } + return i, nil +} + +func (m *ClientAuthenticationFailed) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientAuthenticationFailed) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonEventDetails.Size())) + n8, err := m.CommonEventDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n8 + dAtA[i] = 0x12 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonConnectionDetails.Size())) + n9, err := m.CommonConnectionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n9 + dAtA[i] = 0x1a + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonSessionDetails.Size())) + n10, err := m.CommonSessionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n10 + if len(m.Reason) > 0 { + dAtA[i] = 0x22 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.Reason))) + i += copy(dAtA[i:], m.Reason) + } + if len(m.Detail) > 0 { + dAtA[i] = 0x2a + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.Detail))) + i += copy(dAtA[i:], m.Detail) + } + if len(m.Method) > 0 { + dAtA[i] = 0x32 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.Method))) + i += copy(dAtA[i:], m.Method) + } + return i, nil +} + +func (m *ClientAuthenticationOk) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientAuthenticationOk) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonEventDetails.Size())) + n11, err := m.CommonEventDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n11 + dAtA[i] = 0x12 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonConnectionDetails.Size())) + n12, err := m.CommonConnectionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n12 + dAtA[i] = 0x1a + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonSessionDetails.Size())) + n13, err := m.CommonSessionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n13 + if len(m.Method) > 0 { + dAtA[i] = 0x22 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.Method))) + i += copy(dAtA[i:], m.Method) + } + return i, nil +} + +func (m *ClientAuthenticationInfo) Marshal() (dAtA []byte, err error) { + size := m.Size() + dAtA = make([]byte, size) + n, err := m.MarshalTo(dAtA) + if err != nil { + return nil, err + } + return dAtA[:n], nil +} + +func (m *ClientAuthenticationInfo) MarshalTo(dAtA []byte) (int, error) { + var i int + _ = i + var l int + _ = l + dAtA[i] = 0xa + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonEventDetails.Size())) + n14, err := m.CommonEventDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n14 + dAtA[i] = 0x12 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonConnectionDetails.Size())) + n15, err := m.CommonConnectionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n15 + dAtA[i] = 0x1a + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(m.CommonSessionDetails.Size())) + n16, err := m.CommonSessionDetails.MarshalTo(dAtA[i:]) + if err != nil { + return 0, err + } + i += n16 + if len(m.Method) > 0 { + dAtA[i] = 0x22 + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.Method))) + i += copy(dAtA[i:], m.Method) + } + if len(m.Info) > 0 { + dAtA[i] = 0x2a + i++ + i = encodeVarintSessionEvents(dAtA, i, uint64(len(m.Info))) + i += copy(dAtA[i:], m.Info) + } + return i, nil +} + +func encodeVarintSessionEvents(dAtA []byte, offset int, v uint64) int { + for v >= 1<<7 { + dAtA[offset] = uint8(v&0x7f | 0x80) + v >>= 7 + offset++ + } + dAtA[offset] = uint8(v) + return offset + 1 +} +func (m *CommonConnectionDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + if m.InstanceID != 0 { + n += 1 + sovSessionEvents(uint64(m.InstanceID)) + } + l = len(m.Network) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + l = len(m.RemoteAddress) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + return n +} + +func (m *CommonSessionDetails) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = len(m.Transport) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + l = len(m.User) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + return n +} + +func (m *ClientConnectionStart) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonConnectionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + return n +} + +func (m *ClientConnectionEnd) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonConnectionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + if m.Duration != 0 { + n += 1 + sovSessionEvents(uint64(m.Duration)) + } + return n +} + +func (m *ClientSessionEnd) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonConnectionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonSessionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + if m.Duration != 0 { + n += 1 + sovSessionEvents(uint64(m.Duration)) + } + return n +} + +func (m *ClientAuthenticationFailed) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonConnectionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonSessionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = len(m.Reason) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + l = len(m.Detail) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + l = len(m.Method) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + return n +} + +func (m *ClientAuthenticationOk) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonConnectionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonSessionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = len(m.Method) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + return n +} + +func (m *ClientAuthenticationInfo) Size() (n int) { + if m == nil { + return 0 + } + var l int + _ = l + l = m.CommonEventDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonConnectionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = m.CommonSessionDetails.Size() + n += 1 + l + sovSessionEvents(uint64(l)) + l = len(m.Method) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + l = len(m.Info) + if l > 0 { + n += 1 + l + sovSessionEvents(uint64(l)) + } + return n +} + +func sovSessionEvents(x uint64) (n int) { + for { + n++ + x >>= 7 + if x == 0 { + break + } + } + return n +} +func sozSessionEvents(x uint64) (n int) { + return sovSessionEvents(uint64((x << 1) ^ uint64((int64(x) >> 63)))) +} +func (m *CommonConnectionDetails) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CommonConnectionDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CommonConnectionDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field InstanceID", wireType) + } + m.InstanceID = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.InstanceID |= (int32(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Network", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Network = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field RemoteAddress", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.RemoteAddress = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSessionEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *CommonSessionDetails) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: CommonSessionDetails: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: CommonSessionDetails: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Transport", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Transport = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field User", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.User = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSessionEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientConnectionStart) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientConnectionStart: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientConnectionStart: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonConnectionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonConnectionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSessionEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientConnectionEnd) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientConnectionEnd: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientConnectionEnd: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonConnectionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonConnectionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Duration", wireType) + } + m.Duration = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Duration |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipSessionEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientSessionEnd) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientSessionEnd: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientSessionEnd: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonConnectionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonConnectionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSessionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSessionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 0 { + return fmt.Errorf("proto: wrong wireType = %d for field Duration", wireType) + } + m.Duration = 0 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + m.Duration |= (int64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + default: + iNdEx = preIndex + skippy, err := skipSessionEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientAuthenticationFailed) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientAuthenticationFailed: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientAuthenticationFailed: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonConnectionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonConnectionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSessionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSessionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Reason", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Reason = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Detail", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Detail = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 6: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Method", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Method = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSessionEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientAuthenticationOk) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientAuthenticationOk: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientAuthenticationOk: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonConnectionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonConnectionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSessionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSessionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Method", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Method = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSessionEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func (m *ClientAuthenticationInfo) Unmarshal(dAtA []byte) error { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + preIndex := iNdEx + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + fieldNum := int32(wire >> 3) + wireType := int(wire & 0x7) + if wireType == 4 { + return fmt.Errorf("proto: ClientAuthenticationInfo: wiretype end group for non-group") + } + if fieldNum <= 0 { + return fmt.Errorf("proto: ClientAuthenticationInfo: illegal tag %d (wire type %d)", fieldNum, wire) + } + switch fieldNum { + case 1: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonEventDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonEventDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 2: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonConnectionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonConnectionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 3: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field CommonSessionDetails", wireType) + } + var msglen int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + msglen |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + if msglen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + msglen + if postIndex > l { + return io.ErrUnexpectedEOF + } + if err := m.CommonSessionDetails.Unmarshal(dAtA[iNdEx:postIndex]); err != nil { + return err + } + iNdEx = postIndex + case 4: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Method", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Method = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + case 5: + if wireType != 2 { + return fmt.Errorf("proto: wrong wireType = %d for field Info", wireType) + } + var stringLen uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + stringLen |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + intStringLen := int(stringLen) + if intStringLen < 0 { + return ErrInvalidLengthSessionEvents + } + postIndex := iNdEx + intStringLen + if postIndex > l { + return io.ErrUnexpectedEOF + } + m.Info = string(dAtA[iNdEx:postIndex]) + iNdEx = postIndex + default: + iNdEx = preIndex + skippy, err := skipSessionEvents(dAtA[iNdEx:]) + if err != nil { + return err + } + if skippy < 0 { + return ErrInvalidLengthSessionEvents + } + if (iNdEx + skippy) > l { + return io.ErrUnexpectedEOF + } + iNdEx += skippy + } + } + + if iNdEx > l { + return io.ErrUnexpectedEOF + } + return nil +} +func skipSessionEvents(dAtA []byte) (n int, err error) { + l := len(dAtA) + iNdEx := 0 + for iNdEx < l { + var wire uint64 + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + wire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + wireType := int(wire & 0x7) + switch wireType { + case 0: + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + iNdEx++ + if dAtA[iNdEx-1] < 0x80 { + break + } + } + return iNdEx, nil + case 1: + iNdEx += 8 + return iNdEx, nil + case 2: + var length int + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + length |= (int(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + iNdEx += length + if length < 0 { + return 0, ErrInvalidLengthSessionEvents + } + return iNdEx, nil + case 3: + for { + var innerWire uint64 + var start int = iNdEx + for shift := uint(0); ; shift += 7 { + if shift >= 64 { + return 0, ErrIntOverflowSessionEvents + } + if iNdEx >= l { + return 0, io.ErrUnexpectedEOF + } + b := dAtA[iNdEx] + iNdEx++ + innerWire |= (uint64(b) & 0x7F) << shift + if b < 0x80 { + break + } + } + innerWireType := int(innerWire & 0x7) + if innerWireType == 4 { + break + } + next, err := skipSessionEvents(dAtA[start:]) + if err != nil { + return 0, err + } + iNdEx = start + next + } + return iNdEx, nil + case 4: + return iNdEx, nil + case 5: + iNdEx += 4 + return iNdEx, nil + default: + return 0, fmt.Errorf("proto: illegal wireType %d", wireType) + } + } + panic("unreachable") +} + +var ( + ErrInvalidLengthSessionEvents = fmt.Errorf("proto: negative length found during unmarshaling") + ErrIntOverflowSessionEvents = fmt.Errorf("proto: integer overflow") +) + +func init() { + proto.RegisterFile("util/log/eventpb/session_events.proto", fileDescriptor_session_events_eb3571f04b8bd5df) +} + +var fileDescriptor_session_events_eb3571f04b8bd5df = []byte{ + // 563 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xec, 0x56, 0x4d, 0x6f, 0xd3, 0x40, + 0x10, 0xb5, 0x93, 0x34, 0xa1, 0x13, 0xa8, 0x90, 0x29, 0x60, 0x45, 0xe0, 0x54, 0x96, 0x40, 0x01, + 0x55, 0x0e, 0x4a, 0xc5, 0x91, 0x43, 0x93, 0x16, 0x29, 0x27, 0xa4, 0x16, 0x24, 0xc4, 0x25, 0x72, + 0xed, 0x6d, 0xb2, 0x8a, 0xbd, 0x13, 0xed, 0x6e, 0x40, 0xfc, 0x0b, 0xee, 0xfc, 0x12, 0xee, 0x20, + 0x72, 0xcc, 0xb1, 0xa7, 0x08, 0x92, 0x5b, 0x25, 0xf8, 0x0d, 0xc8, 0xeb, 0x4d, 0x4b, 0xf3, 0x41, + 0xc4, 0xb1, 0x52, 0x6e, 0x5e, 0xbf, 0x37, 0xcf, 0x6f, 0xde, 0xac, 0xd7, 0x86, 0x47, 0x7d, 0x49, + 0xa3, 0x6a, 0x84, 0xed, 0x2a, 0x79, 0x4f, 0x98, 0xec, 0x9d, 0x54, 0x05, 0x11, 0x82, 0x22, 0x6b, + 0xa9, 0xb5, 0xf0, 0x7a, 0x1c, 0x25, 0x5a, 0xa5, 0x00, 0x83, 0x2e, 0x47, 0x3f, 0xe8, 0x78, 0x49, + 0x81, 0x17, 0x61, 0xdb, 0xd3, 0x05, 0xa5, 0xed, 0x36, 0xb6, 0x51, 0xd1, 0xaa, 0xc9, 0x55, 0x5a, + 0x51, 0x7a, 0x38, 0x27, 0xfc, 0xb7, 0xa0, 0xfb, 0xc5, 0x84, 0xfb, 0x0d, 0x8c, 0x63, 0x64, 0x0d, + 0x64, 0x8c, 0x04, 0x92, 0x22, 0x3b, 0x20, 0xd2, 0xa7, 0x91, 0xb0, 0x5e, 0x40, 0x91, 0x32, 0x21, + 0x7d, 0x16, 0x90, 0x16, 0x0d, 0x6d, 0x73, 0xc7, 0xac, 0x6c, 0xd4, 0x1f, 0x8c, 0x47, 0x65, 0x68, + 0xea, 0xdb, 0xcd, 0x83, 0xf3, 0x51, 0x19, 0x76, 0x31, 0xa6, 0x92, 0xc4, 0x3d, 0xf9, 0xf1, 0x08, + 0xa6, 0x05, 0xcd, 0xd0, 0xaa, 0x40, 0x81, 0x11, 0xf9, 0x01, 0x79, 0xd7, 0xce, 0xec, 0x98, 0x95, + 0xcd, 0xfa, 0xd6, 0x0c, 0x79, 0x0a, 0x5b, 0xcf, 0x61, 0x8b, 0x93, 0x18, 0x25, 0x69, 0xf9, 0x61, + 0xc8, 0x89, 0x10, 0x76, 0x76, 0x61, 0xc1, 0xad, 0x94, 0xb5, 0x9f, 0x92, 0xdc, 0x0e, 0x6c, 0xa7, + 0xd6, 0x8f, 0xd3, 0xa8, 0xa6, 0xbe, 0x77, 0x61, 0x53, 0x72, 0x9f, 0x89, 0x1e, 0x72, 0xa9, 0x5c, + 0xcf, 0x2b, 0x5d, 0x12, 0x2c, 0x17, 0x72, 0x7d, 0x41, 0xf8, 0x12, 0x8f, 0x0a, 0x73, 0xbf, 0x9a, + 0x70, 0xb7, 0x11, 0x51, 0xc2, 0xe4, 0x65, 0x4a, 0xc7, 0xd2, 0xe7, 0xd2, 0x7a, 0x0d, 0xf9, 0x40, + 0x79, 0x50, 0x0f, 0x2a, 0xd6, 0x3c, 0x6f, 0xf9, 0x84, 0xbc, 0xd4, 0xed, 0x61, 0xb2, 0xd2, 0x5e, + 0xeb, 0x37, 0x07, 0xa3, 0xb2, 0x31, 0x1c, 0x95, 0xcd, 0xf3, 0x51, 0xd9, 0x38, 0xd2, 0x5a, 0xd6, + 0x1b, 0xc8, 0x05, 0xc8, 0x98, 0xf2, 0x54, 0xac, 0xed, 0xad, 0xd6, 0x9c, 0x1b, 0xde, 0x8c, 0xb0, + 0x92, 0x73, 0x7f, 0x9b, 0x70, 0x67, 0xb6, 0x8d, 0x43, 0x16, 0x5e, 0xab, 0x26, 0xac, 0xa7, 0x70, + 0x23, 0xec, 0x73, 0x3f, 0xa1, 0xa9, 0x6d, 0x92, 0x9d, 0x9b, 0xd9, 0x05, 0xee, 0x7e, 0xcb, 0xc0, + 0xed, 0xb4, 0x61, 0xbd, 0x45, 0xae, 0x5d, 0xb7, 0x6f, 0xa1, 0xa0, 0x0f, 0x02, 0xd5, 0x6c, 0xb1, + 0xf6, 0x6c, 0xb5, 0xf2, 0xd5, 0xd7, 0x61, 0x46, 0x76, 0x2a, 0x77, 0x25, 0xc7, 0xdc, 0x8a, 0x1c, + 0x3f, 0x67, 0xa1, 0x94, 0xe6, 0xb8, 0xdf, 0x97, 0x1d, 0xc2, 0x24, 0x0d, 0x14, 0xf0, 0xd2, 0xa7, + 0x11, 0x59, 0x27, 0xaa, 0x13, 0x7d, 0x0c, 0x79, 0x4e, 0x7c, 0xa1, 0xf3, 0x9c, 0x3f, 0x4b, 0x34, + 0x9a, 0xf0, 0x42, 0xa5, 0x64, 0x6f, 0x2c, 0xe6, 0xa5, 0x68, 0xc2, 0x8b, 0x89, 0xec, 0x60, 0x68, + 0xe7, 0x17, 0xf3, 0x52, 0xd4, 0xfd, 0x9e, 0x81, 0x7b, 0x8b, 0xa6, 0xf3, 0xaa, 0xbb, 0x9e, 0xcc, + 0xc5, 0x64, 0x74, 0x92, 0xb9, 0x7f, 0x26, 0xf9, 0x2b, 0x03, 0xf6, 0xa2, 0x24, 0x9b, 0xec, 0x14, + 0xd7, 0x59, 0xfe, 0x57, 0x96, 0xc9, 0x77, 0x95, 0xb2, 0x53, 0x5c, 0xb2, 0xc7, 0x15, 0x56, 0x7f, + 0x32, 0xf8, 0xe9, 0x18, 0x83, 0xb1, 0x63, 0x0e, 0xc7, 0x8e, 0x79, 0x36, 0x76, 0xcc, 0x1f, 0x63, + 0xc7, 0xfc, 0x34, 0x71, 0x8c, 0xe1, 0xc4, 0x31, 0xce, 0x26, 0x8e, 0xf1, 0xae, 0xa0, 0x2d, 0x9e, + 0xe4, 0xd5, 0xff, 0xca, 0xde, 0x9f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xa7, 0x09, 0x24, 0x1b, 0x29, + 0x09, 0x00, 0x00, +} diff --git a/pkg/util/log/eventpb/session_events.proto b/pkg/util/log/eventpb/session_events.proto new file mode 100644 index 000000000000..39e17fbc67d2 --- /dev/null +++ b/pkg/util/log/eventpb/session_events.proto @@ -0,0 +1,142 @@ +// Copyright 2020 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +syntax = "proto3"; +package cockroach.util.log.eventpb; +option go_package = "eventpb"; + +import "gogoproto/gogo.proto"; +import "util/log/eventpb/events.proto"; + +// Category: SQL Session events +// Channel: SESSIONS +// +// Events in this category report SQL client connections +// and sessions. +// +// They are relative to a particular SQL tenant. +// In a multi-tenant setup, copies of these miscellaneous events are +// preserved in each tenant's own system.eventlog table. + +// Notes to CockroachDB maintainers: refer to doc.go at the package +// level for more details. Beware that JSON compatibility rules apply +// here, not protobuf. +// *Really look at doc.go before modifying this file.* + +// CommonConnectionDetails are payload fields common to all +// connection/session events. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_connections.enabled` is set. +message CommonConnectionDetails { + // The instance ID (not tenant ID) of the SQL server where the event was originated. + int32 instance_id = 1 [(gogoproto.customname) = "InstanceID", (gogoproto.jsontag) = ",omitempty"]; + // The network protocol for this connection: tcp4, tcp6, unix, etc. + string network = 2 [(gogoproto.jsontag) = ",omitempty"]; + // The remote address of the SQL client. Note that when using a + // proxy or other intermediate server, this field will contain the + // address of the intermediate server. + string remote_address = 3 [(gogoproto.jsontag) = ",omitempty"]; +} + +// CommonConnectionDetails are payload fields common to all +// session events. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_connections.enabled` is set. +message CommonSessionDetails { + // The connection type after transport negotiation. + string transport = 1 [(gogoproto.jsontag) = ",omitempty"]; + // The username the session is for. This is the username passed by + // the client, after case-folding and Unicode normalization. + string user = 2 [(gogoproto.jsontag) = ",omitempty"]; +} + +// ClientConnectionStart is reported when a client connection +// is established. This is reported even when authentication +// fails, and even for simple cancellation messages. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_connections.enabled` is set. +message ClientConnectionStart { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonConnectionDetails conn = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; +} + +// ClientConnectionEnd is reported when a client connection +// is closed. This is reported even when authentication +// fails, and even for simple cancellation messages. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_connections.enabled` is set. +message ClientConnectionEnd { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonConnectionDetails conn = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + // The duration of the connection in nanoseconds. + int64 duration = 3 [(gogoproto.jsontag) = ",omitempty"]; +} + +// ClientSessionEnd is reported when a client session +// is completed. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_sessions.enabled` is set. +message ClientSessionEnd { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonConnectionDetails conn = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSessionDetails session = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + // The duration of the connection in nanoseconds. + int64 duration = 4 [(gogoproto.jsontag) = ",omitempty"]; +} + +// ClientAuthenticationFailed is reported when a client session +// did not authenticate successfully. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_sessions.enabled` is set. +message ClientAuthenticationFailed { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonConnectionDetails conn = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSessionDetails session = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + // The reason for the authentication failure. + string reason = 4 [(gogoproto.jsontag) = ",omitempty"]; + // The detailed error for the authentication failure. + string detail = 5 [(gogoproto.jsontag) = ",omitempty"]; + // The authentication method used. + string method = 6 [(gogoproto.jsontag) = ",omitempty"]; +} + +// ClientAuthenticationOk is reported when a client session +// was authenticated successfully. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_sessions.enabled` is set. +message ClientAuthenticationOk { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonConnectionDetails conn = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSessionDetails session = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + // The authentication method used. + string method = 4 [(gogoproto.jsontag) = ",omitempty"]; +} + +// ClientAuthenticationInfo is reported for intermediate +// steps during the authentication process. +// +// Events of this type are only emitted when the cluster setting +// `server.auth_log.sql_sessions.enabled` is set. +message ClientAuthenticationInfo { + CommonEventDetails common = 1 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonConnectionDetails conn = 2 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + CommonSessionDetails session = 3 [(gogoproto.nullable) = false, (gogoproto.jsontag) = "", (gogoproto.embed) = true]; + // The authentication method used, once known. + string method = 4 [(gogoproto.jsontag) = ",omitempty"]; + // The authentication progress message. + string info = 5 [(gogoproto.jsontag) = ",omitempty"]; +}