Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
112106: server: add tests for combined stmts api source tables r=maryliag a=xinhaoz

This patch creates the sqlstatstestutils pkg in
sqlstats/persistedsqlstats.  It moves the function
`GetRandomizedCollectedStatementStatisticsForTest` which was
previously in sqlstatsutil. This pkg is introduced in
preparation for adding more testing utilities to mock sql
stats tables without creating cyclic dependencies with
sqlstatsutil.

Epic: none

Release note: None

### 2: [server: add tests for combined stmts api](1bf95c1)

Add testing to verify that the combined stmts api uses the
correct source tables given different table states.

Epic: none

112990: kvserver: setup ranges in large unsplittable replicate test r=andrewbaptist a=kvoli

`TestLargeUnsplittableRangeReplicate` would fail when the SQL query to
show the table's replicas returned no results for the unsplittable range
being tested. No results would be returned when the table split was
delayed.

Update the test to insert a split at the first row, as well as the
existing split at the second row. This avoids the first range starting
with 5 replicas and the test's reliance on timely span config splits.

The previous deflake attempt is reverted.

Resolves: #112774
Release note: None

113500: roachtest: skip restore/tpce/32TB/inc-count=400/gce/nodes=15/cpus=16 r=msbutler a=msbutler

This weekly test is very costly (leads to cross region egress) and is already
run on aws. We can reconsider unskipping it after #111371 is addressed.

Epic: none

Release note: none

Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com>
Co-authored-by: Austen McClernon <austen@cockroachlabs.com>
Co-authored-by: Michael Butler <butler@cockroachlabs.com>
  • Loading branch information
