Skip to content

Commit

Permalink
server: flush SQL stats during drain
Browse files Browse the repository at this point in the history
Previously, SQL stats would be lost when a node drains. Now a drain
triggers a flush of the SQL stats into the statement statistics
system table while the SQL layer is being drained.

Release note (cli change): a drain of node now ensures that
SQL statistics are not lost during the process; they are now
preserved in the statement statistics system table.
  • Loading branch information
cameronnunez committed Feb 16, 2022
1 parent 601d840 commit 56ff1ac
Show file tree
Hide file tree
Showing 3 changed files with 68 additions and 0 deletions.
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -178,6 +178,7 @@ go_library(
"//pkg/sql/sqlliveness",
"//pkg/sql/sqlliveness/slprovider",
"//pkg/sql/sqlstats",
"//pkg/sql/sqlstats/persistedsqlstats",
"//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil",
"//pkg/sql/sqlutil",
"//pkg/sql/stats",
Expand Down
4 changes: 4 additions & 0 deletions pkg/server/drain.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
Expand Down Expand Up @@ -329,6 +330,9 @@ func (s *drainServer) drainClients(
// The queryWait duration is used to wait on currently running flows to finish.
s.sqlServer.distSQLServer.Drain(ctx, queryMaxWait, reporter)

// Flush in-memory SQL stats into the statement stats system table.
s.sqlServer.pgServer.SQLServer.GetSQLStatsProvider().(*persistedsqlstats.PersistedSQLStats).Flush(ctx)

// Drain all SQL table leases. This must be done after the pgServer has
// given sessions a chance to finish ongoing work.
s.sqlServer.leaseMgr.SetDraining(ctx, true /* drain */, reporter)
Expand Down
63 changes: 63 additions & 0 deletions pkg/server/drain_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,20 @@ import (
"time"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
"github.com/kr/pretty"
"github.com/stretchr/testify/require"
"google.golang.org/grpc"
)

Expand Down Expand Up @@ -93,6 +97,65 @@ func doTestDrain(tt *testing.T) {
})
}

func TestEnsureSQLStatsAreFlushedDuringDrain(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()

var drainSleepCallCount = 0
drainCtx := newTestDrainContext(t, &drainSleepCallCount)
defer drainCtx.Close()

var (
ts = drainCtx.tc.Server(0).SQLServer().(*sql.Server)
sqlDB = sqlutils.MakeSQLRunner(drainCtx.tc.ServerConn(0))
)

// Issue queries to be registered in stats.
sqlDB.Exec(t, `
CREATE DATABASE t;
CREATE TABLE t.test (x INT PRIMARY KEY);
INSERT INTO t.test VALUES (1);
INSERT INTO t.test VALUES (2);
INSERT INTO t.test VALUES (3);
`)

// Find the in-memory stats for the queries.
stats, err := ts.GetScrubbedStmtStats(ctx)
require.NoError(t, err)
require.Truef(t,
func(stats []roachpb.CollectedStatementStatistics) bool {
for _, stat := range stats {
if stat.Key.Query == "INSERT INTO _ VALUES (_)" {
return true
}
}
return false
}(stats),
"expected to find in-memory stats",
)

// Sanity check: verify that the statement statistics system table is empty.
sqlDB.CheckQueryResults(t,
`SELECT count(*) FROM system.statement_statistics WHERE node_id = 1`,
[][]string{{"0"}},
)

// Issue a drain.
drainCtx.sendDrainNoShutdown()

// Open a new SQL connection.
sqlDB = sqlutils.MakeSQLRunner(drainCtx.tc.ServerConn(1))

// Check that the stats were flushed into the statement stats system table.
// Verify that the number of statistics for node 1 are non-zero.
sqlDB.CheckQueryResults(t,
`SELECT count(*) > 0 FROM system.statement_statistics WHERE node_id = 1`,
[][]string{{"true"}},
)
}

type testDrainContext struct {
*testing.T
tc *testcluster.TestCluster
Expand Down

0 comments on commit 56ff1ac

Please sign in to comment.