Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
75304: execbuilder: deflake TestExecBuild r=irfansharif a=irfansharif

Fixes #74933. This test asserts on physical plans of statements after
moving ranges around, i.e. whether distsql execution is "local" or
"distributed". This is determined by:
- where the distsql planner places processor cores,
- which in turn is a function of the span resolver,
- which delegates to the embedded distsender,
- which operates off of a range cache.

The range cache, like the name suggests, can hold stale data. This test
moved replicas of indexes around to induce distributed execution, but
was not accounting for the caches holding onto stale data. In my test
runs every so often I was seeing stale range descriptors from before the
relocate trip up the planner to generate a local execution, flaking the
test. Fortunately for us, all we need to do is slap on a retry. To
repro:

    # Took under a minute on my gceworker. Helped to trim down the test
    # file slightly.
    dev test pkg/sql/opt/exec/execbuilder \
      -f 'TestExecBuild/5node' -v --show-logs \
      --stress --stress-args '-p 4'

Release note: None

75436: kvserver: de-flake TestProtectedTimestamps r=irfansharif a=irfansharif

Fixed #75020. This test makes use of an "upsert-until-backpressure"
primitive, where it continually writes blobs to a scratch table
configured with a lower max range size (1<<18 from the default of
512<<20) until it experiences replica backpressure. Before #73876 (when
using the system config span), the splitting off of the scratch table's
ranges happened near instantly after the creation of the table itself.
This changed slightly with the span configs infrastructure, where
there's more of asynchrony built in and ranges may appear after a while
longer.

