Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
125152: sql: extend EXPLAIN ANALYZE for follower reads and AOST r=yuzefovich a=yuzefovich

**sql: extend EXPLAIN ANALYZE to indicate that follower reads were used**

This commit extends EXPLAIN ANALYZE output for KV-reading operators to indicate whether the follower reads were used. This is achieved by reusing existing `ScanStats` infrastructure by adding a "marker" protobuf message. Note that we couldn't directly reuse the same `kvpb.ScanStats` object as the one we create in `evaluateBatch` since the determination whether the follower reads are eligible is done at a different point in time. We could have extended that protobuf and created a fresh object, but that seems a bit wrong.

Co-authored with `@Uzair5162.`

Release note (sql change): New field "used follower read" is added to EXPLAIN ANALYZE output to SQL operators whenever their reads were served by the follower replicas (previously, this information was only available in the trace).

**sql: extend EXPLAIN ANALYZE for "historical" reads**

This commit adds `historical: AS OF SYSTEM TIME ...` top-level attribute to EXPLAIN ANALYZE whenever the txn is historical (meaning AS OF SYSTEM TIME was specified either in the stmt or at the txn level).

Release note (sql change): New attribute `historical: AS OF SYSTEM TIME ...` is now included into EXPLAIN ANALYZE output whenever the query performs historical reads.

Fixes: cockroachdb#99984.

125431: roachtest: exclude .perf directories from artifacts.zip r=DarrylWong,renatolabs a=srosenberg

In cockroachdb#125022, we excluded stats.json files from being zipped. However, because the stats.json file lives in a .perf dir this dir is found first by filterDirEntries and zipped. This change instead excludes the .perf directory from being zipped.

Epic: none
Fixes: none

Release note: None

Co-authored-by: Yahor Yuzefovich <yahor@cockroachlabs.com>
Co-authored-by: Stan Rosenberg <stan.rosenberg@gmail.com>
  • Loading branch information
3 people committed Jun 10, 2024
3 parents 36905d3 + 1c0b408 + c01cf9b commit 41c37cb
Show file tree
Hide file tree
Showing 34 changed files with 221 additions and 83 deletions.
1 change: 1 addition & 0 deletions docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -3025,6 +3025,7 @@ contains common SQL event/execution details.
| `SQLInstanceIDs` | SQLInstanceIDs is a list of all the SQL instances used in this statement's execution. | no |
| `KVNodeIDs` | KVNodeIDs is a list of all the KV nodes used in this statement's execution. | no |
| `StatementFingerprintID` | Statement fingerprint ID of the query. | no |
| `UsedFollowerRead` | UsedFollowerRead indicates whether at least some reads were served by the follower replicas. | no |


