From b0c5325de6d49e5bea44d9bd990a213b0a81c9a0 Mon Sep 17 00:00:00 2001 From: Tim Vaillancourt Date: Tue, 4 Jun 2024 19:11:04 +0200 Subject: [PATCH] `slack-15.0`: v20 backports, pt. 2 (#394) * `vtgate`: support filtering tablets by tablet-tags (#15911) Signed-off-by: Tim Vaillancourt Co-authored-by: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> * Add support for sampling rate in `streamlog` (#15919) Signed-off-by: Tim Vaillancourt * Fix merge conflict resolution Signed-off-by: Tim Vaillancourt * update rand import Signed-off-by: Tim Vaillancourt * Add sql text counts stats to `vtcombo`,`vtgate`+`vttablet` (#15897) Signed-off-by: Tim Vaillancourt Signed-off-by: Harshit Gangal Co-authored-by: Harshit Gangal Co-authored-by: Deepthi Sigireddi * missing rename Signed-off-by: Tim Vaillancourt * missing rename again Signed-off-by: Tim Vaillancourt --------- Signed-off-by: Tim Vaillancourt Signed-off-by: Harshit Gangal Co-authored-by: Shlomi Noach <2607934+shlomi-noach@users.noreply.github.com> Co-authored-by: Harshit Gangal Co-authored-by: Deepthi Sigireddi --- go/flags/endtoend/vtgate.txt | 2 + go/flags/endtoend/vttablet.txt | 1 + go/streamlog/streamlog.go | 21 ++ go/streamlog/streamlog_flaky_test.go | 131 +++++++ go/vt/discovery/healthcheck.go | 16 +- go/vt/discovery/topology_watcher.go | 42 +++ go/vt/discovery/topology_watcher_test.go | 27 ++ go/vt/vtgate/vtgate.go | 12 +- go/vt/vttablet/tabletserver/query_engine.go | 6 +- .../tabletserver/query_engine_test.go | 328 +++++++++--------- 10 files changed, 409 insertions(+), 177 deletions(-) diff --git a/go/flags/endtoend/vtgate.txt b/go/flags/endtoend/vtgate.txt index 16f6ba455ae..1ce1c80dc9e 100644 --- a/go/flags/endtoend/vtgate.txt +++ b/go/flags/endtoend/vtgate.txt @@ -137,6 +137,7 @@ Usage of vtgate: --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-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. + --querylog-sample-rate float Sample rate for logging queries. Value must be between 0.0 (no logging) and 1.0 (all queries) --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) --retry-count int retry count (default 2) @@ -159,6 +160,7 @@ Usage of vtgate: --stderrthreshold severityFlag logs at or above this threshold go to stderr (default 1) --stream_buffer_size int the number of bytes sent from vtgate for each stream call. It's recommended to keep this value in sync with vttablet's query-server-config-stream-buffer-size. (default 32768) --structured-logging whether to use structured logging (PlanetScale Log) logger or the original (glog) logger + --tablet-filter-tags StringMap Specifies a comma-separated list of tablet tags (as key:value pairs) to filter the tablets to watch. --tablet_filters strings Specifies a comma-separated list of 'keyspace|shard_name or keyrange' values to filter the tablets to watch. --tablet_grpc_ca string the server ca to use to validate servers when connecting --tablet_grpc_cert string the cert to use to connect diff --git a/go/flags/endtoend/vttablet.txt b/go/flags/endtoend/vttablet.txt index 8e5b7af0338..941df99ba04 100644 --- a/go/flags/endtoend/vttablet.txt +++ b/go/flags/endtoend/vttablet.txt @@ -210,6 +210,7 @@ Usage of vttablet: --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-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. + --querylog-sample-rate float Sample rate for logging queries. Value must be between 0.0 (no logging) and 1.0 (all queries) --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 --queryserver-config-enable-table-acl-dry-run If this flag is enabled, tabletserver will emit monitoring metrics and let the request pass regardless of table acl check results diff --git a/go/streamlog/streamlog.go b/go/streamlog/streamlog.go index 7875ae1146b..4f47353e4e8 100644 --- a/go/streamlog/streamlog.go +++ b/go/streamlog/streamlog.go @@ -20,6 +20,7 @@ package streamlog import ( "fmt" "io" + rand "math/rand" "net/http" "net/url" "os" @@ -53,6 +54,7 @@ var ( queryLogFilterTag string queryLogRowThreshold uint64 queryLogFormat = "text" + queryLogSampleRate float64 ) func GetRedactDebugUIQueries() bool { @@ -79,6 +81,10 @@ func SetQueryLogRowThreshold(newQueryLogRowThreshold uint64) { queryLogRowThreshold = newQueryLogRowThreshold } +func SetQueryLogSampleRate(sampleRate float64) { + queryLogSampleRate = sampleRate +} + func GetQueryLogFormat() string { return queryLogFormat } @@ -106,6 +112,8 @@ func registerStreamLogFlags(fs *pflag.FlagSet) { // QueryLogRowThreshold only log queries returning or affecting this many rows fs.Uint64Var(&queryLogRowThreshold, "querylog-row-threshold", queryLogRowThreshold, "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.") + // QueryLogSampleRate causes a sample of queries to be logged + fs.Float64Var(&queryLogSampleRate, "querylog-sample-rate", queryLogSampleRate, "Sample rate for logging queries. Value must be between 0.0 (no logging) and 1.0 (all queries)") } const ( @@ -259,9 +267,22 @@ func GetFormatter(logger *StreamLogger) LogFormatter { } } +// shouldSampleQuery returns true if a query should be sampled based on queryLogSampleRate +func shouldSampleQuery() bool { + if queryLogSampleRate <= 0 { + return false + } else if queryLogSampleRate >= 1 { + return true + } + return rand.Float64() <= queryLogSampleRate +} + // ShouldEmitLog returns whether the log with the given SQL query // should be emitted or filtered func ShouldEmitLog(sql string, rowsAffected, rowsReturned uint64) bool { + if shouldSampleQuery() { + return true + } if queryLogRowThreshold > maxUint64(rowsAffected, rowsReturned) && queryLogFilterTag == "" { return false } diff --git a/go/streamlog/streamlog_flaky_test.go b/go/streamlog/streamlog_flaky_test.go index 0f2083a8486..98dddcc9e05 100644 --- a/go/streamlog/streamlog_flaky_test.go +++ b/go/streamlog/streamlog_flaky_test.go @@ -28,6 +28,9 @@ import ( "syscall" "testing" "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" ) type logMessage struct { @@ -253,3 +256,131 @@ func TestFile(t *testing.T) { t.Errorf("streamlog file: want %q got %q", want, got) } } + +func TestShouldSampleQuery(t *testing.T) { + queryLogSampleRate = -1 + assert.False(t, shouldSampleQuery()) + + queryLogSampleRate = 0 + assert.False(t, shouldSampleQuery()) + + // for test coverage, can't test a random result + queryLogSampleRate = 0.5 + shouldSampleQuery() + + queryLogSampleRate = 1.0 + assert.True(t, shouldSampleQuery()) + + queryLogSampleRate = 100.0 + assert.True(t, shouldSampleQuery()) +} + +func TestShouldEmitLog(t *testing.T) { + origQueryLogFilterTag := queryLogFilterTag + origQueryLogRowThreshold := queryLogRowThreshold + origQueryLogSampleRate := queryLogSampleRate + defer func() { + SetQueryLogFilterTag(origQueryLogFilterTag) + SetQueryLogRowThreshold(origQueryLogRowThreshold) + SetQueryLogSampleRate(origQueryLogSampleRate) + }() + + tests := []struct { + sql string + qLogFilterTag string + qLogRowThreshold uint64 + qLogSampleRate float64 + rowsAffected uint64 + rowsReturned uint64 + ok bool + }{ + { + sql: "queryLogThreshold smaller than affected and returned", + qLogFilterTag: "", + qLogRowThreshold: 2, + qLogSampleRate: 0.0, + rowsAffected: 7, + rowsReturned: 7, + ok: true, + }, + { + sql: "queryLogThreshold greater than affected and returned", + qLogFilterTag: "", + qLogRowThreshold: 27, + qLogSampleRate: 0.0, + rowsAffected: 7, + rowsReturned: 17, + ok: false, + }, + { + sql: "this doesn't contains queryFilterTag: TAG", + qLogFilterTag: "special tag", + qLogRowThreshold: 10, + qLogSampleRate: 0.0, + rowsAffected: 7, + rowsReturned: 17, + ok: false, + }, + { + sql: "this contains queryFilterTag: TAG", + qLogFilterTag: "TAG", + qLogRowThreshold: 0, + qLogSampleRate: 0.0, + rowsAffected: 7, + rowsReturned: 17, + ok: true, + }, + { + sql: "this contains querySampleRate: 1.0", + qLogFilterTag: "", + qLogRowThreshold: 0, + qLogSampleRate: 1.0, + rowsAffected: 7, + rowsReturned: 17, + ok: true, + }, + { + sql: "this contains querySampleRate: 1.0 without expected queryFilterTag", + qLogFilterTag: "TAG", + qLogRowThreshold: 0, + qLogSampleRate: 1.0, + rowsAffected: 7, + rowsReturned: 17, + ok: true, + }, + } + + for _, tt := range tests { + t.Run(tt.sql, func(t *testing.T) { + SetQueryLogFilterTag(tt.qLogFilterTag) + SetQueryLogRowThreshold(tt.qLogRowThreshold) + SetQueryLogSampleRate(tt.qLogSampleRate) + + require.Equal(t, tt.ok, ShouldEmitLog(tt.sql, tt.rowsAffected, tt.rowsReturned)) + }) + } +} + +func BenchmarkShouldEmitLog(b *testing.B) { + b.Run("default", func(b *testing.B) { + SetQueryLogSampleRate(0.0) + for i := 0; i < b.N; i++ { + ShouldEmitLog("select * from test where user='someone'", 0, 123) + } + }) + b.Run("filter_tag", func(b *testing.B) { + SetQueryLogSampleRate(0.0) + SetQueryLogFilterTag("LOG_QUERY") + defer SetQueryLogFilterTag("") + for i := 0; i < b.N; i++ { + ShouldEmitLog("select /* LOG_QUERY=1 */ * from test where user='someone'", 0, 123) + } + }) + b.Run("50%_sample_rate", func(b *testing.B) { + SetQueryLogSampleRate(0.5) + defer SetQueryLogSampleRate(0.0) + for i := 0; i < b.N; i++ { + ShouldEmitLog("select * from test where user='someone'", 0, 123) + } + }) +} diff --git a/go/vt/discovery/healthcheck.go b/go/vt/discovery/healthcheck.go index 9ffd90ad649..6374ecd5078 100644 --- a/go/vt/discovery/healthcheck.go +++ b/go/vt/discovery/healthcheck.go @@ -47,6 +47,7 @@ import ( "github.com/spf13/pflag" "golang.org/x/sync/semaphore" + "vitess.io/vitess/go/flagutil" "vitess.io/vitess/go/netutil" "vitess.io/vitess/go/stats" "vitess.io/vitess/go/vt/log" @@ -80,6 +81,9 @@ var ( // tabletFilters are the keyspace|shard or keyrange filters to apply to the full set of tablets. tabletFilters []string + // tabletFilterTags are the tablet tag filters (as key:value pairs) to apply to the full set of tablets. + tabletFilterTags flagutil.StringMapValue + // refreshInterval is the interval at which healthcheck refreshes its list of tablets from topo. refreshInterval = 1 * time.Minute @@ -161,6 +165,7 @@ func init() { func registerDiscoveryFlags(fs *pflag.FlagSet) { fs.StringSliceVar(&tabletFilters, "tablet_filters", []string{}, "Specifies a comma-separated list of 'keyspace|shard_name or keyrange' values to filter the tablets to watch.") + fs.Var(&tabletFilterTags, "tablet-filter-tags", "Specifies a comma-separated list of tablet tags (as key:value pairs) to filter the tablets to watch.") fs.Var((*topoproto.TabletTypeListFlag)(&AllowedTabletTypes), "allowed_tablet_types", "Specifies the tablet types this vtgate is allowed to route queries to. Should be provided as a comma-separated set of tablet types.") fs.StringSliceVar(&KeyspacesToWatch, "keyspaces_to_watch", []string{}, "Specifies which keyspaces this vtgate should have access to while routing queries or accessing the vschema.") } @@ -331,13 +336,13 @@ func NewHealthCheck(ctx context.Context, retryDelay, healthCheckTimeout time.Dur cellAliases: make(map[string]string), } var topoWatchers []*TopologyWatcher - var filter TabletFilter cells := strings.Split(cellsToWatch, ",") if cellsToWatch == "" { cells = append(cells, localCell) } for _, c := range cells { + var filters TabletFilters log.Infof("Setting up healthcheck for cell: %v", c) if c == "" { continue @@ -351,11 +356,14 @@ func NewHealthCheck(ctx context.Context, retryDelay, healthCheckTimeout time.Dur if err != nil { log.Exitf("Cannot parse tablet_filters parameter: %v", err) } - filter = fbs + filters = append(filters, fbs) } else if len(KeyspacesToWatch) > 0 { - filter = NewFilterByKeyspace(KeyspacesToWatch) + filters = append(filters, NewFilterByKeyspace(KeyspacesToWatch)) + } + if len(tabletFilterTags) > 0 { + filters = append(filters, NewFilterByTabletTags(tabletFilterTags)) } - topoWatchers = append(topoWatchers, NewCellTabletsWatcher(ctx, topoServer, hc, filter, c, refreshInterval, refreshKnownTablets, topoReadConcurrency)) + topoWatchers = append(topoWatchers, NewCellTabletsWatcher(ctx, topoServer, hc, filters, c, refreshInterval, refreshKnownTablets, topoReadConcurrency)) } hc.topoWatchers = topoWatchers diff --git a/go/vt/discovery/topology_watcher.go b/go/vt/discovery/topology_watcher.go index 78d5f696f6f..57a29679633 100644 --- a/go/vt/discovery/topology_watcher.go +++ b/go/vt/discovery/topology_watcher.go @@ -293,6 +293,19 @@ type TabletFilter interface { IsIncluded(tablet *topodata.Tablet) bool } +// TabletFilters contains filters for tablets. +type TabletFilters []TabletFilter + +// IsIncluded returns true if a tablet passes all filters. +func (tf TabletFilters) IsIncluded(tablet *topodata.Tablet) bool { + for _, filter := range tf { + if !filter.IsIncluded(tablet) { + return false + } + } + return true +} + // FilterByShard is a filter that filters tablets by // keyspace/shard. type FilterByShard struct { @@ -394,3 +407,32 @@ func (fbk *FilterByKeyspace) IsIncluded(tablet *topodata.Tablet) bool { _, exist := fbk.keyspaces[tablet.Keyspace] return exist } + +// FilterByTabletTags is a filter that filters tablets by tablet tag key/values. +type FilterByTabletTags struct { + tags map[string]string +} + +// NewFilterByTabletTags creates a new FilterByTabletTags. All tablets that match +// all tablet tags will be forwarded to the TopologyWatcher's consumer. +func NewFilterByTabletTags(tabletTags map[string]string) *FilterByTabletTags { + return &FilterByTabletTags{ + tags: tabletTags, + } +} + +// IsIncluded returns true if the tablet's tags match what we expect. +func (fbtg *FilterByTabletTags) IsIncluded(tablet *topodata.Tablet) bool { + if fbtg.tags == nil { + return true + } + if tablet.Tags == nil { + return false + } + for key, val := range fbtg.tags { + if tabletVal, found := tablet.Tags[key]; !found || tabletVal != val { + return false + } + } + return true +} diff --git a/go/vt/discovery/topology_watcher_test.go b/go/vt/discovery/topology_watcher_test.go index dff8ba720c7..c372365626c 100644 --- a/go/vt/discovery/topology_watcher_test.go +++ b/go/vt/discovery/topology_watcher_test.go @@ -614,3 +614,30 @@ func TestFilterByKeypsaceSkipsIgnoredTablets(t *testing.T) { tw.Stop() } + +func TestNewFilterByTabletTags(t *testing.T) { + // no required tags == true + filter := NewFilterByTabletTags(nil) + assert.True(t, filter.IsIncluded(&topodatapb.Tablet{})) + + tags := map[string]string{ + "instance_type": "i3.xlarge", + "some_key": "some_value", + } + filter = NewFilterByTabletTags(tags) + + assert.False(t, filter.IsIncluded(&topodatapb.Tablet{ + Tags: nil, + })) + assert.False(t, filter.IsIncluded(&topodatapb.Tablet{ + Tags: map[string]string{}, + })) + assert.False(t, filter.IsIncluded(&topodatapb.Tablet{ + Tags: map[string]string{ + "instance_type": "i3.xlarge", + }, + })) + assert.True(t, filter.IsIncluded(&topodatapb.Tablet{ + Tags: tags, + })) +} diff --git a/go/vt/vtgate/vtgate.go b/go/vt/vtgate/vtgate.go index ccffa853000..b9252dc7b44 100644 --- a/go/vt/vtgate/vtgate.go +++ b/go/vt/vtgate/vtgate.go @@ -184,10 +184,10 @@ type VTGate struct { // stats objects. // TODO(sougou): This needs to be cleaned up. There // are global vars that depend on this member var. - timings *stats.MultiTimings - rowsReturned *stats.CountersWithMultiLabels - rowsAffected *stats.CountersWithMultiLabels - sqlTextCounts *stats.CountersWithMultiLabels + timings *stats.MultiTimings + rowsReturned *stats.CountersWithMultiLabels + rowsAffected *stats.CountersWithMultiLabels + queryTextCharsProcessed *stats.CountersWithMultiLabels // the throttled loggers for all errors, one per API entry logExecute *logutil.ThrottledLogger @@ -305,7 +305,7 @@ func Init( "VtgateApiRowsAffected", "Rows affected by a write (DML) operation through the VTgate API", []string{"Operation", "Keyspace", "DbType"}), - sqlTextCounts: stats.NewCountersWithMultiLabels( + queryTextCharsProcessed: stats.NewCountersWithMultiLabels( "VtgateSQLTextCounts", "Vtgate API query SQL text counts", []string{"Operation", "Keyspace", "DbType"}), @@ -438,7 +438,7 @@ func (vtg *VTGate) Execute(ctx context.Context, session *vtgatepb.Session, sql s if err == nil { vtg.rowsReturned.Add(statsKey, int64(len(qr.Rows))) vtg.rowsAffected.Add(statsKey, int64(qr.RowsAffected)) - vtg.sqlTextCounts.Add(statsKey, int64(len(sql))) + vtg.queryTextCharsProcessed.Add(statsKey, int64(len(sql))) return session, qr, nil } diff --git a/go/vt/vttablet/tabletserver/query_engine.go b/go/vt/vttablet/tabletserver/query_engine.go index 9c306bc447f..41d6192dffb 100644 --- a/go/vt/vttablet/tabletserver/query_engine.go +++ b/go/vt/vttablet/tabletserver/query_engine.go @@ -171,7 +171,7 @@ type QueryEngine struct { consolidatorMode sync2.AtomicString // stats - queryCounts, queryTimes, queryRowCounts, queryErrorCounts, queryRowsAffected, queryRowsReturned, querySQLTextCounts *stats.CountersWithMultiLabels + queryCounts, queryTimes, queryRowCounts, queryErrorCounts, queryRowsAffected, queryRowsReturned, queryTextCharsProcessed *stats.CountersWithMultiLabels // stats flags enablePerWorkloadTableMetrics bool @@ -261,7 +261,7 @@ func NewQueryEngine(env tabletenv.Env, se *schema.Engine) *QueryEngine { qe.queryRowCounts = env.Exporter().NewCountersWithMultiLabels("QueryRowCounts", "(DEPRECATED - use QueryRowsAffected and QueryRowsReturned instead) query row counts", labels) qe.queryRowsAffected = env.Exporter().NewCountersWithMultiLabels("QueryRowsAffected", "query rows affected", labels) qe.queryRowsReturned = env.Exporter().NewCountersWithMultiLabels("QueryRowsReturned", "query rows returned", labels) - qe.querySQLTextCounts = env.Exporter().NewCountersWithMultiLabels("QuerySQLTextCounts", "query sql text counts", labels) + qe.queryTextCharsProcessed = env.Exporter().NewCountersWithMultiLabels("QueryTextCharactersProcessed", "query text characters processed", labels) qe.queryErrorCounts = env.Exporter().NewCountersWithMultiLabels("QueryErrorCounts", "query error counts", labels) env.Exporter().HandleFunc("/debug/hotrows", qe.txSerializer.ServeHTTP) @@ -502,7 +502,7 @@ func (qe *QueryEngine) AddStats(plan *TabletPlan, tableName, workload string, qu qe.queryRowCounts.Add(keys, rowsAffected) qe.queryErrorCounts.Add(keys, errorCount) if plan.FullQuery != nil { - qe.querySQLTextCounts.Add(keys, int64(len(plan.FullQuery.Query))) + qe.queryTextCharsProcessed.Add(keys, int64(len(plan.FullQuery.Query))) } // For certain plan types like select, we only want to add their metrics to rows returned diff --git a/go/vt/vttablet/tabletserver/query_engine_test.go b/go/vt/vttablet/tabletserver/query_engine_test.go index 093f8971141..db280a6764f 100644 --- a/go/vt/vttablet/tabletserver/query_engine_test.go +++ b/go/vt/vttablet/tabletserver/query_engine_test.go @@ -586,178 +586,178 @@ func TestAddQueryStats(t *testing.T) { }, } testcases := []struct { - name string - plan *TabletPlan - tableName string - queryCount int64 - duration time.Duration - mysqlTime time.Duration - rowsAffected int64 - rowsReturned int64 - errorCount int64 - errorCode string - enablePerWorkloadTableMetrics bool - workload string - expectedQueryCounts string - expectedQueryTimes string - expectedQueryRowsAffected string - expectedQueryRowsReturned string - expectedQuerySQLTextCounts string - expectedQueryErrorCounts string - expectedQueryRowCounts string + name string + plan *TabletPlan + tableName string + queryCount int64 + duration time.Duration + mysqlTime time.Duration + rowsAffected int64 + rowsReturned int64 + errorCount int64 + errorCode string + enablePerWorkloadTableMetrics bool + workload string + expectedQueryCounts string + expectedQueryTimes string + expectedQueryRowsAffected string + expectedQueryRowsReturned string + expectedQueryTextCharsProcessed string + expectedQueryErrorCounts string + expectedQueryRowCounts string }{ { - name: "select query", - plan: fakeSelectPlan, - tableName: "A", - queryCount: 1, - duration: 10, - rowsAffected: 0, - rowsReturned: 15, - errorCount: 0, - errorCode: "OK", - enablePerWorkloadTableMetrics: false, - workload: "some-workload", - expectedQueryCounts: `{"A.Select": 1}`, - expectedQueryTimes: `{"A.Select": 10}`, - expectedQueryRowsAffected: `{}`, - expectedQueryRowsReturned: `{"A.Select": 15}`, - expectedQuerySQLTextCounts: `{"A.Select": 43}`, - expectedQueryRowCounts: `{"A.Select": 0}`, - expectedQueryErrorCounts: `{"A.Select": 0}`, + name: "select query", + plan: fakeSelectPlan, + tableName: "A", + queryCount: 1, + duration: 10, + rowsAffected: 0, + rowsReturned: 15, + errorCount: 0, + errorCode: "OK", + enablePerWorkloadTableMetrics: false, + workload: "some-workload", + expectedQueryCounts: `{"A.Select": 1}`, + expectedQueryTimes: `{"A.Select": 10}`, + expectedQueryRowsAffected: `{}`, + expectedQueryRowsReturned: `{"A.Select": 15}`, + expectedQueryTextCharsProcessed: `{"A.Select": 43}`, + expectedQueryRowCounts: `{"A.Select": 0}`, + expectedQueryErrorCounts: `{"A.Select": 0}`, }, { - name: "select into query", - plan: fakeSelectPlan, - tableName: "A", - queryCount: 1, - duration: 10, - rowsAffected: 15, - rowsReturned: 0, - errorCount: 0, - errorCode: "OK", - enablePerWorkloadTableMetrics: false, - workload: "some-workload", - expectedQueryCounts: `{"A.Select": 1}`, - expectedQueryTimes: `{"A.Select": 10}`, - expectedQueryRowsAffected: `{"A.Select": 15}`, - expectedQueryRowsReturned: `{"A.Select": 0}`, - expectedQuerySQLTextCounts: `{"A.Select": 43}`, - expectedQueryRowCounts: `{"A.Select": 15}`, - expectedQueryErrorCounts: `{"A.Select": 0}`, + name: "select into query", + plan: fakeSelectPlan, + tableName: "A", + queryCount: 1, + duration: 10, + rowsAffected: 15, + rowsReturned: 0, + errorCount: 0, + errorCode: "OK", + enablePerWorkloadTableMetrics: false, + workload: "some-workload", + expectedQueryCounts: `{"A.Select": 1}`, + expectedQueryTimes: `{"A.Select": 10}`, + expectedQueryRowsAffected: `{"A.Select": 15}`, + expectedQueryRowsReturned: `{"A.Select": 0}`, + expectedQueryTextCharsProcessed: `{"A.Select": 43}`, + expectedQueryRowCounts: `{"A.Select": 15}`, + expectedQueryErrorCounts: `{"A.Select": 0}`, }, { - name: "error", - plan: fakeSelectPlan, - tableName: "A", - queryCount: 1, - duration: 10, - rowsAffected: 0, - rowsReturned: 0, - errorCount: 1, - errorCode: "RESOURCE_EXHAUSTED", - enablePerWorkloadTableMetrics: false, - workload: "some-workload", - expectedQueryCounts: `{"A.Select": 1}`, - expectedQueryTimes: `{"A.Select": 10}`, - expectedQueryRowsAffected: `{}`, - expectedQueryRowsReturned: `{"A.Select": 0}`, - expectedQuerySQLTextCounts: `{"A.Select": 43}`, - expectedQueryRowCounts: `{"A.Select": 0}`, - expectedQueryErrorCounts: `{"A.Select": 1}`, + name: "error", + plan: fakeSelectPlan, + tableName: "A", + queryCount: 1, + duration: 10, + rowsAffected: 0, + rowsReturned: 0, + errorCount: 1, + errorCode: "RESOURCE_EXHAUSTED", + enablePerWorkloadTableMetrics: false, + workload: "some-workload", + expectedQueryCounts: `{"A.Select": 1}`, + expectedQueryTimes: `{"A.Select": 10}`, + expectedQueryRowsAffected: `{}`, + expectedQueryRowsReturned: `{"A.Select": 0}`, + expectedQueryTextCharsProcessed: `{"A.Select": 43}`, + expectedQueryRowCounts: `{"A.Select": 0}`, + expectedQueryErrorCounts: `{"A.Select": 1}`, }, { - name: "insert query", - plan: fakeInsertPlan, - tableName: "A", - queryCount: 1, - duration: 10, - rowsAffected: 15, - rowsReturned: 0, - errorCount: 0, - errorCode: "OK", - enablePerWorkloadTableMetrics: false, - workload: "some-workload", - expectedQueryCounts: `{"A.Insert": 1}`, - expectedQueryTimes: `{"A.Insert": 10}`, - expectedQueryRowsAffected: `{"A.Insert": 15}`, - expectedQueryRowsReturned: `{}`, - expectedQuerySQLTextCounts: `{"A.Insert": 59}`, - expectedQueryRowCounts: `{"A.Insert": 15}`, - expectedQueryErrorCounts: `{"A.Insert": 0}`, + name: "insert query", + plan: fakeInsertPlan, + tableName: "A", + queryCount: 1, + duration: 10, + rowsAffected: 15, + rowsReturned: 0, + errorCount: 0, + errorCode: "OK", + enablePerWorkloadTableMetrics: false, + workload: "some-workload", + expectedQueryCounts: `{"A.Insert": 1}`, + expectedQueryTimes: `{"A.Insert": 10}`, + expectedQueryRowsAffected: `{"A.Insert": 15}`, + expectedQueryRowsReturned: `{}`, + expectedQueryTextCharsProcessed: `{"A.Insert": 59}`, + expectedQueryRowCounts: `{"A.Insert": 15}`, + expectedQueryErrorCounts: `{"A.Insert": 0}`, }, { - name: "select query with per workload metrics", - plan: fakeSelectPlan, - tableName: "A", - queryCount: 1, - duration: 10, - rowsAffected: 0, - rowsReturned: 15, - errorCount: 0, - errorCode: "OK", - enablePerWorkloadTableMetrics: true, - workload: "some-workload", - expectedQueryCounts: `{"A.Select.some-workload": 1}`, - expectedQueryTimes: `{"A.Select.some-workload": 10}`, - expectedQueryRowsAffected: `{}`, - expectedQueryRowsReturned: `{"A.Select.some-workload": 15}`, - expectedQuerySQLTextCounts: `{"A.Select.some-workload": 43}`, - expectedQueryRowCounts: `{"A.Select.some-workload": 0}`, - expectedQueryErrorCounts: `{"A.Select.some-workload": 0}`, + name: "select query with per workload metrics", + plan: fakeSelectPlan, + tableName: "A", + queryCount: 1, + duration: 10, + rowsAffected: 0, + rowsReturned: 15, + errorCount: 0, + errorCode: "OK", + enablePerWorkloadTableMetrics: true, + workload: "some-workload", + expectedQueryCounts: `{"A.Select.some-workload": 1}`, + expectedQueryTimes: `{"A.Select.some-workload": 10}`, + expectedQueryRowsAffected: `{}`, + expectedQueryRowsReturned: `{"A.Select.some-workload": 15}`, + expectedQueryTextCharsProcessed: `{"A.Select.some-workload": 43}`, + expectedQueryRowCounts: `{"A.Select.some-workload": 0}`, + expectedQueryErrorCounts: `{"A.Select.some-workload": 0}`, }, { - name: "select into query with per workload metrics", - plan: fakeSelectPlan, - tableName: "A", - queryCount: 1, - duration: 10, - rowsAffected: 15, - rowsReturned: 0, - errorCount: 0, - errorCode: "OK", - enablePerWorkloadTableMetrics: true, - workload: "some-workload", - expectedQueryCounts: `{"A.Select.some-workload": 1}`, - expectedQueryTimes: `{"A.Select.some-workload": 10}`, - expectedQueryRowsAffected: `{"A.Select.some-workload": 15}`, - expectedQueryRowsReturned: `{"A.Select.some-workload": 0}`, - expectedQuerySQLTextCounts: `{"A.Select.some-workload": 43}`, - expectedQueryRowCounts: `{"A.Select.some-workload": 15}`, - expectedQueryErrorCounts: `{"A.Select.some-workload": 0}`, + name: "select into query with per workload metrics", + plan: fakeSelectPlan, + tableName: "A", + queryCount: 1, + duration: 10, + rowsAffected: 15, + rowsReturned: 0, + errorCount: 0, + errorCode: "OK", + enablePerWorkloadTableMetrics: true, + workload: "some-workload", + expectedQueryCounts: `{"A.Select.some-workload": 1}`, + expectedQueryTimes: `{"A.Select.some-workload": 10}`, + expectedQueryRowsAffected: `{"A.Select.some-workload": 15}`, + expectedQueryRowsReturned: `{"A.Select.some-workload": 0}`, + expectedQueryTextCharsProcessed: `{"A.Select.some-workload": 43}`, + expectedQueryRowCounts: `{"A.Select.some-workload": 15}`, + expectedQueryErrorCounts: `{"A.Select.some-workload": 0}`, }, { - name: "error with per workload metrics", - plan: fakeSelectPlan, - tableName: "A", - queryCount: 1, - duration: 10, - rowsAffected: 0, - rowsReturned: 0, - errorCount: 1, - errorCode: "RESOURCE_EXHAUSTED", - enablePerWorkloadTableMetrics: true, - workload: "some-workload", - expectedQueryCounts: `{"A.Select.some-workload": 1}`, - expectedQueryTimes: `{"A.Select.some-workload": 10}`, - expectedQueryRowsAffected: `{}`, - expectedQueryRowsReturned: `{"A.Select.some-workload": 0}`, - expectedQuerySQLTextCounts: `{"A.Select.some-workload": 43}`, - expectedQueryRowCounts: `{"A.Select.some-workload": 0}`, - expectedQueryErrorCounts: `{"A.Select.some-workload": 1}`, + name: "error with per workload metrics", + plan: fakeSelectPlan, + tableName: "A", + queryCount: 1, + duration: 10, + rowsAffected: 0, + rowsReturned: 0, + errorCount: 1, + errorCode: "RESOURCE_EXHAUSTED", + enablePerWorkloadTableMetrics: true, + workload: "some-workload", + expectedQueryCounts: `{"A.Select.some-workload": 1}`, + expectedQueryTimes: `{"A.Select.some-workload": 10}`, + expectedQueryRowsAffected: `{}`, + expectedQueryRowsReturned: `{"A.Select.some-workload": 0}`, + expectedQueryTextCharsProcessed: `{"A.Select.some-workload": 43}`, + expectedQueryRowCounts: `{"A.Select.some-workload": 0}`, + expectedQueryErrorCounts: `{"A.Select.some-workload": 1}`, }, { - name: "insert query with per workload metrics", - plan: fakeInsertPlan, - tableName: "A", - queryCount: 1, - duration: 10, - rowsAffected: 15, - rowsReturned: 0, - errorCount: 0, - errorCode: "OK", - enablePerWorkloadTableMetrics: true, - workload: "some-workload", - expectedQueryCounts: `{"A.Insert.some-workload": 1}`, - expectedQueryTimes: `{"A.Insert.some-workload": 10}`, - expectedQueryRowsAffected: `{"A.Insert.some-workload": 15}`, - expectedQueryRowsReturned: `{}`, - expectedQuerySQLTextCounts: `{"A.Insert.some-workload": 59}`, - expectedQueryRowCounts: `{"A.Insert.some-workload": 15}`, - expectedQueryErrorCounts: `{"A.Insert.some-workload": 0}`, + name: "insert query with per workload metrics", + plan: fakeInsertPlan, + tableName: "A", + queryCount: 1, + duration: 10, + rowsAffected: 15, + rowsReturned: 0, + errorCount: 0, + errorCode: "OK", + enablePerWorkloadTableMetrics: true, + workload: "some-workload", + expectedQueryCounts: `{"A.Insert.some-workload": 1}`, + expectedQueryTimes: `{"A.Insert.some-workload": 10}`, + expectedQueryRowsAffected: `{"A.Insert.some-workload": 15}`, + expectedQueryRowsReturned: `{}`, + expectedQueryTextCharsProcessed: `{"A.Insert.some-workload": 59}`, + expectedQueryRowCounts: `{"A.Insert.some-workload": 15}`, + expectedQueryErrorCounts: `{"A.Insert.some-workload": 0}`, }, } @@ -776,7 +776,7 @@ func TestAddQueryStats(t *testing.T) { assert.Equal(t, testcase.expectedQueryRowsAffected, qe.queryRowsAffected.String()) assert.Equal(t, testcase.expectedQueryRowsReturned, qe.queryRowsReturned.String()) assert.Equal(t, testcase.expectedQueryRowCounts, qe.queryRowCounts.String()) - assert.Equal(t, testcase.expectedQuerySQLTextCounts, qe.querySQLTextCounts.String()) + assert.Equal(t, testcase.expectedQueryTextCharsProcessed, qe.queryTextCharsProcessed.String()) assert.Equal(t, testcase.expectedQueryErrorCounts, qe.queryErrorCounts.String()) }) }