The test previously started upserting as soon as it created the table,
being able to implicitly rely on the tight synchrony of already having
the table's ranges carved out. This comes when deciding to record a
replica's largest previous max range size -- something we only do if the
replica's current size is larger than the new limit (see
(*Replica).SetSpanCnfig). When we weren't upserting until the range
applied the latest config, this was not possible. With span configs and
its additional asynchrony, this assumption no longer held. It was
possible then for the range to accept writes larger than the newest max
range size, which unintentionally (for this test at least) triggers an
escape hatch where we avoid backpressure when lowering a range's max
size below its current size (#46863).

The failure is easy to repro. This test runs in ~8s, and with the
following we were reliably seeing timeouts where the test was waiting
for backpressure to kick in (but it never did because of the escape
hatch).

    dev test pkg/kv/kvserver \
      -f TestProtectedTimestamps -v --show-logs \
      --timeout 300s --ignore-cache --count 10

De-flaking this test is simple -- we just wait for the table's replicas
to apply the latest config before issuing writes to it.

Release note: None

75443: ui: removed formatting to statements on the details pages r=THardy98 a=THardy98

**ui: removed formatting to statements on the details pages**

Previously, statements displayed on the statement/transaction/index
details pages were formatted (formatting was added to allow for better
readability of statements on these detail pages). However, statements
queries from the frontend were noticeably slower due to this
implementation. This change reverts the changes to statement formatting
(updates the fixtures to show the non-formatted statements), but keeps
the change that uses statement ID as an identifier in the URL instead of
the raw statement.

**Release note (ui change)**: removed formatting to statements on the
statement, transaction and index details pages, change to replace raw
statement with statement ID in the URL remained.

Co-authored-by: irfan sharif <[email protected]>
Co-authored-by: Thomas Hardy <[email protected]>
  • Loading branch information
3 people committed Jan 25, 2022
4 parents 1c4b0dd + 83f584d + 20161f9 + 39604d8 commit da5d0fc
Show file tree
Hide file tree
Showing 11 changed files with 125 additions and 224 deletions.
1 change: 0 additions & 1 deletion pkg/ccl/serverccl/statusccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,6 @@ go_test(
"//pkg/spanconfig",
"//pkg/sql/catalog/catconstants",
"//pkg/sql/catalog/descpb",
"//pkg/sql/sem/tree",
"//pkg/sql/sqlstats",
"//pkg/sql/tests",
"//pkg/testutils",
Expand Down
122 changes: 32 additions & 90 deletions pkg/ccl/serverccl/statusccl/tenant_status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,7 +28,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/spanconfig"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -116,35 +115,19 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
tenantStatusServer := tenant.StatusServer().(serverpb.SQLStatusServer)

type testCase struct {
stmt string
formattedStmt string
fingerprint string
formattedFingerprint string
stmt string
fingerprint string
}

testCaseTenant := []testCase{
{stmt: `CREATE DATABASE roachblog_t`},
{stmt: `SET database = roachblog_t`},
{stmt: `CREATE TABLE posts_t (id INT8 PRIMARY KEY, body STRING)`},
{
stmt: `CREATE DATABASE roachblog_t`,
formattedStmt: "CREATE DATABASE roachblog_t\n",
},
{
stmt: `SET database = roachblog_t`,
formattedStmt: "SET database = roachblog_t\n",
},
{
stmt: `CREATE TABLE posts_t (id INT8 PRIMARY KEY, body STRING)`,
formattedStmt: "CREATE TABLE posts_t (id INT8 PRIMARY KEY, body STRING)\n",
},
{
stmt: `INSERT INTO posts_t VALUES (1, 'foo')`,
fingerprint: `INSERT INTO posts_t VALUES (_, '_')`,
formattedStmt: "INSERT INTO posts_t VALUES (1, 'foo')\n",
formattedFingerprint: "INSERT INTO posts_t VALUES (_, '_')\n",
},
{
stmt: `SELECT * FROM posts_t`,
formattedStmt: "SELECT * FROM posts_t\n",
stmt: `INSERT INTO posts_t VALUES (1, 'foo')`,
fingerprint: `INSERT INTO posts_t VALUES (_, '_')`,
},
{stmt: `SELECT * FROM posts_t`},
}

for _, stmt := range testCaseTenant {
Expand All @@ -156,28 +139,14 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
require.NoError(t, err)

testCaseNonTenant := []testCase{
{stmt: `CREATE DATABASE roachblog_nt`},
{stmt: `SET database = roachblog_nt`},
{stmt: `CREATE TABLE posts_nt (id INT8 PRIMARY KEY, body STRING)`},
{
stmt: `CREATE DATABASE roachblog_nt`,
formattedStmt: "CREATE DATABASE roachblog_nt\n",
},
{
stmt: `SET database = roachblog_nt`,
formattedStmt: "SET database = roachblog_nt\n",
},
{
stmt: `CREATE TABLE posts_nt (id INT8 PRIMARY KEY, body STRING)`,
formattedStmt: "CREATE TABLE posts_nt (id INT8 PRIMARY KEY, body STRING)\n",
},
{
stmt: `INSERT INTO posts_nt VALUES (1, 'foo')`,
fingerprint: `INSERT INTO posts_nt VALUES (_, '_')`,
formattedStmt: "INSERT INTO posts_nt VALUES (1, 'foo')\n",
formattedFingerprint: "INSERT INTO posts_nt VALUES (_, '_')\n",
},
{
stmt: `SELECT * FROM posts_nt`,
formattedStmt: "SELECT * FROM posts_nt\n",
stmt: `INSERT INTO posts_nt VALUES (1, 'foo')`,
fingerprint: `INSERT INTO posts_nt VALUES (_, '_')`,
},
{stmt: `SELECT * FROM posts_nt`},
}

pgURL, cleanupGoDB := sqlutils.PGUrl(
Expand Down Expand Up @@ -230,22 +199,14 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
err = serverutils.GetJSONProto(nonTenant, path, &nonTenantCombinedStats)
require.NoError(t, err)

checkStatements := func(t *testing.T, tc []testCase, actual *serverpb.StatementsResponse, combined bool) {
checkStatements := func(t *testing.T, tc []testCase, actual *serverpb.StatementsResponse) {
t.Helper()
var expectedStatements []string
for _, stmt := range tc {
var expectedStmt = stmt.stmt
if combined {
expectedStmt = stmt.formattedStmt
}
if stmt.fingerprint != "" {
if combined {
expectedStmt = stmt.formattedFingerprint
} else {
expectedStmt = stmt.fingerprint
}
expectedStmt = stmt.fingerprint
}

expectedStatements = append(expectedStatements, expectedStmt)
}

Expand All @@ -272,14 +233,14 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {

// First we verify that we have expected stats from tenants.
t.Run("tenant-stats", func(t *testing.T) {
checkStatements(t, testCaseTenant, tenantStats, false)
checkStatements(t, testCaseTenant, tenantCombinedStats, true)
checkStatements(t, testCaseTenant, tenantStats)
checkStatements(t, testCaseTenant, tenantCombinedStats)
})

// Now we verify the non tenant stats are what we expected.
t.Run("non-tenant-stats", func(t *testing.T) {
checkStatements(t, testCaseNonTenant, &nonTenantStats, false)
checkStatements(t, testCaseNonTenant, &nonTenantCombinedStats, true)
checkStatements(t, testCaseNonTenant, &nonTenantStats)
checkStatements(t, testCaseNonTenant, &nonTenantCombinedStats)
})

// Now we verify that tenant and non-tenant have no visibility into each other's stats.
Expand Down Expand Up @@ -313,29 +274,10 @@ func TestTenantCannotSeeNonTenantStats(t *testing.T) {
func testResetSQLStatsRPCForTenant(
ctx context.Context, t *testing.T, testHelper *tenantTestHelper,
) {

type testCase struct {
stmt string
formattedStmt string
}
stmts := []testCase{
{
stmt: "SELECT 1",
formattedStmt: "SELECT 1\n",
},
{
stmt: "SELECT 1, 1",
formattedStmt: "SELECT 1, 1\n",
},
{
stmt: "SELECT 1, 1, 1",
formattedStmt: "SELECT 1, 1\n",
},
}

var expectedStatements []string
for _, tc := range stmts {
expectedStatements = append(expectedStatements, tc.formattedStmt)
stmts := []string{
"SELECT 1",
"SELECT 1, 1",
"SELECT 1, 1, 1",
}

testCluster := testHelper.testCluster()
Expand Down Expand Up @@ -365,8 +307,8 @@ func testResetSQLStatsRPCForTenant(
}()

for _, stmt := range stmts {
testCluster.tenantConn(randomServer).Exec(t, stmt.stmt)
controlCluster.tenantConn(randomServer).Exec(t, stmt.stmt)
testCluster.tenantConn(randomServer).Exec(t, stmt)
controlCluster.tenantConn(randomServer).Exec(t, stmt)
}

if flushed {
Expand All @@ -383,7 +325,7 @@ func testResetSQLStatsRPCForTenant(

require.NotEqual(t, 0, len(statsPreReset.Statements),
"expected to find stats for at least one statement, but found: %d", len(statsPreReset.Statements))
ensureExpectedStmtFingerprintExistsInRPCResponse(t, expectedStatements, statsPreReset, "test")
ensureExpectedStmtFingerprintExistsInRPCResponse(t, stmts, statsPreReset, "test")

_, err = status.ResetSQLStats(ctx, &serverpb.ResetSQLStatsRequest{
ResetPersistedStats: true,
Expand Down Expand Up @@ -420,7 +362,7 @@ func testResetSQLStatsRPCForTenant(
})
require.NoError(t, err)

ensureExpectedStmtFingerprintExistsInRPCResponse(t, expectedStatements, statsFromControlCluster, "control")
ensureExpectedStmtFingerprintExistsInRPCResponse(t, stmts, statsFromControlCluster, "control")
})
}
}
Expand Down Expand Up @@ -607,10 +549,10 @@ SET DATABASE=test_db1;
SELECT * FROM test;
`)

getCreateStmtQuery := fmt.Sprintf(`
SELECT prettify_statement(indexdef, %d, %d, %d)
FROM pg_catalog.pg_indexes
WHERE tablename = 'test' AND indexname = $1`, tree.ConsoleLineWidth, tree.PrettyAlignAndDeindent, tree.UpperCase)
getCreateStmtQuery := `
SELECT indexdef
FROM pg_catalog.pg_indexes
WHERE tablename = 'test' AND indexname = $1`

// Get index usage stats and assert expected results.
resp := getTableIndexStats(testHelper, "test_db1")
Expand Down
45 changes: 40 additions & 5 deletions pkg/kv/kvserver/client_protectedts_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,8 +68,12 @@ func TestProtectedTimestamps(t *testing.T) {
_, err = conn.Exec("SET CLUSTER SETTING kv.protectedts.poll_interval = '10ms';")
require.NoError(t, err)

_, err = conn.Exec("ALTER TABLE foo CONFIGURE ZONE USING " +
"gc.ttlseconds = 1, range_max_bytes = 1<<18, range_min_bytes = 1<<10;")
_, err = conn.Exec("SET CLUSTER SETTING kv.closed_timestamp.target_duration = '100ms'") // speeds up the test
require.NoError(t, err)

const tableRangeMaxBytes = 1 << 18
_, err = conn.Exec("ALTER TABLE foo CONFIGURE ZONE USING "+
"gc.ttlseconds = 1, range_max_bytes = $1, range_min_bytes = 1<<10;", tableRangeMaxBytes)
require.NoError(t, err)

rRand, _ := randutil.NewTestRand()
Expand All @@ -86,7 +90,25 @@ func TestProtectedTimestamps(t *testing.T) {
const processedPattern = `(?s)shouldQueue=true.*processing replica.*GC score after GC`
processedRegexp := regexp.MustCompile(processedPattern)

getTableStartKey := func(table string) roachpb.Key {
waitForTableSplit := func() {
testutils.SucceedsSoon(t, func() error {
count := 0
if err := conn.QueryRow(
"SELECT count(*) "+
"FROM crdb_internal.ranges_no_leases "+
"WHERE table_name = $1 "+
"AND database_name = current_database()",
"foo").Scan(&count); err != nil {
return err
}
if count == 0 {
return errors.New("waiting for table split")
}
return nil
})
}

getTableStartKey := func() roachpb.Key {
row := conn.QueryRow(
"SELECT start_key "+
"FROM crdb_internal.ranges_no_leases "+
Expand All @@ -102,7 +124,7 @@ func TestProtectedTimestamps(t *testing.T) {
}

getStoreAndReplica := func() (*kvserver.Store, *kvserver.Replica) {
startKey := getTableStartKey("foo")
startKey := getTableStartKey()
// Okay great now we have a key and can go find replicas and stores and what not.
r := tc.LookupRangeOrFatal(t, startKey)
l, _, err := tc.FindRangeLease(r, nil)
Expand All @@ -112,6 +134,16 @@ func TestProtectedTimestamps(t *testing.T) {
return getFirstStoreReplica(t, lhServer, startKey)
}

waitForRangeMaxBytes := func(maxBytes int64) {
testutils.SucceedsSoon(t, func() error {
_, r := getStoreAndReplica()
if r.GetMaxBytes() != maxBytes {
return errors.New("waiting for range_max_bytes to be applied")
}
return nil
})
}

gcSoon := func() {
testutils.SucceedsSoon(t, func() error {
upsertUntilBackpressure()
Expand All @@ -134,13 +166,16 @@ func TestProtectedTimestamps(t *testing.T) {
return thresh
}

waitForTableSplit()
waitForRangeMaxBytes(tableRangeMaxBytes)

beforeWrites := s0.Clock().Now()
gcSoon()

pts := ptstorage.New(s0.ClusterSettings(), s0.InternalExecutor().(*sql.InternalExecutor),
nil /* knobs */)
ptsWithDB := ptstorage.WithDatabase(pts, s0.DB())
startKey := getTableStartKey("foo")
startKey := getTableStartKey()
ptsRec := ptpb.Record{
ID: uuid.MakeV4().GetBytes(),
Timestamp: s0.Clock().Now(),
Expand Down
10 changes: 2 additions & 8 deletions pkg/server/combined_statement_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,18 +116,12 @@ func collectCombinedStatements(
transaction_fingerprint_id,
app_name,
aggregated_ts,
jsonb_set(
metadata,
array['query'],
to_jsonb(
prettify_statement(metadata ->> 'query', %d, %d, %d)
)
),
metadata,
statistics,
sampled_plan,
aggregation_interval
FROM crdb_internal.statement_statistics
%s`, tree.ConsoleLineWidth, tree.PrettyAlignAndDeindent, tree.UpperCase, whereClause)
%s`, whereClause)

const expectedNumDatums = 8

Expand Down
8 changes: 3 additions & 5 deletions pkg/server/index_usage_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,23 +223,21 @@ func getTableIndexUsageStats(

q := makeSQLQuery()
// TODO(#72930): Implement virtual indexes on index_usages_statistics and table_indexes
q.Append(`SELECT
q.Append(`
SELECT
ti.index_id,
ti.index_name,
ti.index_type,
total_reads,
last_read,
prettify_statement(indexdef, $, $, $)
indexdef
FROM crdb_internal.index_usage_statistics AS us
JOIN crdb_internal.table_indexes AS ti ON us.index_id = ti.index_id
AND us.table_id = ti.descriptor_id
JOIN pg_catalog.pg_index AS pgidx ON indrelid = us.table_id
JOIN pg_catalog.pg_indexes AS pgidxs ON pgidxs.crdb_oid = indexrelid
AND indexname = ti.index_name
WHERE ti.descriptor_id = $::REGCLASS`,
tree.ConsoleLineWidth,
tree.PrettyAlignAndDeindent,
tree.UpperCase,
tableID,
)

Expand Down
Loading

0 comments on commit da5d0fc

Please sign in to comment.