Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
70361: sql: return an error message when trying to show grants for an invalid user r=jackcwu a=jackcwu

Release justification: None, waiting for 22.1
Release note: None

Resolves #32811

70933: reduce: show original test case output r=mgartner a=mgartner

The `reduce` tool now shows the original test case output in verbose
mode if it is not found to be interesting. This makes it easier for a
user to determine why a reproduction is not producing output that
matches the `--contains` regex.

Release note: None

70966: serverccl: support IndexUsageStatistics RPC fanout for tenant status server r=maryliag a=Azhng

Follow up to #70959

Resolves #70878

Release note (api change): Serverless's IndexUsageStatistics RPC now
returns cluster-wide data.

70994: server: Fix query profiler panic for serverless. r=jaylim-crl a=rimadeodhar

The Query Profiler would panic for serverless clusters under memory
pressure as the session registry was not set. This PR updates the
serverless code to pass in the session registry to the Query Profiler.


Fixes: #70945

Release note (bug fix): The SQL server no longer panics under
memory pressure when the query profiler is enabled.


71000: roachtest/sqlsmith: swallow "no volatility for cast" error r=yuzefovich a=yuzefovich

Touches: #70831.

Release note: None

Co-authored-by: Jack Wu <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
Co-authored-by: Azhng <[email protected]>
Co-authored-by: rimadeodhar <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
  • Loading branch information
6 people committed Oct 1, 2021
6 parents ebca030 + 59e8587 + a42d5c9 + e074477 + ae5885f + 9ee7777 commit a3ba197
Show file tree
Hide file tree
Showing 22 changed files with 311 additions and 80 deletions.
1 change: 1 addition & 0 deletions pkg/ccl/serverccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,7 @@ go_test(
"//pkg/sql",
"//pkg/sql/catalog/catconstants",
"//pkg/sql/catalog/descpb",
"//pkg/sql/idxusage",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/sqlstats",
"//pkg/sql/tests",
Expand Down
90 changes: 88 additions & 2 deletions pkg/ccl/serverccl/tenant_status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"strconv"
"strings"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/base"
_ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl"
Expand All @@ -25,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/idxusage"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
Expand Down Expand Up @@ -208,7 +210,7 @@ func TestResetSQLStatsRPCForTenant(t *testing.T) {
"SELECT 1, 1, 1",
}

testHelper := newTestTenantHelper(t, 3 /* tenantClusterSize */)
testHelper := newTestTenantHelper(t, 3 /* tenantClusterSize */, tests.CreateTestingKnobs())
defer testHelper.cleanup(ctx, t)

testCluster := testHelper.testCluster()
Expand Down Expand Up @@ -318,7 +320,8 @@ func TestContentionEventsForTenant(t *testing.T) {

ctx := context.Background()

testHelper := newTestTenantHelper(t, 3 /* tenantClusterSize */)
testHelper :=
newTestTenantHelper(t, 3 /* tenantClusterSize */, tests.CreateTestingKnobs())
defer testHelper.cleanup(ctx, t)

testingCluster := testHelper.testCluster()
Expand Down Expand Up @@ -383,3 +386,86 @@ SET TRACING=off;
}
}
}

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

skip.UnderStressRace(t, "expensive tests")

ctx := context.Background()

statsIngestionCb, statsIngestionNotifier := idxusage.CreateIndexStatsIngestedCallbackForTest()

knobs := tests.CreateTestingKnobs()
knobs.IndexUsageStatsKnobs = &idxusage.TestingKnobs{
OnIndexUsageStatsProcessedCallback: statsIngestionCb,
}
testHelper := newTestTenantHelper(t, 3 /* tenantClusterSize */, knobs)
defer testHelper.cleanup(ctx, t)

testingCluster := testHelper.testCluster()
controlledCluster := testHelper.controlCluster()

testingCluster.tenantConn(0).Exec(t, `
CREATE TABLE test (
k INT PRIMARY KEY,
a INT,
b INT,
INDEX(a)
)
`)

testingCluster.tenantConn(0).Exec(t, `
INSERT INTO test
VALUES (1, 10, 100), (2, 20, 200), (3, 30, 300)
`)

// Record scan on primary index.
testingCluster.tenantConn(0).Exec(t, "SELECT * FROM test")

// Record scan on secondary index.
testingCluster.tenantConn(1).Exec(t, "SELECT * FROM test@test_a_idx")
testTableIDStr := testingCluster.tenantConn(2).QueryStr(t, "SELECT 'test'::regclass::oid")[0][0]
testTableID, err := strconv.Atoi(testTableIDStr)
require.NoError(t, err)

