Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
106072: server: close net.Conn unconditionally after accepting r=rafiss,knz a=ecwall

Informs #105448

Previously after the server accepted a connection, it was closed in multiple paths and in multiple layers after it was done being used. Now it is always closed in the same layer that accepted it after the serveConn callback returns.

Release note: None

106238: server: fix crdb_internal.cluster_inflight_traces in shared process mode r=yuzefovich a=yuzefovich

This commit fixes a panic that would previously occur when querying `crdb_internal.cluster_inflight_traces` virtual table when running in shared-process multi-tenant mode. In particular, the problem was that we tried to access node liveness which isn't available, and now we will fall back to the multi-tenant way of doing things (using the instances reader). Additionally, this commit extends the existing test to also run in shared-process multi-tenant config which serves as a regression test for this bug. There is no release note since it's not a user-visible bug.

Fixes: #106182.
Epic: CRDB-26691

Release note: None

106320: sql: unskip TestSchemaChangeGCJob r=chengxiong-ruan a=chengxiong-ruan

Informs: #85876
fixes #60664

A few assumptions has been changed since the test was skipped. For example, the first user defined table descriptor ID and expected error messages. There was actually also bugs in the error assertion in the test for some reason, that is fixed by this PR too.

Release note: None

106407: sql: add tests for CTAS, CMVAS with every SHOW statement r=chengxiong-ruan a=ecwall

Fixes #105895

Adds tests for CREATE TABLE AS, CREATE MATERIALIZED VIEW AS sourcing from
all SHOW statements.

Release note: None


106420: server: unskip TestDatabasesTablesV2 r=THardy98 a=THardy98

Resolves: #87074

This test no longer seems flaky after testing (via `./dev test pkg/server --filter=TestDatabasesTablesV2 --stress --stress-args="-p=4" --race --timeout=30m` on gceworker).

Release note: None

Co-authored-by: Evan Wall <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: Chengxiong Ruan <[email protected]>
Co-authored-by: Thomas Hardy <[email protected]>
  • Loading branch information