4 people committed Oct 31, 2023
4 parents d482bde + 392a28b + ad54149 + 285fda2 commit 6bd0235
Show file tree
Hide file tree
Showing 13 changed files with 591 additions and 74 deletions.
1 change: 1 addition & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -2142,6 +2142,7 @@ GO_TARGETS = [
"//pkg/sql/sqlstats/insights/integration:integration_test",
"//pkg/sql/sqlstats/insights:insights",
"//pkg/sql/sqlstats/insights:insights_test",
"//pkg/sql/sqlstats/persistedsqlstats/sqlstatstestutil:sqlstatstestutil",
"//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil:sqlstatsutil",
"//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil:sqlstatsutil_test",
"//pkg/sql/sqlstats/persistedsqlstats:persistedsqlstats",
Expand Down
5 changes: 5 additions & 0 deletions pkg/cmd/roachtest/tests/restore.go
Original file line number Diff line number Diff line change
Expand Up @@ -420,6 +420,7 @@ func registerRestore(r registry.Registry) {
`SET CLUSTER SETTING backup.restore_span.target_size = '0'`,
},
restoreUptoIncremental: 400,
skip: "a recent gcp pricing policy makes this test very expensive. unskip after #111371 is addressed",
},
{
// A teeny weeny 15GB restore that could be used to bisect scale agnostic perf regressions.
Expand Down Expand Up @@ -453,6 +454,7 @@ func registerRestore(r registry.Registry) {
CompatibleClouds: sp.clouds,
Suites: sp.suites,
Tags: sp.tags,
Skip: sp.skip,
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {

rd := makeRestoreDriver(t, c, sp)
Expand Down Expand Up @@ -825,6 +827,9 @@ type restoreSpecs struct {

testName string
setUpStmts []string

// skip, if non-empty, skips the test with the given reason.
skip string
}

func (sp *restoreSpecs) initTestName() {
Expand Down
32 changes: 9 additions & 23 deletions pkg/kv/kvserver/replicate_queue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1774,8 +1774,6 @@ func TestLargeUnsplittableRangeReplicate(t *testing.T) {
base.TestClusterArgs{
ReplicationMode: base.ReplicationAuto,
ServerArgs: base.TestServerArgs{
ScanMinIdleTime: time.Millisecond,
ScanMaxIdleTime: time.Millisecond,
Knobs: base.TestingKnobs{
Server: &server.TestingKnobs{
DefaultZoneConfigOverride: &zcfg,
Expand All @@ -1788,11 +1786,12 @@ func TestLargeUnsplittableRangeReplicate(t *testing.T) {

// We're going to create a table with many versions of a big row and a small
// row. We'll split the table in between the rows, to produce a large range
// and a small one. Then we'll increase the replication factor to 5 and check
// that both ranges behave the same - i.e. they both get up-replicated. For
// the purposes of this test we're only worried about the large one
// up-replicating, but we test the small one as a control so that we don't
// fool ourselves.
// and a small one. We'll also split the first row into its own range, to
// avoid the range inheriting 5 replicas from the system ranges. Then we'll
// increase the replication factor to 5 and check that both ranges behave the
// same - i.e. they both get up-replicated. For the purposes of this test
// we're only worried about the large one up-replicating, but we test the
// small one as a control so that we don't fool ourselves.

// Disable the queues so they don't mess with our manual relocation. We'll
// re-enable them later.
Expand All @@ -1805,27 +1804,14 @@ func TestLargeUnsplittableRangeReplicate(t *testing.T) {

_, err = db.Exec(`ALTER TABLE t EXPERIMENTAL_RELOCATE VALUES (ARRAY[1,2,3], 1)`)
require.NoError(t, err)
_, err = db.Exec(`ALTER TABLE t SPLIT AT VALUES (1)`)
require.NoError(t, err)
_, err = db.Exec(`ALTER TABLE t SPLIT AT VALUES (2)`)
require.NoError(t, err)

toggleReplicationQueues(tc, true /* active */)
toggleSplitQueues(tc, true /* active */)

// Check that the two ranges exist for table t.
testutils.SucceedsSoon(t, func() error {
r := db.QueryRow(
"SELECT count(*) FROM [SHOW RANGES FROM TABLE t]")
var count int
if err := r.Scan(&count); err != nil {
return err
}
if count != 2 {
return fmt.Errorf(
"splits not created, expected %d ranges, found %d", 2, count)
}
return nil
})

// We're going to create a large row, but now large enough that write
// back-pressuring kicks in and refuses it.
var sb strings.Builder
Expand Down Expand Up @@ -1877,7 +1863,7 @@ func TestLargeUnsplittableRangeReplicate(t *testing.T) {
testutils.SucceedsSoon(t, func() error {
forceProcess()
r := db.QueryRow(
"SELECT replicas FROM [SHOW RANGES FROM TABLE t] WHERE start_key LIKE '%TableMin%'")
"SELECT replicas FROM [SHOW RANGES FROM TABLE t] WHERE start_key LIKE '%/1'")
var repl string
if err := r.Scan(&repl); err != nil {
return err
Expand Down
1 change: 1 addition & 0 deletions pkg/server/application_api/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -67,6 +67,7 @@ go_test(
"//pkg/sql/sessiondata",
"//pkg/sql/sqlstats",
"//pkg/sql/sqlstats/persistedsqlstats",
"//pkg/sql/sqlstats/persistedsqlstats/sqlstatstestutil",
"//pkg/testutils",
"//pkg/testutils/diagutils",
"//pkg/testutils/serverutils",
Expand Down
248 changes: 248 additions & 0 deletions pkg/server/application_api/sql_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,15 +24,19 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/security/username"
"github.com/cockroachdb/cockroach/pkg/server"
"github.com/cockroachdb/cockroach/pkg/server/apiconstants"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/srvtestutils"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/appstatspb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats/persistedsqlstats/sqlstatstestutil"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
Expand Down Expand Up @@ -1135,3 +1139,247 @@ func TestUnprivilegedUserResetIndexUsageStats(t *testing.T) {

require.Contains(t, err.Error(), "requires admin privilege")
}

// TestCombinedStatementUsesCorrectSourceTable tests that requests read from
// the expected crdb_internal table given the table states. We have a lot of
// different tables that requests could potentially read from (in-memory, cached,
// system tables etc.), so we should sanity check that we are using the expected ones.
// given some simple table states.
func TestCombinedStatementUsesCorrectSourceTable(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx := context.Background()

// Disable flushing sql stats so we can manually set the table states
// without worrying about unexpected stats appearing.
settings := cluster.MakeTestingClusterSettings()
statsKnobs := sqlstats.CreateTestingKnobs()
defaultMockInsertedAggTs := timeutil.Unix(1696906800, 0)
statsKnobs.StubTimeNow = func() time.Time { return defaultMockInsertedAggTs }
persistedsqlstats.SQLStatsFlushEnabled.Override(ctx, &settings.SV, false)
srv := serverutils.StartServerOnly(t, base.TestServerArgs{
Settings: settings,
Knobs: base.TestingKnobs{
SQLStatsKnobs: statsKnobs,
},
})
defer srv.Stopper().Stop(ctx)

conn := sqlutils.MakeSQLRunner(srv.ApplicationLayer().SQLConn(t))
conn.Exec(t, "SET application_name = $1", server.CrdbInternalStmtStatsCombined)
conn.Exec(t, "SET CLUSTER SETTING sql.stats.activity.flush.enabled = 'f'")
// Clear the in-memory stats so we only have the above app name.
// Then populate it with 1 query.
conn.Exec(t, "SELECT crdb_internal.reset_sql_stats()")
conn.Exec(t, "SELECT 1")

type testCase struct {
name string
tableSetupFn func() error
expectedStmtsTable string
expectedTxnsTable string
reqs []serverpb.CombinedStatementsStatsRequest
isEmpty bool
}

ie := srv.InternalExecutor().(*sql.InternalExecutor)

defaultMockOneEach := func() error {
startTs := defaultMockInsertedAggTs
stmt := sqlstatstestutil.GetRandomizedCollectedStatementStatisticsForTest(t)
stmt.ID = 1
stmt.AggregatedTs = startTs
stmt.Key.App = server.CrdbInternalStmtStatsPersisted
stmt.Key.TransactionFingerprintID = 1
require.NoError(t, sqlstatstestutil.InsertMockedIntoSystemStmtStats(ctx, ie, &stmt, 1 /* nodeId */, nil))

stmt.Key.App = server.CrdbInternalStmtStatsCached
require.NoError(t, sqlstatstestutil.InsertMockedIntoSystemStmtActivity(ctx, ie, &stmt, nil))

txn := sqlstatstestutil.GetRandomizedCollectedTransactionStatisticsForTest(t)
txn.StatementFingerprintIDs = []appstatspb.StmtFingerprintID{1}
txn.TransactionFingerprintID = 1
txn.AggregatedTs = startTs
txn.App = server.CrdbInternalTxnStatsPersisted
require.NoError(t, sqlstatstestutil.InsertMockedIntoSystemTxnStats(ctx, ie, &txn, 1, nil))
txn.App = server.CrdbInternalTxnStatsCached
require.NoError(t, sqlstatstestutil.InsertMockedIntoSystemTxnActivity(ctx, ie, &txn, nil))

return nil
}
testCases := []testCase{
{
name: "activity and persisted tables empty",
tableSetupFn: func() error { return nil },
// We should attempt to read from the in-memory tables, since
// they are the last resort.
expectedStmtsTable: server.CrdbInternalStmtStatsCombined,
expectedTxnsTable: server.CrdbInternalTxnStatsCombined,
reqs: []serverpb.CombinedStatementsStatsRequest{
{FetchMode: createTxnFetchMode(0)},
},
},
{
name: "all tables have data in selected range",
tableSetupFn: defaultMockOneEach,
expectedStmtsTable: server.CrdbInternalStmtStatsCached,
expectedTxnsTable: server.CrdbInternalTxnStatsCached,
reqs: []serverpb.CombinedStatementsStatsRequest{
{Start: defaultMockInsertedAggTs.Unix()},
{
Start: defaultMockInsertedAggTs.Unix(),
End: defaultMockInsertedAggTs.Unix(),
},
},
},
{
name: "all tables have data but no start range is provided",
tableSetupFn: defaultMockOneEach,
// When no date range is provided, we should default to reading from
// persisted or in-memory (whichever has data first). In this case the
// persisted table has data.
expectedStmtsTable: server.CrdbInternalStmtStatsPersisted,
expectedTxnsTable: server.CrdbInternalTxnStatsPersisted,
reqs: []serverpb.CombinedStatementsStatsRequest{
{},
{End: defaultMockInsertedAggTs.Unix()},
},
},
{
name: "all tables have data but not in the selected range",
tableSetupFn: defaultMockOneEach,
expectedStmtsTable: server.CrdbInternalStmtStatsCombined,
expectedTxnsTable: server.CrdbInternalTxnStatsCombined,
reqs: []serverpb.CombinedStatementsStatsRequest{
{Start: defaultMockInsertedAggTs.Add(time.Hour).Unix()},
{End: defaultMockInsertedAggTs.Truncate(time.Hour * 2).Unix()},
},
isEmpty: true,
},
{
name: "activity table has data in range with specified sort, fetchmode=txns",
tableSetupFn: defaultMockOneEach,
// For txn mode, we should not use the activity table for stmts.
expectedStmtsTable: server.CrdbInternalStmtStatsPersisted,
expectedTxnsTable: server.CrdbInternalTxnStatsCached,
// These sort options do exist on the activity table.
reqs: []serverpb.CombinedStatementsStatsRequest{
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(0)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(1)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(2)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(3)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(4)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(5)},
},
},
{
name: "activity table has data in range with specified sort, fetchmode=stmts",
tableSetupFn: defaultMockOneEach,
expectedStmtsTable: server.CrdbInternalStmtStatsCached,
expectedTxnsTable: "",
// These sort options do exist on the activity table.
reqs: []serverpb.CombinedStatementsStatsRequest{
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(0)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(1)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(2)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(3)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(4)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(5)},
},
},
{
name: "activity table has data in range, but selected sort is not on it, fetchmode=txns",
tableSetupFn: defaultMockOneEach,
expectedStmtsTable: server.CrdbInternalStmtStatsPersisted,
expectedTxnsTable: server.CrdbInternalTxnStatsPersisted,
// These sort options do not exist on the activity table.
reqs: []serverpb.CombinedStatementsStatsRequest{
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(6)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(7)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(8)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(9)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(10)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(11)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(12)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(13)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createTxnFetchMode(14)},
},
},
{
name: "activity table has data in range, but selected sort is not on it, fetchmode=stmts",
tableSetupFn: defaultMockOneEach,
expectedStmtsTable: server.CrdbInternalStmtStatsPersisted,
expectedTxnsTable: "",
// These sort options do not exist on the activity table.
reqs: []serverpb.CombinedStatementsStatsRequest{
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(6)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(7)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(8)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(9)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(10)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(11)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(12)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(13)},
{Start: defaultMockInsertedAggTs.Unix(), FetchMode: createStmtFetchMode(14)},
},
},
}

client := srv.ApplicationLayer().GetStatusClient(t)

for _, tc := range testCases {
t.Run(tc.name, func(t *testing.T) {
require.NoError(t, tc.tableSetupFn())

defer func() {
// Reset tables.
conn.Exec(t, "SELECT crdb_internal.reset_sql_stats()")
conn.Exec(t, "SELECT crdb_internal.reset_activity_tables()")
conn.Exec(t, "SELECT 1")
}()

for _, r := range tc.reqs {
resp, err := client.CombinedStatementStats(ctx, &r)
require.NoError(t, err)

require.Equal(t, tc.expectedStmtsTable, resp.StmtsSourceTable, "req: %v", r)
require.Equal(t, tc.expectedTxnsTable, resp.TxnsSourceTable, "req: %v", r)

if tc.isEmpty {
continue
}

require.NotZero(t, len(resp.Statements), "req: %v", r)
// Verify we used the correct queries to return data.
require.Equal(t, tc.expectedStmtsTable, resp.Statements[0].Key.KeyData.App, "req: %v", r)
if tc.expectedTxnsTable == server.CrdbInternalTxnStatsCombined {
// For the combined query, we're using in-mem data and we set the
// app name there to the in-memory stmts table.
require.Equal(t, server.CrdbInternalStmtStatsCombined, resp.Transactions[0].StatsData.App, "req: %v", r)
} else if tc.expectedTxnsTable != "" {
require.NotZero(t, len(resp.Transactions))
require.Equal(t, tc.expectedTxnsTable, resp.Transactions[0].StatsData.App, "req: %v", r)
}
}

})
}
}

func createStmtFetchMode(
sort serverpb.StatsSortOptions,
) *serverpb.CombinedStatementsStatsRequest_FetchMode {
return &serverpb.CombinedStatementsStatsRequest_FetchMode{
StatsType: serverpb.CombinedStatementsStatsRequest_StmtStatsOnly,
Sort: sort,
}
}
func createTxnFetchMode(
sort serverpb.StatsSortOptions,
) *serverpb.CombinedStatementsStatsRequest_FetchMode {
return &serverpb.CombinedStatementsStatsRequest_FetchMode{
StatsType: serverpb.CombinedStatementsStatsRequest_TxnStatsOnly,
Sort: sort,
}
}
Loading

0 comments on commit 6bd0235

Please sign in to comment.