// Wait for the stats to be ingested.
require.NoError(t,
idxusage.WaitForIndexStatsIngestionForTest(statsIngestionNotifier, map[roachpb.IndexUsageKey]struct{}{
{
TableID: roachpb.TableID(testTableID),
IndexID: 1,
}: {},
{
TableID: roachpb.TableID(testTableID),
IndexID: 2,
}: {},
}, 2 /* expectedEventCnt*/, 5*time.Second /* timeout */),
)

query := `
SELECT
table_id,
index_id,
total_reads,
extract_duration('second', now() - last_read) < 5
FROM
crdb_internal.index_usage_statistics
WHERE
table_id = $1
`
actual := testingCluster.tenantConn(2).QueryStr(t, query, testTableID)
expected := [][]string{
{testTableIDStr, "1", "1", "true"},
{testTableIDStr, "2", "1", "true"},
}

require.Equal(t, expected, actual)

// Ensure tenant data isolation.
actual = controlledCluster.tenantConn(2).QueryStr(t, query, testTableID)
expected = [][]string{}

require.Equal(t, expected, actual)
}
22 changes: 18 additions & 4 deletions pkg/ccl/serverccl/tenant_test_utils.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,12 +35,17 @@ type testTenant struct {
}

func newTestTenant(
t *testing.T, server serverutils.TestServerInterface, existing bool, tenantID roachpb.TenantID,
t *testing.T,
server serverutils.TestServerInterface,
existing bool,
tenantID roachpb.TenantID,
knobs base.TestingKnobs,
) *testTenant {
t.Helper()

tenantParams := tests.CreateTestTenantParams(tenantID)
tenantParams.Existing = existing
tenantParams.TestingKnobs = knobs

log.TestingClearServerIdentifiers()
tenant, tenantConn := serverutils.StartTenant(t, server, tenantParams)
Expand Down Expand Up @@ -71,7 +76,9 @@ type tenantTestHelper struct {
tenantControlCluster tenantCluster
}

func newTestTenantHelper(t *testing.T, tenantClusterSize int) *tenantTestHelper {
func newTestTenantHelper(
t *testing.T, tenantClusterSize int, knobs base.TestingKnobs,
) *tenantTestHelper {
t.Helper()

params, _ := tests.CreateTestServerParams()
Expand All @@ -87,12 +94,14 @@ func newTestTenantHelper(t *testing.T, tenantClusterSize int) *tenantTestHelper
server,
tenantClusterSize,
security.EmbeddedTenantIDs()[0],
knobs,
),
tenantControlCluster: newTenantCluster(
t,
server,
tenantClusterSize,
security.EmbeddedTenantIDs()[1],
knobs,
),
}
}
Expand All @@ -115,14 +124,19 @@ func (h *tenantTestHelper) cleanup(ctx context.Context, t *testing.T) {
type tenantCluster []*testTenant

func newTenantCluster(
t *testing.T, server serverutils.TestServerInterface, tenantClusterSize int, tenantID uint64,
t *testing.T,
server serverutils.TestServerInterface,
tenantClusterSize int,
tenantID uint64,
knobs base.TestingKnobs,
) tenantCluster {
t.Helper()

cluster := make([]*testTenant, tenantClusterSize)
existing := false
for i := 0; i < tenantClusterSize; i++ {
cluster[i] = newTestTenant(t, server, existing, roachpb.MakeTenantID(tenantID))
cluster[i] =
newTestTenant(t, server, existing, roachpb.MakeTenantID(tenantID), knobs)
existing = true
}

Expand Down
13 changes: 9 additions & 4 deletions pkg/cmd/reduce/main.go
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ var (
flags = flag.NewFlagSet(os.Args[0], flag.ExitOnError)
binary = flags.String("binary", "./cockroach", "path to cockroach binary")
file = flags.String("file", "", "the path to a file containing a SQL query to reduce")
verbose = flags.Bool("v", false, "print progress to standard output")
verbose = flags.Bool("v", false, "print progress to standard output and the original test case output if it is not interesting")
contains = flags.String("contains", "", "error regex to search for")
unknown = flags.Bool("unknown", false, "print unknown types during walk")
workers = flags.Int("goroutines", goroutines, "number of worker goroutines (defaults to NumCPU/3")
Expand Down Expand Up @@ -141,7 +141,7 @@ func reduceSQL(
chunkReducer = reducesql.NewSQLChunkReducer(chunkReductions)
}

interesting := func(ctx context.Context, sql string) bool {
isInteresting := func(ctx context.Context, sql string) (interesting bool, logOriginalHint func()) {
// Disable telemetry and license generation.
cmd := exec.CommandContext(ctx, binary,
"demo",
Expand All @@ -162,13 +162,18 @@ func reduceSQL(
case errors.HasType(err, (*os.PathError)(nil)):
log.Fatal(err)
}
return containsRE.Match(out)
if verbose {
logOriginalHint = func() {
fmt.Fprintf(logger, "output did not match regex %s:\n\n%s", contains, string(out))
}
}
return containsRE.Match(out), logOriginalHint
}

out, err := reduce.Reduce(
logger,
inputSQL,
interesting,
isInteresting,
workers,
reduce.ModeInteresting,
chunkReducer,
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/roachtest/tests/sqlsmith.go
Original file line number Diff line number Diff line change
Expand Up @@ -211,10 +211,11 @@ func registerSQLSmith(r registry.Registry) {
es := err.Error()
if strings.Contains(es, "internal error") {
// TODO(yuzefovich): we temporarily ignore internal errors
// that are because of #40929.
// that are because of #40929 and #70831.
var expectedError bool
for _, exp := range []string{
"could not parse \"0E-2019\" as type decimal",
"no volatility for cast tuple",
} {
expectedError = expectedError || strings.Contains(es, exp)
}
Expand Down
3 changes: 3 additions & 0 deletions pkg/rpc/auth_tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,9 @@ func (a tenantAuthorizer) authorize(
case "/cockroach.server.serverpb.Status/ListLocalSessions":
return a.authTenant(tenID)

case "/cockroach.server.serverpb.Status/IndexUsageStatistics":
return a.authTenant(tenID)

case "/cockroach.roachpb.Internal/GetSpanConfigs":
return a.authGetSpanConfigs(tenID, req.(*roachpb.GetSpanConfigsRequest))

Expand Down
52 changes: 7 additions & 45 deletions pkg/server/index_usage_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -54,51 +54,11 @@ func compareStatsHelper(
require.Equal(t, expected, actual)
}

func createIndexStatsIngestedCallback() (
func(key roachpb.IndexUsageKey),
chan roachpb.IndexUsageKey,
) {
// Create a buffered channel so the callback is non-blocking.
notify := make(chan roachpb.IndexUsageKey, 100)

cb := func(key roachpb.IndexUsageKey) {
notify <- key
}

return cb, notify
}

func waitForStatsIngestion(
t *testing.T,
notify chan roachpb.IndexUsageKey,
expectedKeys map[roachpb.IndexUsageKey]struct{},
expectedEventCnt int,
timeout time.Duration,
) {
var timer timeutil.Timer
eventCnt := 0

timer.Reset(timeout)

for eventCnt < expectedEventCnt {
select {
case key := <-notify:
if _, ok := expectedKeys[key]; ok {
eventCnt++
}
continue
case <-timer.C:
timer.Read = true
t.Fatalf("expected stats ingestion to complete within %s, but it timed out", timeout)
}
}
}

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

statsIngestionCb, statsIngestionNotifier := createIndexStatsIngestedCallback()
statsIngestionCb, statsIngestionNotifier := idxusage.CreateIndexStatsIngestedCallbackForTest()

testCluster := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Expand Down Expand Up @@ -214,10 +174,12 @@ func TestStatusAPIIndexUsage(t *testing.T) {
thirdLocalStatsReader := thirdServer.SQLServer().(*sql.Server).GetLocalIndexStatistics()

// Wait for the stats to be ingested.
waitForStatsIngestion(t, statsIngestionNotifier, map[roachpb.IndexUsageKey]struct{}{
indexKeyA: {},
indexKeyB: {},
}, /* expectedKeys */ 4 /* expectedEventCnt*/, 5*time.Second /* timeout */)
require.NoError(t,
idxusage.WaitForIndexStatsIngestionForTest(statsIngestionNotifier, map[roachpb.IndexUsageKey]struct{}{
indexKeyA: {},
indexKeyB: {},
}, /* expectedKeys */ 4 /* expectedEventCnt*/, 5*time.Second /* timeout */),
)

// First node should have nothing.
stats := firstLocalStatsReader.Get(indexKeyA.TableID, indexKeyA.IndexID)
Expand Down
1 change: 1 addition & 0 deletions pkg/server/tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -247,6 +247,7 @@ func StartTenant(
goroutineDumpDirName: args.GoroutineDumpDirName,
heapProfileDirName: args.HeapProfileDirName,
runtime: args.runtime,
sessionRegistry: args.sessionRegistry,
}); err != nil {
return nil, "", "", err
}
Expand Down
Loading

0 comments on commit a3ba197

Please sign in to comment.