5 people committed Jul 7, 2023
6 parents 88c5907 + 62b92f2 + f65fd0b + ebee8de + 576bb93 + 84e8dc7 commit 38cd5c4
Show file tree
Hide file tree
Showing 13 changed files with 389 additions and 121 deletions.
2 changes: 0 additions & 2 deletions pkg/server/api_v2_sql_schema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -62,7 +61,6 @@ func TestUsersV2(t *testing.T) {

func TestDatabasesTablesV2(t *testing.T) {
defer leaktest.AfterTest(t)()
skip.UnderRaceWithIssue(t, 87074, "flaky test")
defer log.Scope(t).Close(t)

testCluster := serverutils.StartNewTestCluster(t, 3, base.TestClusterArgs{})
Expand Down
4 changes: 4 additions & 0 deletions pkg/server/server_controller_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,10 @@ func (c *serverController) sqlMux(
// The concurrent dispatch gives a chance to the succeeding
// servers to see and process the cancel at approximately the
// same time as every other.
//
// Cancel requests are unauthenticated so run the cancel async to prevent
// the client from deriving any info about the cancel based on how long it
// takes.
if err := c.stopper.RunAsyncTask(ctx, "cancel", func(ctx context.Context) {
s.handleCancel(ctx, status.CancelKey)
}); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -910,7 +910,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
// The collector requires nodeliveness to get a list of all the nodes in the
// cluster.
var getNodes func(ctx context.Context) ([]roachpb.NodeID, error)
if isMixedSQLAndKVNode {
if isMixedSQLAndKVNode && hasNodeLiveness {
// TODO(dt): any reason not to just always use the instance reader? And just
// pass it directly instead of making a new closure here?
getNodes = func(ctx context.Context) ([]roachpb.NodeID, error) {
Expand Down
8 changes: 6 additions & 2 deletions pkg/server/tenant.go
Original file line number Diff line number Diff line change
Expand Up @@ -833,8 +833,12 @@ func (s *SQLServerWrapper) serveConn(
pgServer := s.PGServer()
switch status.State {
case pgwire.PreServeCancel:
pgServer.HandleCancel(ctx, status.CancelKey)
return nil
// Cancel requests are unauthenticated so run the cancel async to prevent
// the client from deriving any info about the cancel based on how long it
// takes.
return s.stopper.RunAsyncTask(ctx, "cancel", func(ctx context.Context) {
pgServer.HandleCancel(ctx, status.CancelKey)
})
case pgwire.PreServeReady:
return pgServer.ServeConn(ctx, conn, status)
default:
Expand Down
215 changes: 215 additions & 0 deletions pkg/sql/create_as_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,221 @@ func TestCreateAsVTable(t *testing.T) {
waitForJobsSuccess(t, sqlRunner)
}

func TestCreateAsShow(t *testing.T) {
defer leaktest.AfterTest(t)()

testCases := []struct {
sql string
setup string
skip bool
}{
{
sql: "SHOW CLUSTER SETTINGS",
},
{
sql: "SHOW CLUSTER SETTINGS FOR TENANT [2]",
setup: "SELECT crdb_internal.create_tenant(2)",
},
{
sql: "SHOW DATABASES",
},
{
sql: "SHOW ENUMS",
setup: "CREATE TYPE e AS ENUM ('a', 'b')",
},
{
sql: "SHOW TYPES",
setup: "CREATE TYPE p AS (x int, y int)",
},
{
sql: "SHOW CREATE DATABASE defaultdb",
},
{
sql: "SHOW CREATE ALL SCHEMAS",
},
{
sql: "SHOW CREATE ALL TABLES",
},
{
sql: "SHOW CREATE TABLE show_create_tbl",
setup: "CREATE TABLE show_create_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_create_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW CREATE FUNCTION show_create_fn",
setup: "CREATE FUNCTION show_create_fn(i int) RETURNS INT AS 'SELECT i' LANGUAGE SQL",
// TODO(sql-foundations): Fix `unknown function: show_create_fn(): function undefined` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106268.
skip: true,
},
{
sql: "SHOW CREATE ALL TYPES",
},
{
sql: "SHOW INDEXES FROM DATABASE defaultdb",
},
{
sql: "SHOW INDEXES FROM show_indexes_tbl",
setup: "CREATE TABLE show_indexes_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_indexes_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW COLUMNS FROM show_columns_tbl",
setup: "CREATE TABLE show_columns_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_columns_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW CONSTRAINTS FROM show_constraints_tbl",
setup: "CREATE TABLE show_constraints_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_constraints_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW PARTITIONS FROM DATABASE defaultdb",
},
{
sql: "SHOW PARTITIONS FROM TABLE show_partitions_tbl",
setup: "CREATE TABLE show_partitions_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_partitions_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW PARTITIONS FROM INDEX show_partitions_idx_tbl@show_partitions_idx_tbl_pkey",
setup: "CREATE TABLE show_partitions_idx_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `relation "show_partitions_idx_tbl" does not exist` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106260.
skip: true,
},
{
sql: "SHOW GRANTS",
},
{
sql: "SHOW JOBS",
},
{
sql: "SHOW CHANGEFEED JOBS",
},
{
sql: "SHOW ALL CLUSTER STATEMENTS",
},
{
sql: "SHOW ALL LOCAL STATEMENTS",
},
{
sql: "SHOW ALL LOCAL STATEMENTS",
},
{
sql: "SHOW RANGES WITH DETAILS, KEYS, TABLES",
},
{
sql: "SHOW RANGE FROM TABLE show_ranges_tbl FOR ROW (0)",
setup: "CREATE TABLE show_ranges_tbl (id int PRIMARY KEY)",
// TODO(sql-foundations): Fix `invalid memory address or nil pointer dereference` error in job.
// See https://github.com/cockroachdb/cockroach/issues/106397.
skip: true,
},
{
sql: "SHOW SURVIVAL GOAL FROM DATABASE",
},
{
sql: "SHOW REGIONS FROM DATABASE",
},
{
sql: "SHOW GRANTS ON ROLE",
},
{
sql: "SHOW ROLES",
},
{
sql: "SHOW SCHEMAS",
},
{
sql: "SHOW SEQUENCES",
setup: "CREATE SEQUENCE seq",
},
{
sql: "SHOW ALL SESSIONS",
},
{
sql: "SHOW CLUSTER SESSIONS",
},
{
sql: "SHOW SYNTAX 'SELECT 1'",
},
{
sql: "SHOW FUNCTIONS",
setup: "CREATE FUNCTION show_functions_fn(i int) RETURNS INT AS 'SELECT i' LANGUAGE SQL",
},
{
sql: "SHOW TABLES",
},
{
sql: "SHOW ALL TRANSACTIONS",
},
{
sql: "SHOW CLUSTER TRANSACTIONS",
},
{
sql: "SHOW USERS",
},
{
sql: "SHOW ALL",
},
{
sql: "SHOW ZONE CONFIGURATIONS",
},
{
sql: "SHOW SCHEDULES",
},
{
sql: "SHOW JOBS FOR SCHEDULES SELECT id FROM [SHOW SCHEDULES]",
},
{
sql: "SHOW FULL TABLE SCANS",
},
{
sql: "SHOW DEFAULT PRIVILEGES",
},
}

ctx := context.Background()
testCluster := serverutils.StartNewTestCluster(t, 1, base.TestClusterArgs{})
defer testCluster.Stopper().Stop(ctx)
sqlRunner := sqlutils.MakeSQLRunner(testCluster.ServerConn(0))

for i, testCase := range testCases {
t.Run(testCase.sql, func(t *testing.T) {
if testCase.skip {
return
}
if testCase.setup != "" {
sqlRunner.Exec(t, testCase.setup)
}
createTableStmt := fmt.Sprintf(
"CREATE TABLE test_table_%d AS SELECT * FROM [%s]",
i, testCase.sql,
)
sqlRunner.Exec(t, createTableStmt)
createViewStmt := fmt.Sprintf(
"CREATE MATERIALIZED VIEW test_view_%d AS SELECT * FROM [%s]",
i, testCase.sql,
)
sqlRunner.Exec(t, createViewStmt)
i++
})
}

waitForJobsSuccess(t, sqlRunner)
}

func waitForJobsSuccess(t *testing.T, sqlRunner *sqlutils.SQLRunner) {
query := `SELECT job_id, status, error, description
FROM [SHOW JOBS]
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/gcjob_test/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,6 @@ go_test(
"//pkg/testutils",
"//pkg/testutils/jobutils",
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/util/hlc",
"//pkg/util/leaktest",
Expand Down
44 changes: 23 additions & 21 deletions pkg/sql/gcjob_test/gc_job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,7 +45,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/jobutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand All @@ -58,7 +57,6 @@ import (
// TODO(pbardea): Add more testing around the timer calculations.
func TestSchemaChangeGCJob(t *testing.T) {
defer leaktest.AfterTest(t)()
skip.WithIssue(t, 60664, "flaky test")

type DropItem int
const (
Expand All @@ -78,6 +76,7 @@ func TestSchemaChangeGCJob(t *testing.T) {
for _, ttlTime := range []TTLTime{PAST, SOON, FUTURE} {
blockGC := make(chan struct{}, 1)
params := base.TestServerArgs{}
params.ScanMaxIdleTime = time.Millisecond
params.Knobs.JobsTestingKnobs = jobs.NewTestingKnobsWithShortIntervals()
params.Knobs.GCJob = &sql.GCJobTestingKnobs{
RunBeforePerformGC: func(_ jobspb.JobID) error {
Expand All @@ -90,6 +89,11 @@ func TestSchemaChangeGCJob(t *testing.T) {
defer s.Stopper().Stop(ctx)
sqlDB := sqlutils.MakeSQLRunner(db)

sqlDB.Exec(t, `SET CLUSTER SETTING sql.gc_job.wait_for_gc.interval = '1s';`)
// Refresh protected timestamp cache immediately to make MVCC GC queue to
// process GC immediately.
sqlDB.Exec(t, `SET CLUSTER SETTING kv.protectedts.poll_interval = '1s';`)

jobRegistry := s.JobRegistry().(*jobs.Registry)

sqlDB.Exec(t, "CREATE DATABASE my_db")
Expand All @@ -100,9 +104,10 @@ func TestSchemaChangeGCJob(t *testing.T) {
sqlDB.Exec(t, "ALTER TABLE my_table CONFIGURE ZONE USING gc.ttlseconds = 1")
sqlDB.Exec(t, "ALTER TABLE my_other_table CONFIGURE ZONE USING gc.ttlseconds = 1")
}
myDBID := descpb.ID(bootstrap.TestingUserDescID(2))
myTableID := descpb.ID(bootstrap.TestingUserDescID(3))
myOtherTableID := descpb.ID(bootstrap.TestingUserDescID(4))

myDBID := descpb.ID(bootstrap.TestingUserDescID(4))
myTableID := descpb.ID(bootstrap.TestingUserDescID(6))
myOtherTableID := descpb.ID(bootstrap.TestingUserDescID(7))

var myTableDesc *tabledesc.Mutable
var myOtherTableDesc *tabledesc.Mutable
Expand Down Expand Up @@ -141,7 +146,7 @@ func TestSchemaChangeGCJob(t *testing.T) {
ParentID: myTableID,
}
myTableDesc.SetPublicNonPrimaryIndexes([]descpb.IndexDescriptor{})
expectedRunningStatus = "performing garbage collection on index 2"
expectedRunningStatus = "deleting data"
case TABLE:
details = jobspb.SchemaChangeGCDetails{
Tables: []jobspb.SchemaChangeGCDetails_DroppedID{
Expand All @@ -153,7 +158,7 @@ func TestSchemaChangeGCJob(t *testing.T) {
}
myTableDesc.State = descpb.DescriptorState_DROP
myTableDesc.DropTime = dropTime
expectedRunningStatus = fmt.Sprintf("performing garbage collection on table %d", myTableID)
expectedRunningStatus = "deleting data"
case DATABASE:
details = jobspb.SchemaChangeGCDetails{
Tables: []jobspb.SchemaChangeGCDetails_DroppedID{
Expand All @@ -172,7 +177,7 @@ func TestSchemaChangeGCJob(t *testing.T) {
myTableDesc.DropTime = dropTime
myOtherTableDesc.State = descpb.DescriptorState_DROP
myOtherTableDesc.DropTime = dropTime
expectedRunningStatus = fmt.Sprintf("performing garbage collection on tables %d, %d", myTableID, myOtherTableID)
expectedRunningStatus = "deleting data"
}

if err := kvDB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
Expand Down Expand Up @@ -238,24 +243,21 @@ func TestSchemaChangeGCJob(t *testing.T) {
if err := sql.TestingDescsTxn(ctx, s, func(ctx context.Context, txn isql.Txn, col *descs.Collection) error {
myImm, err := col.ByID(txn.KV()).Get().Table(ctx, myTableID)
if err != nil {
if ttlTime != FUTURE && (dropItem == TABLE || dropItem == DATABASE) {
// We dropped the table, so expect it to not be found.
require.EqualError(t, err, fmt.Sprintf(`relation "[%d]" does not exist`, myTableID))
return nil
}
return err
}
if ttlTime != FUTURE && (dropItem == TABLE || dropItem == DATABASE) {
// We dropped the table, so expect it to not be found.
require.EqualError(t, err, "descriptor not found")
return nil
}
myTableDesc = tabledesc.NewBuilder(myImm.TableDesc()).BuildExistingMutableTable()
myOtherImm, err := col.ByID(txn.KV()).Get().Table(ctx, myOtherTableID)
if err != nil {
return err
}
if ttlTime != FUTURE && dropItem == DATABASE {
// We dropped the entire database, so expect none of the tables to be found.
require.EqualError(t, err, "descriptor not found")
return nil
}
if err != nil {
if ttlTime != FUTURE && dropItem == DATABASE {
// We dropped the entire database, so expect none of the tables to be found.
require.EqualError(t, err, fmt.Sprintf(`relation "[%d]" does not exist`, myOtherTableID))
return nil
}
return err
}
myOtherTableDesc = tabledesc.NewBuilder(myOtherImm.TableDesc()).BuildExistingMutableTable()
Expand Down
Loading

0 comments on commit 38cd5c4

Please sign in to comment.