diff --git a/go/flags/endtoend/vtcombo.txt b/go/flags/endtoend/vtcombo.txt index 1712f2181f1..d97836fc5e3 100644 --- a/go/flags/endtoend/vtcombo.txt +++ b/go/flags/endtoend/vtcombo.txt @@ -270,6 +270,7 @@ Flags: --querylog-buffer-size int Maximum number of buffered query logs before throttling log output (default 10) --querylog-filter-tag string string that must be present in the query for it to be logged; if using a value as the tag, you need to disable query normalization --querylog-format string format for query logs ("text" or "json") (default "text") + --querylog-json-v2 use v2 format for querylog-format=json --querylog-row-threshold uint Number of rows a query has to return or affect before being logged; not useful for streaming queries. 0 means all queries will be logged. --queryserver-config-acl-exempt-acl string an acl that exempt from table acl checking (this acl is free to access any vitess tables). --queryserver-config-annotate-queries prefix queries to MySQL backend with comment indicating vtgate principal (user) and target tablet type diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index 7585c71be6f..7ade3a3ec6b 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -172,6 +172,7 @@ Flags: --querylog-buffer-size int Maximum number of buffered query logs before throttling log output (default 10) --querylog-filter-tag string string that must be present in the query for it to be logged; if using a value as the tag, you need to disable query normalization --querylog-format string format for query logs ("text" or "json") (default "text") + --querylog-json-v2 use v2 format for querylog-format=json --querylog-row-threshold uint Number of rows a query has to return or affect before being logged; not useful for streaming queries. 0 means all queries will be logged. --redact-debug-ui-queries redact full queries and bind variables from debug UI --remote_operation_timeout duration time to wait for a remote operation (default 15s) diff --git a/go/flags/endtoend/vttablet.txt b/go/flags/endtoend/vttablet.txt index f0e69db12fc..088198cd3b2 100644 --- a/go/flags/endtoend/vttablet.txt +++ b/go/flags/endtoend/vttablet.txt @@ -263,6 +263,7 @@ Flags: --query-log-stream-handler string URL handler for streaming queries log (default "/debug/querylog") --querylog-filter-tag string string that must be present in the query for it to be logged; if using a value as the tag, you need to disable query normalization --querylog-format string format for query logs ("text" or "json") (default "text") + --querylog-json-v2 use v2 format for querylog-format=json --querylog-row-threshold uint Number of rows a query has to return or affect before being logged; not useful for streaming queries. 0 means all queries will be logged. --queryserver-config-acl-exempt-acl string an acl that exempt from table acl checking (this acl is free to access any vitess tables). --queryserver-config-annotate-queries prefix queries to MySQL backend with comment indicating vtgate principal (user) and target tablet type diff --git a/go/streamlog/bind_variable.go b/go/streamlog/bind_variable.go new file mode 100644 index 00000000000..9e3de31903d --- /dev/null +++ b/go/streamlog/bind_variable.go @@ -0,0 +1,140 @@ +/* +Copyright 2024 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package streamlog + +import ( + "encoding/json" + "errors" + + querypb "vitess.io/vitess/go/vt/proto/query" +) + +var ErrUnrecognizedBindVarType = errors.New("unrecognized bind variable type") + +// BindVariableValue is used to store querypb.BindVariable values. +type BindVariableValue struct { + Type string + Value []byte +} + +// MarshalJSON renders the BindVariableValue as json and optionally redacts the value. +func (bv BindVariableValue) MarshalJSON() ([]byte, error) { + out := map[string]interface{}{ + "Type": bv.Type, + "Value": bv.Value, + } + if GetRedactDebugUIQueries() { + out["Value"] = nil + } + return json.Marshal(out) +} + +// BindVariable is a wrapper for marshal/unmarshaling querypb.BindVariable as json. +// It ensures that the "Type" field is a string representation of the variable +// type instead of an integer-based code that is less portable and human-readable. +// +// This allows a *querypb.BindVariable that would have marshaled +// to this: +// +// {"Type":10262,"Value":"FmtAtEq6S9Y="} +// +// to marshal to this: +// +// {"Type":"VARBINARY","Value":"FmtAtEq6S9Y="} +// +// or if query redaction is enabled, like this: +// +// {"Type":"VARBINARY","Value":null} +type BindVariable struct { + Type string + Value []byte + Values []*BindVariableValue +} + +// NewBindVariable returns a wrapped *querypb.BindVariable object. +func NewBindVariable(bv *querypb.BindVariable) BindVariable { + newBv := BindVariable{ + Type: bv.Type.String(), + Value: bv.Value, + } + for _, val := range bv.Values { + newBv.Values = append(newBv.Values, &BindVariableValue{ + Type: val.Type.String(), + Value: val.Value, + }) + } + return newBv +} + +// NewBindVariables returns a string-map of wrapped *querypb.BindVariable objects. +func NewBindVariables(bvs map[string]*querypb.BindVariable) map[string]BindVariable { + out := make(map[string]BindVariable, len(bvs)) + for key, bindVar := range bvs { + out[key] = NewBindVariable(bindVar) + } + return out +} + +// MarshalJSON renders the BindVariable as json and optionally redacts the value. +func (bv BindVariable) MarshalJSON() ([]byte, error) { + out := map[string]interface{}{ + "Type": bv.Type, + "Value": bv.Value, + } + if GetRedactDebugUIQueries() { + out["Value"] = nil + } + return json.Marshal(out) +} + +// bindVariablesValuesToProto converts a slice of *BindVariableValue to *querypb.Value. +func bindVariablesValuesToProto(vals []*BindVariableValue) ([]*querypb.Value, error) { + values := make([]*querypb.Value, len(vals)) + for _, val := range vals { + varType, found := querypb.Type_value[val.Type] + if !found { + return nil, ErrUnrecognizedBindVarType + } + values = append(values, &querypb.Value{ + Type: querypb.Type(varType), + Value: val.Value, + }) + } + return values, nil +} + +// BindVariablesToProto converts a string-map of BindVariable to a string-map of *querypb.BindVariable. +func BindVariablesToProto(bvs map[string]BindVariable) (map[string]*querypb.BindVariable, error) { + out := make(map[string]*querypb.BindVariable, len(bvs)) + for key, bindVar := range bvs { + // convert type string to querypb.Type. + varType, found := querypb.Type_value[bindVar.Type] + if !found { + return nil, ErrUnrecognizedBindVarType + } + values, err := bindVariablesValuesToProto(bindVar.Values) + if err != nil { + return nil, err + } + out[key] = &querypb.BindVariable{ + Type: querypb.Type(varType), + Value: bindVar.Value, + Values: values, + } + } + return out, nil +} diff --git a/go/streamlog/streamlog.go b/go/streamlog/streamlog.go index 6d9f81f98d9..d98916ec982 100644 --- a/go/streamlog/streamlog.go +++ b/go/streamlog/streamlog.go @@ -51,6 +51,7 @@ var ( queryLogFilterTag string queryLogRowThreshold uint64 queryLogFormat = "text" + queryLogJSONV2 bool ) func GetRedactDebugUIQueries() bool { @@ -77,6 +78,14 @@ func SetQueryLogFormat(newQueryLogFormat string) { queryLogFormat = newQueryLogFormat } +func UseQueryLogJSONV2() bool { + return queryLogJSONV2 +} + +func SetQueryLogJSONV2(enabled bool) { + queryLogJSONV2 = enabled +} + func init() { servenv.OnParseFor("vtcombo", registerStreamLogFlags) servenv.OnParseFor("vttablet", registerStreamLogFlags) @@ -90,6 +99,9 @@ func registerStreamLogFlags(fs *pflag.FlagSet) { // QueryLogFormat controls the format of the query log (either text or json) fs.StringVar(&queryLogFormat, "querylog-format", queryLogFormat, "format for query logs (\"text\" or \"json\")") + // QueryLogJSONFormat controls whether the new querylog json format is used + fs.BoolVar(&queryLogJSONV2, "querylog-json-v2", false, "use v2 format for querylog-format=json") + // QueryLogFilterTag contains an optional string that must be present in the query for it to be logged fs.StringVar(&queryLogFilterTag, "querylog-filter-tag", queryLogFilterTag, "string that must be present in the query for it to be logged; if using a value as the tag, you need to disable query normalization") diff --git a/go/vt/vtgate/executor.go b/go/vt/vtgate/executor.go index 867685b8fec..b0d17d8acf8 100644 --- a/go/vt/vtgate/executor.go +++ b/go/vt/vtgate/executor.go @@ -1093,7 +1093,7 @@ func (e *Executor) getPlan( } logStats.SQL = comments.Leading + query + comments.Trailing - logStats.BindVariables = sqltypes.CopyBindVariables(bindVars) + logStats.BindVariables = streamlog.NewBindVariables(sqltypes.CopyBindVariables(bindVars)) return e.cacheAndBuildStatement(ctx, vcursor, query, stmt, reservedVars, bindVarNeeds, logStats) } diff --git a/go/vt/vtgate/logstats/logstats.go b/go/vt/vtgate/logstats/logstats.go index 5ea7820a72e..4df588cd598 100644 --- a/go/vt/vtgate/logstats/logstats.go +++ b/go/vt/vtgate/logstats/logstats.go @@ -36,14 +36,13 @@ import ( querypb "vitess.io/vitess/go/vt/proto/query" ) -// LogStats records the stats for a single vtgate query type LogStats struct { - Ctx context.Context + Ctx context.Context `json:"-"` Method string TabletType string StmtType string SQL string - BindVariables map[string]*querypb.BindVariable + BindVariables map[string]streamlog.BindVariable `json:",omitempty"` StartTime time.Time EndTime time.Time ShardQueries uint64 @@ -52,13 +51,21 @@ type LogStats struct { PlanTime time.Duration ExecuteTime time.Duration CommitTime time.Duration - Error error + Error error `json:",omitempty"` TablesUsed []string SessionUUID string CachedPlan bool ActiveKeyspace string // ActiveKeyspace is the selected keyspace `use ks` } +type LogStatsJSON struct { + RemoteAddr string + Username string + ImmediateCaller string + EffectiveCaller string + LogStats +} + // NewLogStats constructs a new LogStats with supplied Method and ctx // field values, and the StartTime field set to the present time. func NewLogStats(ctx context.Context, methodName, sql, sessionUUID string, bindVars map[string]*querypb.BindVariable) *LogStats { @@ -67,7 +74,7 @@ func NewLogStats(ctx context.Context, methodName, sql, sessionUUID string, bindV Method: methodName, SQL: sql, SessionUUID: sessionUUID, - BindVariables: bindVars, + BindVariables: streamlog.NewBindVariables(bindVars), StartTime: time.Now(), } } @@ -137,10 +144,14 @@ func (stats *LogStats) Logf(w io.Writer, params url.Values) error { }() formattedBindVars := "\"[REDACTED]\"" - if !streamlog.GetRedactDebugUIQueries() { + if !streamlog.GetRedactDebugUIQueries() && !streamlog.UseQueryLogJSONV2() { _, fullBindParams := params["full"] + bindVarsProto, err := streamlog.BindVariablesToProto(stats.BindVariables) + if err != nil { + return err + } formattedBindVars = sqltypes.FormatBindVariables( - stats.BindVariables, + bindVarsProto, fullBindParams, streamlog.GetQueryLogFormat() == streamlog.QueryLogFormatJSON, ) @@ -154,6 +165,16 @@ func (stats *LogStats) Logf(w io.Writer, params url.Values) error { case streamlog.QueryLogFormatText: fmtString = "%v\t%v\t%v\t'%v'\t'%v'\t%v\t%v\t%.6f\t%.6f\t%.6f\t%.6f\t%v\t%q\t%v\t%v\t%v\t%q\t%q\t%q\t%v\t%v\t%q\n" case streamlog.QueryLogFormatJSON: + if streamlog.UseQueryLogJSONV2() { + // flag --querylog-json-v2 + return json.NewEncoder(w).Encode(LogStatsJSON{ + EffectiveCaller: stats.EffectiveCaller(), + ImmediateCaller: stats.ImmediateCaller(), + LogStats: *stats, + RemoteAddr: remoteAddr, + Username: username, + }) + } fmtString = "{\"Method\": %q, \"RemoteAddr\": %q, \"Username\": %q, \"ImmediateCaller\": %q, \"Effective Caller\": %q, \"Start\": \"%v\", \"End\": \"%v\", \"TotalTime\": %.6f, \"PlanTime\": %v, \"ExecuteTime\": %v, \"CommitTime\": %v, \"StmtType\": %q, \"SQL\": %q, \"BindVars\": %v, \"ShardQueries\": %v, \"RowsAffected\": %v, \"Error\": %q, \"TabletType\": %q, \"SessionUUID\": %q, \"Cached Plan\": %v, \"TablesUsed\": %v, \"ActiveKeyspace\": %q}\n" } diff --git a/go/vt/vtgate/logstats/logstats_test.go b/go/vt/vtgate/logstats/logstats_test.go index dbe49b200b8..e70d81b83bd 100644 --- a/go/vt/vtgate/logstats/logstats_test.go +++ b/go/vt/vtgate/logstats/logstats_test.go @@ -66,15 +66,19 @@ func TestLogStatsFormat(t *testing.T) { logStats.TabletType = "PRIMARY" logStats.ActiveKeyspace = "db" params := map[string][]string{"full": {}} - intBindVar := map[string]*querypb.BindVariable{"intVal": sqltypes.Int64BindVariable(1)} - stringBindVar := map[string]*querypb.BindVariable{"strVal": sqltypes.StringBindVariable("abc")} + intBindVar := map[string]streamlog.BindVariable{ + "intVal": streamlog.NewBindVariable(sqltypes.Int64BindVariable(1)), + } + stringBindVar := map[string]streamlog.BindVariable{ + "strVal": streamlog.NewBindVariable(sqltypes.StringBindVariable("abc")), + } tests := []struct { name string redact bool format string expected string - bindVars map[string]*querypb.BindVariable + bindVars map[string]streamlog.BindVariable }{ { // 0 redact: false, @@ -148,6 +152,48 @@ func TestLogStatsFormat(t *testing.T) { } } +func TestLogStatsFormatJSONV2(t *testing.T) { + defer func() { + streamlog.SetRedactDebugUIQueries(false) + streamlog.SetQueryLogFormat("text") + streamlog.SetQueryLogJSONV2(false) + }() + logStats := NewLogStats(context.Background(), "test", "select * from testtable where name = :strVal and message = :bytesVal", "suuid", nil) + logStats.StartTime = time.Date(2017, time.January, 1, 1, 2, 3, 0, time.UTC) + logStats.EndTime = time.Date(2017, time.January, 1, 1, 2, 4, 1234, time.UTC) + logStats.StmtType = "select" + logStats.TablesUsed = []string{"ks1.tbl1", "ks2.tbl2"} + logStats.TabletType = "PRIMARY" + logStats.ActiveKeyspace = "db" + logStats.BindVariables = map[string]streamlog.BindVariable{ + "strVal": streamlog.NewBindVariable(sqltypes.StringBindVariable("abc")), + "bytesVal": streamlog.NewBindVariable(sqltypes.BytesBindVariable([]byte("\x16k@\xb4J\xbaK\xd6"))), + } + streamlog.SetQueryLogFormat("json") + streamlog.SetQueryLogJSONV2(true) + var cmpStats LogStatsJSON + { + // unredacted bind variables + streamlog.SetRedactDebugUIQueries(false) + var buf bytes.Buffer + assert.Nil(t, logStats.Logf(&buf, nil)) + assert.Equal(t, `{"RemoteAddr":"","Username":"","ImmediateCaller":"","EffectiveCaller":"","Method":"test","TabletType":"PRIMARY","StmtType":"select","SQL":"select * from testtable where name = :strVal and message = :bytesVal","BindVariables":{"bytesVal":{"Type":"VARBINARY","Value":"FmtAtEq6S9Y="},"strVal":{"Type":"VARCHAR","Value":"YWJj"}},"StartTime":"2017-01-01T01:02:03Z","EndTime":"2017-01-01T01:02:04.000001234Z","ShardQueries":0,"RowsAffected":0,"RowsReturned":0,"PlanTime":0,"ExecuteTime":0,"CommitTime":0,"TablesUsed":["ks1.tbl1","ks2.tbl2"],"SessionUUID":"suuid","CachedPlan":false,"ActiveKeyspace":"db"}`, strings.TrimSpace(buf.String())) + assert.Nil(t, json.Unmarshal(buf.Bytes(), &cmpStats)) + assert.Equal(t, "VARBINARY", cmpStats.BindVariables["bytesVal"].Type) + assert.Equal(t, []byte("\x16k@\xb4J\xbaK\xd6"), cmpStats.BindVariables["bytesVal"].Value) + assert.Equal(t, "VARCHAR", cmpStats.BindVariables["strVal"].Type) + assert.Equal(t, []byte("abc"), cmpStats.BindVariables["strVal"].Value) + } + { + // redacted bind variables + streamlog.SetRedactDebugUIQueries(true) + var buf bytes.Buffer + assert.Nil(t, logStats.Logf(&buf, nil)) + assert.Equal(t, `{"RemoteAddr":"","Username":"","ImmediateCaller":"","EffectiveCaller":"","Method":"test","TabletType":"PRIMARY","StmtType":"select","SQL":"select * from testtable where name = :strVal and message = :bytesVal","BindVariables":{"bytesVal":{"Type":"VARBINARY","Value":null},"strVal":{"Type":"VARCHAR","Value":null}},"StartTime":"2017-01-01T01:02:03Z","EndTime":"2017-01-01T01:02:04.000001234Z","ShardQueries":0,"RowsAffected":0,"RowsReturned":0,"PlanTime":0,"ExecuteTime":0,"CommitTime":0,"TablesUsed":["ks1.tbl1","ks2.tbl2"],"SessionUUID":"suuid","CachedPlan":false,"ActiveKeyspace":"db"}`, strings.TrimSpace(buf.String())) + assert.Nil(t, json.Unmarshal(buf.Bytes(), &cmpStats)) + } +} + func TestLogStatsFilter(t *testing.T) { defer func() { streamlog.SetQueryLogFilterTag("") }() diff --git a/go/vt/vttablet/tabletserver/tabletenv/logstats.go b/go/vt/vttablet/tabletserver/tabletenv/logstats.go index 962b5f0b122..837f42bffa8 100644 --- a/go/vt/vttablet/tabletserver/tabletenv/logstats.go +++ b/go/vt/vttablet/tabletserver/tabletenv/logstats.go @@ -18,6 +18,7 @@ package tabletenv import ( "context" + "encoding/json" "fmt" "io" "net/url" @@ -43,13 +44,13 @@ const ( // LogStats records the stats for a single query type LogStats struct { - Ctx context.Context + Ctx context.Context `json:"-"` Method string - Target *querypb.Target + Target *querypb.Target `json:"-"` PlanType string OriginalSQL string - BindVariables map[string]*querypb.BindVariable - rewrittenSqls []string + BindVariables map[string]streamlog.BindVariable `json:",omitempty"` + rewrittenSqls []string `json:"-"` RowsAffected int NumberOfQueries int StartTime time.Time @@ -57,13 +58,24 @@ type LogStats struct { MysqlResponseTime time.Duration WaitingForConnection time.Duration QuerySources byte - Rows [][]sqltypes.Value + Rows [][]sqltypes.Value `json:"-"` TransactionID int64 ReservedID int64 - Error error + Error error `json:",omitempty"` CachedPlan bool } +type LogStatsJSON struct { + CallInfo string + Username string + ImmediateCaller string + EffectiveCaller string + RewrittenSQL string + TotalTime time.Duration + ResponseSize int + LogStats +} + // NewLogStats constructs a new LogStats with supplied Method and ctx // field values, and the StartTime field set to the present time. func NewLogStats(ctx context.Context, methodName string) *LogStats { @@ -111,6 +123,9 @@ func (stats *LogStats) TotalTime() time.Duration { // RewrittenSQL returns a semicolon separated list of SQL statements // that were executed. func (stats *LogStats) RewrittenSQL() string { + if streamlog.GetRedactDebugUIQueries() { + return "[REDACTED]" + } return strings.Join(stats.rewrittenSqls, "; ") } @@ -181,15 +196,15 @@ func (stats *LogStats) Logf(w io.Writer, params url.Values) error { return nil } - rewrittenSQL := "[REDACTED]" formattedBindVars := "\"[REDACTED]\"" - - if !streamlog.GetRedactDebugUIQueries() { - rewrittenSQL = stats.RewrittenSQL() - + if !streamlog.GetRedactDebugUIQueries() && !streamlog.UseQueryLogJSONV2() { _, fullBindParams := params["full"] + bindVarsProto, err := streamlog.BindVariablesToProto(stats.BindVariables) + if err != nil { + return err + } formattedBindVars = sqltypes.FormatBindVariables( - stats.BindVariables, + bindVarsProto, fullBindParams, streamlog.GetQueryLogFormat() == streamlog.QueryLogFormatJSON, ) @@ -204,6 +219,19 @@ func (stats *LogStats) Logf(w io.Writer, params url.Values) error { case streamlog.QueryLogFormatText: fmtString = "%v\t%v\t%v\t'%v'\t'%v'\t%v\t%v\t%.6f\t%v\t%q\t%v\t%v\t%q\t%v\t%.6f\t%.6f\t%v\t%v\t%v\t%q\t\n" case streamlog.QueryLogFormatJSON: + if streamlog.UseQueryLogJSONV2() { + // flag --querylog-json-v2 + return json.NewEncoder(w).Encode(LogStatsJSON{ + EffectiveCaller: stats.EffectiveCaller(), + ImmediateCaller: stats.ImmediateCaller(), + LogStats: *stats, + Username: username, + CallInfo: callInfo, + RewrittenSQL: stats.RewrittenSQL(), + ResponseSize: stats.SizeOfResponse(), + TotalTime: stats.TotalTime(), + }) + } fmtString = "{\"Method\": %q, \"CallInfo\": %q, \"Username\": %q, \"ImmediateCaller\": %q, \"Effective Caller\": %q, \"Start\": \"%v\", \"End\": \"%v\", \"TotalTime\": %.6f, \"PlanType\": %q, \"OriginalSQL\": %q, \"BindVars\": %v, \"Queries\": %v, \"RewrittenSQL\": %q, \"QuerySources\": %q, \"MysqlTime\": %.6f, \"ConnWaitTime\": %.6f, \"RowsAffected\": %v,\"TransactionID\": %v,\"ResponseSize\": %v, \"Error\": %q}\n" } @@ -222,7 +250,7 @@ func (stats *LogStats) Logf(w io.Writer, params url.Values) error { stats.OriginalSQL, formattedBindVars, stats.NumberOfQueries, - rewrittenSQL, + stats.RewrittenSQL(), stats.FmtQuerySources(), stats.MysqlResponseTime.Seconds(), stats.WaitingForConnection.Seconds(), diff --git a/go/vt/vttablet/tabletserver/tabletenv/logstats_test.go b/go/vt/vttablet/tabletserver/tabletenv/logstats_test.go index 51e056687b5..9d62d8cc248 100644 --- a/go/vt/vttablet/tabletserver/tabletenv/logstats_test.go +++ b/go/vt/vttablet/tabletserver/tabletenv/logstats_test.go @@ -17,6 +17,7 @@ limitations under the License. package tabletenv import ( + "bytes" "context" "encoding/json" "errors" @@ -26,12 +27,12 @@ import ( "time" "github.com/google/safehtml/testconversions" + "github.com/stretchr/testify/assert" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/streamlog" "vitess.io/vitess/go/vt/callinfo" "vitess.io/vitess/go/vt/callinfo/fakecallinfo" - querypb "vitess.io/vitess/go/vt/proto/query" ) func TestLogStats(t *testing.T) { @@ -63,7 +64,9 @@ func TestLogStatsFormat(t *testing.T) { logStats.StartTime = time.Date(2017, time.January, 1, 1, 2, 3, 0, time.UTC) logStats.EndTime = time.Date(2017, time.January, 1, 1, 2, 4, 1234, time.UTC) logStats.OriginalSQL = "sql" - logStats.BindVariables = map[string]*querypb.BindVariable{"intVal": sqltypes.Int64BindVariable(1)} + logStats.BindVariables = map[string]streamlog.BindVariable{ + "intVal": streamlog.NewBindVariable(sqltypes.Int64BindVariable(1)), + } logStats.AddRewrittenSQL("sql with pii", time.Now()) logStats.MysqlResponseTime = 0 logStats.TransactionID = 12345 @@ -123,7 +126,9 @@ func TestLogStatsFormat(t *testing.T) { // Make sure formatting works for string bind vars. We can't do this as part of a single // map because the output ordering is undefined. - logStats.BindVariables = map[string]*querypb.BindVariable{"strVal": sqltypes.StringBindVariable("abc")} + logStats.BindVariables = map[string]streamlog.BindVariable{ + "strVal": streamlog.NewBindVariable(sqltypes.StringBindVariable("abc")), + } streamlog.SetQueryLogFormat("text") got = testFormat(logStats, url.Values(params)) @@ -157,7 +162,9 @@ func TestLogStatsFilter(t *testing.T) { logStats.StartTime = time.Date(2017, time.January, 1, 1, 2, 3, 0, time.UTC) logStats.EndTime = time.Date(2017, time.January, 1, 1, 2, 4, 1234, time.UTC) logStats.OriginalSQL = "sql /* LOG_THIS_QUERY */" - logStats.BindVariables = map[string]*querypb.BindVariable{"intVal": sqltypes.Int64BindVariable(1)} + logStats.BindVariables = map[string]streamlog.BindVariable{ + "intVal": streamlog.NewBindVariable(sqltypes.Int64BindVariable(1)), + } logStats.AddRewrittenSQL("sql with pii", time.Now()) logStats.MysqlResponseTime = 0 logStats.Rows = [][]sqltypes.Value{{sqltypes.NewVarBinary("a")}} @@ -184,6 +191,49 @@ func TestLogStatsFilter(t *testing.T) { } } +func TestLogStatsFormatJSONV2(t *testing.T) { + defer func() { + streamlog.SetRedactDebugUIQueries(false) + streamlog.SetQueryLogFormat("text") + streamlog.SetQueryLogJSONV2(false) + }() + logStats := NewLogStats(context.Background(), "test") + logStats.StartTime = time.Date(2017, time.January, 1, 1, 2, 3, 0, time.UTC) + logStats.EndTime = time.Date(2017, time.January, 1, 1, 2, 4, 1234, time.UTC) + logStats.OriginalSQL = "sql" + logStats.BindVariables = map[string]streamlog.BindVariable{ + "bytesVal": streamlog.NewBindVariable(sqltypes.BytesBindVariable([]byte("\x16k@\xb4J\xbaK\xd6"))), + "intVal": streamlog.NewBindVariable(sqltypes.Int64BindVariable(1)), + } + logStats.AddRewrittenSQL("sql with pii", time.Now()) + logStats.MysqlResponseTime = 0 + logStats.TransactionID = 12345 + logStats.Rows = [][]sqltypes.Value{{sqltypes.NewVarBinary("a")}} + streamlog.SetQueryLogFormat("json") + streamlog.SetQueryLogJSONV2(true) + var cmpStats LogStatsJSON + { + // unredacted bind variables + streamlog.SetRedactDebugUIQueries(false) + var buf bytes.Buffer + assert.Nil(t, logStats.Logf(&buf, nil)) + assert.Equal(t, `{"CallInfo":"","Username":"","ImmediateCaller":"","EffectiveCaller":"","RewrittenSQL":"sql with pii","TotalTime":1000001234,"ResponseSize":1,"Method":"test","PlanType":"","OriginalSQL":"sql","BindVariables":{"bytesVal":{"Type":"VARBINARY","Value":"FmtAtEq6S9Y="},"intVal":{"Type":"INT64","Value":"MQ=="}},"RowsAffected":0,"NumberOfQueries":1,"StartTime":"2017-01-01T01:02:03Z","EndTime":"2017-01-01T01:02:04.000001234Z","MysqlResponseTime":0,"WaitingForConnection":0,"QuerySources":2,"TransactionID":12345,"ReservedID":0,"CachedPlan":false}`, strings.TrimSpace(buf.String())) + assert.Nil(t, json.Unmarshal(buf.Bytes(), &cmpStats)) + assert.Equal(t, "VARBINARY", cmpStats.BindVariables["bytesVal"].Type) + assert.Equal(t, []byte("\x16k@\xb4J\xbaK\xd6"), cmpStats.BindVariables["bytesVal"].Value) + assert.Equal(t, "INT64", cmpStats.BindVariables["intVal"].Type) + assert.Equal(t, []byte("1"), cmpStats.BindVariables["intVal"].Value) + } + { + // redacted bind variables + streamlog.SetRedactDebugUIQueries(true) + var buf bytes.Buffer + assert.Nil(t, logStats.Logf(&buf, nil)) + assert.Equal(t, `{"CallInfo":"","Username":"","ImmediateCaller":"","EffectiveCaller":"","RewrittenSQL":"[REDACTED]","TotalTime":1000001234,"ResponseSize":1,"Method":"test","PlanType":"","OriginalSQL":"sql","BindVariables":{"bytesVal":{"Type":"VARBINARY","Value":null},"intVal":{"Type":"INT64","Value":null}},"RowsAffected":0,"NumberOfQueries":1,"StartTime":"2017-01-01T01:02:03Z","EndTime":"2017-01-01T01:02:04.000001234Z","MysqlResponseTime":0,"WaitingForConnection":0,"QuerySources":2,"TransactionID":12345,"ReservedID":0,"CachedPlan":false}`, strings.TrimSpace(buf.String())) + assert.Nil(t, json.Unmarshal(buf.Bytes(), &cmpStats)) + } +} + func TestLogStatsFormatQuerySources(t *testing.T) { logStats := NewLogStats(context.Background(), "test") if logStats.FmtQuerySources() != "none" { diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 6ecc46c68ab..6be65681501 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -38,6 +38,7 @@ import ( "vitess.io/vitess/go/pools/smartconnpool" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/stats" + "vitess.io/vitess/go/streamlog" "vitess.io/vitess/go/tb" "vitess.io/vitess/go/trace" "vitess.io/vitess/go/vt/callerid" @@ -1546,7 +1547,7 @@ func (tsv *TabletServer) execRequest( logStats := tabletenv.NewLogStats(ctx, requestName) logStats.Target = target logStats.OriginalSQL = sql - logStats.BindVariables = sqltypes.CopyBindVariables(bindVariables) + logStats.BindVariables = streamlog.NewBindVariables(sqltypes.CopyBindVariables(bindVariables)) defer tsv.handlePanicAndSendLogStats(sql, bindVariables, logStats) if err = tsv.sm.StartRequest(ctx, target, allowOnShutdown); err != nil {