Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 8 additions & 1 deletion pkg/base/test_server_args.go
Original file line number Diff line number Diff line change
Expand Up @@ -326,7 +326,8 @@ type DefaultTestTenantOptions struct {
// warn".
noWarnImplicitInterfaces bool

// If test tenant is disabled, issue and label to link in log message.
// If test tenant is disabled, issue and label to link in log message. These
// can be left unset if one of the tenant modes is explicitly skipped.
issueNum int
label string
}
Expand Down Expand Up @@ -575,6 +576,12 @@ func TestDoesNotWorkWithExternalProcessMode(issueNumber int) DefaultTestTenantOp
return testSkippedForExternalProcessMode(issueNumber)
}

// TestSkipForExternalProcessMode disables selecting the external process
// virtual cluster for tests that are not applicable to that mode.
func TestSkipForExternalProcessMode() DefaultTestTenantOptions {
return testSkippedForExternalProcessMode(0 /* issueNumber */)
}

func testSkippedForExternalProcessMode(issueNumber int) DefaultTestTenantOptions {
return DefaultTestTenantOptions{
testBehavior: ttSharedProcess,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,8 +145,7 @@ func FetchDescVersionModificationTime(
tableName string,
version int,
) hlc.Timestamp {
db := serverutils.OpenDBConn(
t, s.SQLAddr(), dbName, false, s.AppStopper())
db := s.SQLConn(t, serverutils.DBName(dbName))

tblKey := s.Codec().IndexPrefix(keys.DescriptorTableID, keys.DescriptorTablePrimaryKeyIndexID)
header := kvpb.RequestHeader{
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/testccl/sqlccl/gc_job_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ func TestGCJobGetsMarkedIdle(t *testing.T) {
s, mainDB, _ := serverutils.StartServer(t, base.TestServerArgs{
DefaultTestTenant: base.TestControlsTenantsExplicitly,
})
sqltestutils.SetShortRangeFeedIntervals(t, mainDB)
sqltestutils.SetShortRangeFeedIntervals(t, s)
defer s.Stopper().Stop(ctx)
tenant, tenantDB := serverutils.StartTenant(t, s, base.TestTenantArgs{
TenantID: serverutils.TestTenantID(),
Expand Down
30 changes: 12 additions & 18 deletions pkg/kv/range_lookup.go
Original file line number Diff line number Diff line change
Expand Up @@ -450,10 +450,19 @@ func kvsToRangeDescriptors(kvs []roachpb.KeyValue) ([]roachpb.RangeDescriptor, e
// RangeLookup scan. It can return false positives and should only be used in
// tests.
func TestingIsRangeLookup(ba *kvpb.BatchRequest) bool {
if ba.IsSingleRequest() {
return TestingIsRangeLookupRequest(ba.Requests[0].GetInner())
if !ba.IsSingleRequest() {
return false
}
req := ba.Requests[0].GetInner()
switch req.(type) {
case *kvpb.ScanRequest:
case *kvpb.ReverseScanRequest:
default:
return false
}
return false
s := req.Header()
return rangeLookupStartKeyBounds.ContainsKey(s.Key) &&
rangeLookupEndKeyBounds.ContainsKey(s.EndKey)
}

// These spans bounds the start and end keys of the spans returned from
Expand All @@ -467,18 +476,3 @@ var rangeLookupEndKeyBounds = roachpb.Span{
Key: keys.Meta1Prefix.Next(),
EndKey: keys.SystemPrefix.Next(),
}

// TestingIsRangeLookupRequest returns if the provided Request looks like a single
// RangeLookup scan. It can return false positives and should only be used in
// tests.
func TestingIsRangeLookupRequest(req kvpb.Request) bool {
switch req.(type) {
case *kvpb.ScanRequest:
case *kvpb.ReverseScanRequest:
default:
return false
}
s := req.Header()
return rangeLookupStartKeyBounds.ContainsKey(s.Key) &&
rangeLookupEndKeyBounds.ContainsKey(s.EndKey)
}
4 changes: 0 additions & 4 deletions pkg/sql/as_of_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,6 @@ import (
"github.com/cockroachdb/apd/v3"
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/kv/kvpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/sql"
Expand Down Expand Up @@ -313,9 +312,6 @@ func TestAsOfRetry(t *testing.T) {

switch req := args.Req.(type) {
case *kvpb.GetRequest:
if kv.TestingIsRangeLookupRequest(req) {
return nil
}
for key, count := range magicVals.restartCounts {
if err := checkCorrectTxn(string(req.Key), magicVals, args.Hdr.Txn); err != nil {
return kvpb.NewError(err)
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/authorization_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -98,16 +98,16 @@ func TestConcurrentGrants(t *testing.T) {

runConcurrentGrantsWithPriority := func(t *testing.T, priority string) {
ctx := context.Background()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer srv.Stopper().Stop(ctx)
tdb := sqlutils.MakeSQLRunner(db)

// Shortening this cluster setting is essential for this test. A small value
// will force `txn1` to bump up its write timestamp by the time it gets to
// commit. This is needed to force `txn2` to encounter a WriteTooOldError
// after being unblocked, and hence enter a retry, as retry in `txn2` is
// prerequisite for the potential deadlock described in #117144.
tdb.Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '1ms'")
sqlutils.MakeSQLRunner(srv.SystemLayer().SQLConn(t)).Exec(t, "SET CLUSTER SETTING kv.closed_timestamp.target_duration = '1ms'")

tdb.Exec(t, "CREATE ROLE developer;")
tdb.Exec(t, "CREATE USER user1")
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/backfill_protected_timestamp_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -241,6 +241,7 @@ func TestBackfillQueryWithProtectedTS(t *testing.T) {
var db *gosql.DB
var tableID uint32
s, db, _ = serverutils.StartServer(t, base.TestServerArgs{
DefaultTestTenant: base.TestIsForStuffThatShouldWorkWithSecondaryTenantsButDoesntYet(156127),
Knobs: base.TestingKnobs{
SQLEvalContext: &eval.TestingKnobs{
ForceProductionValues: true,
Expand Down
34 changes: 14 additions & 20 deletions pkg/sql/conn_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -643,7 +643,9 @@ func TestPrepareInExplicitTransactionDoesNotDeadlock(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{})
s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{
DefaultTestTenant: base.TestDoesNotWorkWithExternalProcessMode(156146),
})
defer s.Stopper().Stop(context.Background())

testDB := sqlutils.MakeSQLRunner(sqlDB)
Expand Down Expand Up @@ -1737,8 +1739,11 @@ func TestAbortedTxnLocks(t *testing.T) {
defer log.Scope(t).Close(t)
ctx := context.Background()

s := serverutils.StartServerOnly(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
srv := serverutils.StartServerOnly(t, base.TestServerArgs{
DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(156127),
})
defer srv.Stopper().Stop(ctx)
s := srv.ApplicationLayer()

var TransactionStatus string

Expand Down Expand Up @@ -2425,22 +2430,19 @@ func TestInternalAppNamePrefix(t *testing.T) {
ctx := context.Background()
params := base.TestServerArgs{}
params.Insecure = true
s, sqlDB, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(ctx)
srv, sqlDB, _ := serverutils.StartServer(t, params)
defer srv.Stopper().Stop(ctx)
s := srv.ApplicationLayer()

// Create a test table.
_, err := sqlDB.Exec("CREATE TABLE test (k INT PRIMARY KEY, v INT)")
require.NoError(t, err)

t.Run("app name set at conn init", func(t *testing.T) {
// Create a connection.
connURL := url.URL{
Scheme: "postgres",
User: url.User(username.RootUser),
Host: s.AdvSQLAddr(),
}
connURL, cleanup := s.PGUrl(t, serverutils.User(username.RootUser))
defer cleanup()
q := connURL.Query()
q.Add("sslmode", "disable")
q.Add("application_name", catconstants.InternalAppNamePrefix+"mytest")
connURL.RawQuery = q.Encode()
db, err := gosql.Open("postgres", connURL.String())
Expand All @@ -2462,15 +2464,7 @@ func TestInternalAppNamePrefix(t *testing.T) {

t.Run("app name set in session", func(t *testing.T) {
// Create a connection.
connURL := url.URL{
Scheme: "postgres",
User: url.User(username.RootUser),
Host: s.AdvSQLAddr(),
RawQuery: "sslmode=disable",
}
db, err := gosql.Open("postgres", connURL.String())
require.NoError(t, err)
defer db.Close()
db := s.SQLConn(t, serverutils.User(username.RootUser))
runner := sqlutils.MakeSQLRunner(db)

// Get initial metric values
Expand Down
36 changes: 20 additions & 16 deletions pkg/sql/crdb_internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@ import (
"context"
gosql "database/sql"
"fmt"
"net/url"
"strings"
"sync"
"sync/atomic"
Expand All @@ -32,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptstorage"
"github.com/cockroachdb/cockroach/pkg/multitenant/tenantcapabilitiespb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/scheduledjobs"
"github.com/cockroachdb/cockroach/pkg/security/username"
Expand Down Expand Up @@ -122,7 +122,8 @@ func TestRangeLocalityBasedOnNodeIDs(t *testing.T) {
tc := testcluster.StartTestCluster(t, 1,
base.TestClusterArgs{
ServerArgs: base.TestServerArgs{
Locality: roachpb.Locality{Tiers: []roachpb.Tier{{Key: "node", Value: "1"}}},
Locality: roachpb.Locality{Tiers: []roachpb.Tier{{Key: "node", Value: "1"}}},
DefaultTestTenant: base.TestIsSpecificToStorageLayerAndNeedsASystemTenant,
},
ReplicationMode: base.ReplicationAuto,
},
Expand Down Expand Up @@ -581,6 +582,11 @@ func TestDistSQLFlowsVirtualTables(t *testing.T) {
})
defer tc.Stopper().Stop(context.Background())

if tc.DefaultTenantDeploymentMode().IsExternal() {
tc.GrantTenantCapabilities(context.Background(), t, serverutils.TestTenantID(),
map[tenantcapabilitiespb.ID]string{tenantcapabilitiespb.CanAdminRelocateRange: "true"})
}

// Create a table with 3 rows, split them into 3 ranges with each node
// having one.
db := tc.ServerConn(gatewayNodeID)
Expand Down Expand Up @@ -855,13 +861,17 @@ func TestTxnContentionEventsTableWithRangeDescriptor(t *testing.T) {
defer log.Scope(t).Close(t)

ctx := context.Background()
s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
srv, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{
DefaultTestTenant: base.TestDoesNotWorkWithSecondaryTenantsButWeDontKnowWhyYet(156145),
})
defer srv.Stopper().Stop(ctx)
s := srv.ApplicationLayer()

_, err := sqlDB.Exec("SET CLUSTER SETTING sql.contention.event_store.resolution_interval = '10ms'")
require.NoError(t, err)
rangeKey := "/Local/Range/Table/106/1/-1704619207610523008/RangeDescriptor"
rangeKeyEscaped := fmt.Sprintf("\"%s\"", rangeKey)
s.ApplicationLayer().ExecutorConfig().(sql.ExecutorConfig).ContentionRegistry.AddContentionEvent(contentionpb.ExtendedContentionEvent{
s.ExecutorConfig().(sql.ExecutorConfig).ContentionRegistry.AddContentionEvent(contentionpb.ExtendedContentionEvent{
BlockingEvent: kvpb.ContentionEvent{
Key: roachpb.Key(rangeKey),
TxnMeta: enginepb.TxnMeta{
Expand Down Expand Up @@ -1367,33 +1377,27 @@ func TestInternalSystemJobsAccess(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

s, db, _ := serverutils.StartServer(t, base.TestServerArgs{
srv, db, _ := serverutils.StartServer(t, base.TestServerArgs{
Knobs: base.TestingKnobs{
KeyVisualizer: &keyvisualizer.TestingKnobs{SkipJobBootstrap: true},
},
})
ctx := context.Background()
defer s.Stopper().Stop(ctx)
defer srv.Stopper().Stop(ctx)
s := srv.ApplicationLayer()
rootDB := sqlutils.MakeSQLRunner(db)

// Even though this test modifies the system.jobs table and asserts its contents, we
// do not disable background job creation nor job adoption. This is because creating
// users requires jobs to be created and run. Thus, this test only creates jobs of type
// jobspb.TypeImport and overrides the import resumer.
registry := s.ApplicationLayer().JobRegistry().(*jobs.Registry)
registry := s.JobRegistry().(*jobs.Registry)
registry.TestingWrapResumerConstructor(jobspb.TypeImport, func(r jobs.Resumer) jobs.Resumer {
return &fakeResumer{}
})

asUser := func(user string, f func(userDB *sqlutils.SQLRunner)) {
pgURL := url.URL{
Scheme: "postgres",
User: url.UserPassword(user, "test"),
Host: s.AdvSQLAddr(),
}
db2, err := gosql.Open("postgres", pgURL.String())
assert.NoError(t, err)
defer db2.Close()
db2 := s.SQLConn(t, serverutils.UserPassword(user, "test"), serverutils.ClientCerts(false))
userDB := sqlutils.MakeSQLRunner(db2)

f(userDB)
Expand Down
8 changes: 8 additions & 0 deletions pkg/sql/create_as_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/testutils/kvclientutils"
"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/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand All @@ -30,6 +31,8 @@ func TestCreateAsVTable(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

skip.UnderRace(t, "too slow")

ctx := context.Background()
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
Expand Down Expand Up @@ -83,12 +86,17 @@ func TestCreateAsVTable(t *testing.T) {
`"".crdb_internal.gossip_liveness`: {},
`"".crdb_internal.gossip_nodes`: {},
`"".crdb_internal.kv_flow_controller`: {},
`"".crdb_internal.kv_flow_controller_v2`: {},
`"".crdb_internal.kv_flow_control_handles`: {},
`"".crdb_internal.kv_flow_control_handles_v2`: {},
`"".crdb_internal.kv_flow_token_deductions`: {},
`"".crdb_internal.kv_flow_token_deductions_v2`: {},
`"".crdb_internal.kv_node_status`: {},
`"".crdb_internal.kv_node_liveness`: {},
`"".crdb_internal.kv_store_status`: {},
`"".crdb_internal.node_tenant_capabilities_cache`: {},
`"".crdb_internal.store_liveness_support_for`: {},
`"".crdb_internal.store_liveness_support_from`: {},
`"".crdb_internal.tenant_usage_details`: {},
}
if _, ok := onlySystemTenant[fqName]; ok {
Expand Down
4 changes: 4 additions & 0 deletions pkg/sql/create_stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -59,6 +59,10 @@ func TestStatsWithLowTTL(t *testing.T) {
},
},
},
// In external-process mode the tenant doesn't have kvpb.GCRequest
// capability (and this capability can't be granted at the time of
// writing either), so we skip the external mode only.
DefaultTestTenant: base.TestSkipForExternalProcessMode(),
})
defer s.Stopper().Stop(context.Background())

Expand Down
6 changes: 5 additions & 1 deletion pkg/sql/distsql_physical_planner_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -278,6 +278,10 @@ func TestDistSQLRangeCachesIntegrationTest(t *testing.T) {
ReplicationMode: base.ReplicationManual,
ServerArgs: base.TestServerArgs{
UseDatabase: "test",
// Probably this test could work in shared-process mode, but
// it's occasionally flaking there and doesn't seem worth
// investigating since we're touching the ranges directly.
DefaultTestTenant: base.TestIsSpecificToStorageLayerAndNeedsASystemTenant,
},
})
defer tc.Stopper().Stop(context.Background())
Expand All @@ -298,7 +302,7 @@ func TestDistSQLRangeCachesIntegrationTest(t *testing.T) {
//
// TODO(andrei): This is super hacky. What this test really wants to do is to
// precisely control the contents of the range cache on node 4.
tc.Server(3).DistSenderI().(*kvcoord.DistSender).DisableFirstRangeUpdates()
tc.ApplicationLayer(3).DistSenderI().(*kvcoord.DistSender).DisableFirstRangeUpdates()
db3 := tc.ServerConn(3)
// Force the DistSQL on this connection.
_, err := db3.Exec(`SET CLUSTER SETTING sql.defaults.distsql = always;`)
Expand Down
Loading