#### Common fields
Expand Down
22 changes: 21 additions & 1 deletion pkg/ccl/kvccl/kvfollowerreadsccl/followerreads_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/randutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -1018,6 +1019,7 @@ func TestSecondaryTenantFollowerReadsRouting(t *testing.T) {
skip.UnderRace(t, "times out")
skip.UnderDeadlock(t)

rng, _ := randutil.NewTestRand()
for _, testCase := range []struct {
name string
sharedProcess bool
Expand Down Expand Up @@ -1078,6 +1080,10 @@ func TestSecondaryTenantFollowerReadsRouting(t *testing.T) {
systemSQL.Exec(t, `SET CLUSTER SETTING kv.closed_timestamp.side_transport_interval = '0.1s'`)

historicalQuery := `SELECT * FROM t.test AS OF SYSTEM TIME follower_read_timestamp() WHERE k=2`
useExplainAnalyze := rng.Float64() < 0.5
if useExplainAnalyze {
historicalQuery = "EXPLAIN ANALYZE " + historicalQuery
}
recCh := make(chan tracingpb.Recording, 1)

var tenants [numNodes]serverutils.ApplicationLayerInterface
Expand Down Expand Up @@ -1211,7 +1217,7 @@ func TestSecondaryTenantFollowerReadsRouting(t *testing.T) {
{NodeID: 3, StoreID: 3, ReplicaID: 3},
}, entry.Desc.Replicas().Descriptors())

tenantSQL.Exec(t, historicalQuery)
rows := tenantSQL.QueryStr(t, historicalQuery)
rec := <-recCh

// Look at the trace and check that the follower read was served by
Expand All @@ -1229,6 +1235,20 @@ func TestSecondaryTenantFollowerReadsRouting(t *testing.T) {
}
require.Equal(t, numFRs, 1, "query wasn't served through follower reads: %s", rec)
require.Equal(t, numN2FRs, 1, "follower read wasn't served by n2: %s", rec)

if useExplainAnalyze {
frMessage, historicalMessage := "used follower read", "historical"
var foundFRMessage, foundHistoricalMessage bool
for _, row := range rows {
if strings.TrimSpace(row[0]) == frMessage {
foundFRMessage = true
} else if strings.HasPrefix(strings.TrimSpace(row[0]), historicalMessage) {
foundHistoricalMessage = true
}
}
require.True(t, foundFRMessage, "didn't see %q message in EXPLAIN ANALYZE: %v", frMessage, rows)
require.True(t, foundHistoricalMessage, "didn't see %q message in EXPLAIN ANALYZE: %v", historicalMessage, rows)
}
})
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -218,10 +218,10 @@ SELECT * FROM crdb_internal.leases WHERE node_id < 0
node_id table_id name parent_id expiration deleted


query ITTTTTIIITTRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRBBTTTTTTRRRRRI colnames
query ITTTTTIIITTRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRBBTTTTBTTRRRRRI colnames
SELECT * FROM crdb_internal.node_statement_statistics WHERE node_id < 0
----
node_id application_name flags statement_id key anonymized count first_attempt_count max_retries last_error last_error_code rows_avg rows_var idle_lat_avg idle_lat_var parse_lat_avg parse_lat_var plan_lat_avg plan_lat_var run_lat_avg run_lat_var service_lat_avg service_lat_var overhead_lat_avg overhead_lat_var bytes_read_avg bytes_read_var rows_read_avg rows_read_var rows_written_avg rows_written_var network_bytes_avg network_bytes_var network_msgs_avg network_msgs_var max_mem_usage_avg max_mem_usage_var max_disk_usage_avg max_disk_usage_var contention_time_avg contention_time_var cpu_sql_nanos_avg cpu_sql_nanos_var mvcc_step_avg mvcc_step_var mvcc_step_internal_avg mvcc_step_internal_var mvcc_seek_avg mvcc_seek_var mvcc_seek_internal_avg mvcc_seek_internal_var mvcc_block_bytes_avg mvcc_block_bytes_var mvcc_block_bytes_in_cache_avg mvcc_block_bytes_in_cache_var mvcc_key_bytes_avg mvcc_key_bytes_var mvcc_value_bytes_avg mvcc_value_bytes_var mvcc_point_count_avg mvcc_point_count_var mvcc_points_covered_by_range_tombstones_avg mvcc_points_covered_by_range_tombstones_var mvcc_range_key_count_avg mvcc_range_key_count_var mvcc_range_key_contained_points_avg mvcc_range_key_contained_points_var mvcc_range_key_skipped_points_avg mvcc_range_key_skipped_points_var implicit_txn full_scan sample_plan database_name exec_node_ids kv_node_ids txn_fingerprint_id index_recommendations latency_seconds_min latency_seconds_max latency_seconds_p50 latency_seconds_p90 latency_seconds_p99 failure_count
node_id application_name flags statement_id key anonymized count first_attempt_count max_retries last_error last_error_code rows_avg rows_var idle_lat_avg idle_lat_var parse_lat_avg parse_lat_var plan_lat_avg plan_lat_var run_lat_avg run_lat_var service_lat_avg service_lat_var overhead_lat_avg overhead_lat_var bytes_read_avg bytes_read_var rows_read_avg rows_read_var rows_written_avg rows_written_var network_bytes_avg network_bytes_var network_msgs_avg network_msgs_var max_mem_usage_avg max_mem_usage_var max_disk_usage_avg max_disk_usage_var contention_time_avg contention_time_var cpu_sql_nanos_avg cpu_sql_nanos_var mvcc_step_avg mvcc_step_var mvcc_step_internal_avg mvcc_step_internal_var mvcc_seek_avg mvcc_seek_var mvcc_seek_internal_avg mvcc_seek_internal_var mvcc_block_bytes_avg mvcc_block_bytes_var mvcc_block_bytes_in_cache_avg mvcc_block_bytes_in_cache_var mvcc_key_bytes_avg mvcc_key_bytes_var mvcc_value_bytes_avg mvcc_value_bytes_var mvcc_point_count_avg mvcc_point_count_var mvcc_points_covered_by_range_tombstones_avg mvcc_points_covered_by_range_tombstones_var mvcc_range_key_count_avg mvcc_range_key_count_var mvcc_range_key_contained_points_avg mvcc_range_key_contained_points_var mvcc_range_key_skipped_points_avg mvcc_range_key_skipped_points_var implicit_txn full_scan sample_plan database_name exec_node_ids kv_node_ids used_follower_read txn_fingerprint_id index_recommendations latency_seconds_min latency_seconds_max latency_seconds_p50 latency_seconds_p90 latency_seconds_p99 failure_count

query ITTTIIRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRRR colnames
SELECT * FROM crdb_internal.node_transaction_statistics WHERE node_id < 0
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/zip_table_registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -990,6 +990,7 @@ var zipInternalTablesPerNode = DebugZipTableRegistry{
"database_name",
"exec_node_ids",
"kv_node_ids",
"used_follower_read",
"txn_fingerprint_id",
"index_recommendations",
"latency_seconds_min",
Expand Down
7 changes: 5 additions & 2 deletions pkg/cmd/roachtest/test_runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -1859,8 +1859,11 @@ func zipArtifacts(t *testImpl) error {
// However, if the order is reversed, or 'stats.json' is created directly on the test runner node,
// it will be moved to the zip archive. The corresponding CI script (build/teamcity/util/roachtest_util.sh) will
// then fail to find 'stats.json' in the artifacts directory, and the roachperf dashboard will be looking rather sad.
if !entry.IsDir() && entry.Name() == "stats.json" {
// Skip any 'stats.json' files.
if (!entry.IsDir() && entry.Name() == "stats.json") ||
// N.B. performance artifacts are expected to be in a directory of the form "2.perf",
// where 2 is node id; see `getPerfArtifacts`.
(entry.IsDir() && strings.HasSuffix(entry.Name(), "."+t.PerfArtifactsDir())) {
// Skip 'stats.json' and directories ending in '.perf'.
return false
}
return true
Expand Down
84 changes: 51 additions & 33 deletions pkg/cmd/roachtest/zip_util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,64 +21,82 @@ import (
)

func TestMoveToZipArchive(t *testing.T) {
dir := t.TempDir()
p := func(elems ...string) string {
return filepath.Join(append([]string{dir}, elems...)...)
}
require.NoError(t, os.WriteFile(p("a1"), []byte("foo"), 0777))
require.NoError(t, os.WriteFile(p("a2"), []byte("foo"), 0777))
require.NoError(t, os.WriteFile(p("a3"), []byte("foo"), 0777))
require.NoError(t, os.Mkdir(p("dir1"), 0777))
require.NoError(t, os.WriteFile(p("dir1", "file1"), []byte("foo"), 0777))
require.NoError(t, os.WriteFile(p("dir1", "file2"), []byte("foo"), 0777))
require.NoError(t, os.Mkdir(p("dir2"), 0777))
require.NoError(t, os.WriteFile(p("dir2", "file1"), []byte("foo"), 0777))
require.NoError(t, os.WriteFile(p("dir2", "file2"), []byte("foo"), 0777))
baseDir := t.TempDir()
require.NoError(t, os.WriteFile(p(baseDir, "a1"), []byte("foo"), 0777))
require.NoError(t, os.WriteFile(p(baseDir, "a2"), []byte("foo"), 0777))
require.NoError(t, os.WriteFile(p(baseDir, "a3"), []byte("foo"), 0777))
require.NoError(t, os.Mkdir(p(baseDir, "dir1"), 0777))
require.NoError(t, os.WriteFile(p(baseDir, "dir1", "file1"), []byte("foo"), 0777))
require.NoError(t, os.WriteFile(p(baseDir, "dir1", "file2"), []byte("foo"), 0777))
require.NoError(t, os.Mkdir(p(baseDir, "dir2"), 0777))
require.NoError(t, os.WriteFile(p(baseDir, "dir2", "file1"), []byte("foo"), 0777))
require.NoError(t, os.WriteFile(p(baseDir, "dir2", "file2"), []byte("foo"), 0777))

// expectLs checks the current directory listing of dir.
expectLs := func(expected ...string) {
t.Helper()
var actual []string
require.NoError(t, filepath.Walk(dir, func(path string, info os.FileInfo, err error) error {
require.NoError(t, filepath.Walk(baseDir, func(path string, info os.FileInfo, err error) error {
require.NoError(t, err)
if !info.IsDir() {
rel, err := filepath.Rel(dir, path)
rel, err := filepath.Rel(baseDir, path)
require.NoError(t, err)
actual = append(actual, rel)
}
return nil
}))
require.Equal(t, expected, actual)
}
// expectZip checks the files contained in the given archive; paths must use
// slashes.
expectZip := func(archiveName string, expected ...string) {
r, err := zip.OpenReader(p(archiveName))
require.NoError(t, err)
var actual []string
for _, f := range r.File {
actual = append(actual, f.Name)
}
require.Equal(t, expected, actual)
require.NoError(t, r.Close())
}
j := filepath.Join
expectLs("a1", "a2", "a3", j("dir1", "file1"), j("dir1", "file2"), j("dir2", "file1"), j("dir2", "file2"))

list, err := filterDirEntries(dir, func(entry os.DirEntry) bool {
list, err := filterDirEntries(baseDir, func(entry os.DirEntry) bool {
return entry.Name() != "a2" && entry.Name() != "dir2"
})
require.NoError(t, err)
require.Equal(t, []string{"a1", "a3", "dir1"}, list)
require.NoError(t, moveToZipArchive("first.zip", dir, list...))
expectZip("first.zip", "a1", "a3", "dir1/file1", "dir1/file2")
require.NoError(t, moveToZipArchive("first.zip", baseDir, list...))
expectZip(t, baseDir, "first.zip", "a1", "a3", "dir1/file1", "dir1/file2")
expectLs("a2", j("dir2", "file1"), j("dir2", "file2"), "first.zip")

list, err = filterDirEntries(dir, func(entry os.DirEntry) bool {
list, err = filterDirEntries(baseDir, func(entry os.DirEntry) bool {
return !strings.HasSuffix(entry.Name(), ".zip")
})
require.NoError(t, err)
require.NoError(t, moveToZipArchive("second.zip", dir, list...))
expectZip("second.zip", "a2", "dir2/file1", "dir2/file2")
require.NoError(t, moveToZipArchive("second.zip", baseDir, list...))
expectZip(t, baseDir, "second.zip", "a2", "dir2/file1", "dir2/file2")
expectLs("first.zip", "second.zip")
}

func TestZipArtifacts(t *testing.T) {
tmp := t.TempDir()
artifactsDir := filepath.Join(tmp, "someTestRun")
require.NoError(t, os.Mkdir(artifactsDir, 0777))
require.NoError(t, os.WriteFile(filepath.Join(artifactsDir, "test.log"), []byte("foobar"), 0777))
require.NoError(t, os.WriteFile(filepath.Join(artifactsDir, "test-teardown.log"), []byte{}, 0777))
perfArtifactsDir := filepath.Join(artifactsDir, "1.perf")
require.NoError(t, os.Mkdir(perfArtifactsDir, 0777))
require.NoError(t, os.WriteFile(filepath.Join(perfArtifactsDir, "stats.json"), []byte("{}"), 0777))
tt := testImpl{
artifactsDir: artifactsDir,
}
require.NoError(t, zipArtifacts(&tt))
expectZip(t, artifactsDir, "artifacts.zip", "test.log", "test-teardown.log")
}

// expectZip checks the files contained in the given archive; paths must use
// slashes.
func expectZip(t *testing.T, basedir string, archiveName string, expected ...string) {
r, err := zip.OpenReader(p(basedir, archiveName))
require.NoError(t, err)
var actual []string
for _, f := range r.File {
actual = append(actual, f.Name)
}
require.ElementsMatch(t, expected, actual)
require.NoError(t, r.Close())
}

func p(baseDir string, elems ...string) string {
return filepath.Join(append([]string{baseDir}, elems...)...)
}
4 changes: 4 additions & 0 deletions pkg/kv/kvpb/api.proto
Original file line number Diff line number Diff line change
Expand Up @@ -3714,3 +3714,7 @@ message ScanStats {
// Region, if set, is the region of the KV server that evaluated the request.
string region = 22;
}

// UsedFollowerRead indicates whether at least some reads were served by the
// follower replicas.
message UsedFollowerRead {}
5 changes: 5 additions & 0 deletions pkg/kv/kvserver/replica_follower_read.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/redact"
)

Expand Down Expand Up @@ -126,6 +128,9 @@ func (r *Replica) canServeFollowerReadRLocked(ctx context.Context, ba *kvpb.Batc
// serve reads for that and smaller timestamps forever.
log.Eventf(ctx, "%s; query timestamp below closed timestamp by %s", redact.Safe(kvbase.FollowerReadServingMsg), -tsDiff)
r.store.metrics.FollowerReadsCount.Inc(1)
if sp := tracing.SpanFromContext(ctx); sp.RecordingType() != tracingpb.RecordingOff {
sp.RecordStructured(&kvpb.UsedFollowerRead{})
}
return true
}

Expand Down
44 changes: 23 additions & 21 deletions pkg/server/application_api/sql_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1641,26 +1641,27 @@ func generateStatisticsColumn(

// Create stats JSON
stats := struct {
BytesRead appstatspb.NumericStat `json:"bytesRead"`
Cnt int64 `json:"cnt"`
FirstAttemptCnt int64 `json:"firstAttemptCnt"`
IdleLat appstatspb.NumericStat `json:"idleLat"`
Indexes []string `json:"indexes"`
LastErrorCode string `json:"lastErrorCode"`
LastExecAt time.Time `json:"lastExecAt"`
MaxRetries int `json:"maxRetries"`
Nodes []int64 `json:"nodes"`
KVNodeIDs []int32 `json:"kvNodeIds"`
NumRows appstatspb.NumericStat `json:"numRows"`
OvhLat appstatspb.NumericStat `json:"ovhLat"`
ParseLat appstatspb.NumericStat `json:"parseLat"`
PlanGists []string `json:"planGists"`
PlanLat appstatspb.NumericStat `json:"planLat"`
Regions []string `json:"regions"`
RowsRead appstatspb.NumericStat `json:"rowsRead"`
RowsWritten appstatspb.NumericStat `json:"rowsWritten"`
RunLat appstatspb.NumericStat `json:"runLat"`
SvcLat appstatspb.NumericStat `json:"svcLat"`
BytesRead appstatspb.NumericStat `json:"bytesRead"`
Cnt int64 `json:"cnt"`
FirstAttemptCnt int64 `json:"firstAttemptCnt"`
IdleLat appstatspb.NumericStat `json:"idleLat"`
Indexes []string `json:"indexes"`
LastErrorCode string `json:"lastErrorCode"`
LastExecAt time.Time `json:"lastExecAt"`
MaxRetries int `json:"maxRetries"`
Nodes []int64 `json:"nodes"`
KVNodeIDs []int32 `json:"kvNodeIds"`
NumRows appstatspb.NumericStat `json:"numRows"`
OvhLat appstatspb.NumericStat `json:"ovhLat"`
ParseLat appstatspb.NumericStat `json:"parseLat"`
PlanGists []string `json:"planGists"`
PlanLat appstatspb.NumericStat `json:"planLat"`
Regions []string `json:"regions"`
UsedFollowerRead bool `json:"usedFollowerRead"`
RowsRead appstatspb.NumericStat `json:"rowsRead"`
RowsWritten appstatspb.NumericStat `json:"rowsWritten"`
RunLat appstatspb.NumericStat `json:"runLat"`
SvcLat appstatspb.NumericStat `json:"svcLat"`
}{
BytesRead: appstatspb.NumericStat{
Mean: 0,
Expand Down Expand Up @@ -1695,7 +1696,8 @@ func generateStatisticsColumn(
Mean: 0,
SquaredDiffs: 0,
},
Regions: statement.Stats.Regions,
Regions: statement.Stats.Regions,
UsedFollowerRead: statement.Stats.UsedFollowerRead,
RowsRead: appstatspb.NumericStat{
Mean: 0,
SquaredDiffs: 0,
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/appstatspb/app_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -189,6 +189,7 @@ func (s *StatementStatistics) Add(other *StatementStatistics) {
s.Nodes = util.CombineUnique(s.Nodes, other.Nodes)
s.KVNodeIDs = util.CombineUnique(s.KVNodeIDs, other.KVNodeIDs)
s.Regions = util.CombineUnique(s.Regions, other.Regions)
s.UsedFollowerRead = s.UsedFollowerRead || other.UsedFollowerRead
s.PlanGists = util.CombineUnique(s.PlanGists, other.PlanGists)
s.Indexes = util.CombineUnique(s.Indexes, other.Indexes)
s.ExecStats.Add(other.ExecStats)
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/appstatspb/app_stats.proto
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,10 @@ message StatementStatistics {
// failure_count is the count of failed executions for a given statement fingerprint.
optional int64 failure_count = 33 [(gogoproto.nullable) = false];

// UsedFollowerRead indicates whether at least some reads were served by the
// follower replicas.
optional bool used_follower_read = 35 [(gogoproto.nullable) = false];

// Note: be sure to update `sql/app_stats.go` when adding/removing fields here!

reserved 13, 14, 17, 18, 19, 20;
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -1838,6 +1838,7 @@ CREATE TABLE crdb_internal.node_statement_statistics (
database_name STRING NOT NULL,
exec_node_ids INT[] NOT NULL,
kv_node_ids INT[] NOT NULL,
used_follower_read BOOL NOT NULL,
txn_fingerprint_id STRING,
index_recommendations STRING[] NOT NULL,
latency_seconds_min FLOAT,
Expand Down Expand Up @@ -1986,8 +1987,9 @@ CREATE TABLE crdb_internal.node_statement_statistics (
tree.MakeDBool(tree.DBool(stats.Key.FullScan)), // full_scan
alloc.NewDJSON(tree.DJSON{JSON: samplePlan}), // sample_plan
alloc.NewDString(tree.DString(stats.Key.Database)), // database_name
execNodeIDs, // exec_node_ids
kvNodeIDs, // kv_node_ids
execNodeIDs, // exec_node_ids
kvNodeIDs, // kv_node_ids
tree.MakeDBool(tree.DBool(stats.Stats.UsedFollowerRead)), // used_follower_read
txnFingerprintID, // txn_fingerprint_id
indexRecommendations, // index_recommendations
alloc.NewDFloat(tree.DFloat(stats.Stats.LatencyInfo.Min)), // latency_seconds_min
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/exec_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -401,6 +401,7 @@ func (p *planner) maybeLogStatementInternal(
Regions: queryLevelStats.Regions,
SQLInstanceIDs: queryLevelStats.SQLInstanceIDs,
KVNodeIDs: queryLevelStats.KVNodeIDs,
UsedFollowerRead: queryLevelStats.UsedFollowerRead,
NetworkBytesSent: queryLevelStats.NetworkBytesSent,
MaxMemUsage: queryLevelStats.MaxMemUsage,
MaxDiskUsage: queryLevelStats.MaxDiskUsage,
Expand Down
Loading

0 comments on commit 41c37cb

Please sign in to comment.