diff --git a/docs/generated/sql/functions.md b/docs/generated/sql/functions.md index 8d36fc521eb7..2e4f39274e5e 100644 --- a/docs/generated/sql/functions.md +++ b/docs/generated/sql/functions.md @@ -3072,6 +3072,18 @@ table. Returns an error if validation fails.

session_user() → string

Returns the session user. This function is provided for compatibility with PostgreSQL.

+to_regclass(text: string) → regtype

Translates a textual relation name to its OID

+
+to_regnamespace(text: string) → regtype

Translates a textual schema name to its OID

+
+to_regproc(text: string) → regtype

Translates a textual function or procedure name to its OID

+
+to_regprocedure(text: string) → regtype

Translates a textual function or procedure name(with argument types) to its OID

+
+to_regrole(text: string) → regtype

Translates a textual role name to its OID

+
+to_regtype(text: string) → regtype

Translates a textual type name to its OID

+
version() → string

Returns the node’s version of CockroachDB.

diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index bf8043007d32..e9a3003815a5 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -98,6 +98,7 @@ ALL_TESTS = [ "//pkg/cmd/roachtest:roachtest_test", "//pkg/cmd/teamcity-trigger:teamcity-trigger_test", "//pkg/cmd/testfilter:testfilter_test", + "//pkg/col/coldata:coldata_disallowed_imports_test", "//pkg/col/coldata:coldata_test", "//pkg/col/coldataext:coldataext_test", "//pkg/col/colserde:colserde_test", @@ -235,25 +236,34 @@ ALL_TESTS = [ "//pkg/sql/colconv:colconv_test", "//pkg/sql/colencoding:colencoding_test", "//pkg/sql/colexec/colbuilder:colbuilder_test", - "//pkg/sql/colexec/colexecagg:colexecagg_test", + "//pkg/sql/colexec/colexecagg:colexecagg_disallowed_imports_test", "//pkg/sql/colexec/colexecargs:colexecargs_test", + "//pkg/sql/colexec/colexecbase:colexecbase_disallowed_imports_test", "//pkg/sql/colexec/colexecbase:colexecbase_test", "//pkg/sql/colexec/colexeccmp:colexeccmp_test", + "//pkg/sql/colexec/colexechash:colexechash_disallowed_imports_test", "//pkg/sql/colexec/colexechash:colexechash_test", + "//pkg/sql/colexec/colexecjoin:colexecjoin_disallowed_imports_test", "//pkg/sql/colexec/colexecjoin:colexecjoin_test", + "//pkg/sql/colexec/colexecproj:colexecproj_disallowed_imports_test", "//pkg/sql/colexec/colexecproj:colexecproj_test", + "//pkg/sql/colexec/colexecsel:colexecsel_disallowed_imports_test", "//pkg/sql/colexec/colexecsel:colexecsel_test", + "//pkg/sql/colexec/colexecspan:colexecspan_disallowed_imports_test", "//pkg/sql/colexec/colexecspan:colexecspan_test", "//pkg/sql/colexec/colexectestutils:colexectestutils_test", "//pkg/sql/colexec/colexecutils:colexecutils_test", + "//pkg/sql/colexec/colexecwindow:colexecwindow_disallowed_imports_test", "//pkg/sql/colexec/colexecwindow:colexecwindow_test", - "//pkg/sql/colexec/execgen/cmd/execgen:execgen_test", + "//pkg/sql/colexec/execgen/cmd/execgen:execgen_disallowed_imports_test", "//pkg/sql/colexec/execgen:execgen_test", + "//pkg/sql/colexec:colexec_disallowed_imports_test", "//pkg/sql/colexec:colexec_test", "//pkg/sql/colexecerror:colexecerror_test", - "//pkg/sql/colexecop:colexecop_test", + "//pkg/sql/colexecop:colexecop_disallowed_imports_test", "//pkg/sql/colfetcher:colfetcher_test", "//pkg/sql/colflow/colrpc:colrpc_test", + "//pkg/sql/colflow:colflow_disallowed_imports_test", "//pkg/sql/colflow:colflow_test", "//pkg/sql/colmem:colmem_test", "//pkg/sql/contention/contentionutils:contentionutils_test", @@ -265,18 +275,22 @@ ALL_TESTS = [ "//pkg/sql/distsql:distsql_test", "//pkg/sql/doctor:doctor_test", "//pkg/sql/enum:enum_test", + "//pkg/sql/execinfra:execinfra_disallowed_imports_test", "//pkg/sql/execinfra:execinfra_test", "//pkg/sql/execinfrapb:execinfrapb_disallowed_imports_test", "//pkg/sql/execinfrapb:execinfrapb_test", "//pkg/sql/execstats:execstats_test", + "//pkg/sql/flowinfra:flowinfra_disallowed_imports_test", "//pkg/sql/flowinfra:flowinfra_test", "//pkg/sql/gcjob/gcjobnotifier:gcjobnotifier_test", "//pkg/sql/gcjob:gcjob_test", "//pkg/sql/gcjob_test:gcjob_test_test", "//pkg/sql/idxusage:idxusage_test", "//pkg/sql/importer:importer_test", + "//pkg/sql/inverted:inverted_disallowed_imports_test", "//pkg/sql/inverted:inverted_test", "//pkg/sql/kvstreamer:kvstreamer_test", + "//pkg/sql/lex:lex_disallowed_imports_test", "//pkg/sql/lex:lex_test", "//pkg/sql/lexbase:lexbase_test", "//pkg/sql/logictest:logictest_test", @@ -326,7 +340,9 @@ ALL_TESTS = [ "//pkg/sql/rowenc/keyside:keyside_test", "//pkg/sql/rowenc/valueside:valueside_test", "//pkg/sql/rowenc:rowenc_test", + "//pkg/sql/rowexec:rowexec_disallowed_imports_test", "//pkg/sql/rowexec:rowexec_test", + "//pkg/sql/rowflow:rowflow_disallowed_imports_test", "//pkg/sql/rowflow:rowflow_test", "//pkg/sql/scanner:scanner_test", "//pkg/sql/scheduledlogging:scheduledlogging_test", @@ -361,6 +377,7 @@ ALL_TESTS = [ "//pkg/sql/stmtdiagnostics:stmtdiagnostics_test", "//pkg/sql/tests:tests_test", "//pkg/sql/ttl/ttljob:ttljob_test", + "//pkg/sql/types:types_disallowed_imports_test", "//pkg/sql/types:types_test", "//pkg/sql:sql_test", "//pkg/startupmigrations/leasemanager:leasemanager_test", @@ -416,6 +433,7 @@ ALL_TESTS = [ "//pkg/util/interval/generic:generic_test", "//pkg/util/interval:interval_test", "//pkg/util/ipaddr:ipaddr_test", + "//pkg/util/json:json_disallowed_imports_test", "//pkg/util/json:json_test", "//pkg/util/limit:limit_test", "//pkg/util/log/eventpb:eventpb_test", diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 321a750e7954..2617d31fd96b 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -16,6 +16,7 @@ go_library( "backup_processor_planning.go", "backup_span_coverage.go", "create_scheduled_backup.go", + "encryption.go", "incrementals.go", "key_rewriter.go", "manifest_handling.go", @@ -118,7 +119,6 @@ go_library( "//pkg/util/contextutil", "//pkg/util/ctxgroup", "//pkg/util/encoding", - "//pkg/util/errorutil/unimplemented", "//pkg/util/hlc", "//pkg/util/interval", "//pkg/util/ioctx", diff --git a/pkg/ccl/backupccl/alter_backup_planning.go b/pkg/ccl/backupccl/alter_backup_planning.go index 17b86852e485..40d132f9596e 100644 --- a/pkg/ccl/backupccl/alter_backup_planning.go +++ b/pkg/ccl/backupccl/alter_backup_planning.go @@ -9,20 +9,16 @@ package backupccl import ( - "bytes" "context" - "fmt" "net/url" "path" "strings" - "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/errors" ) @@ -217,19 +213,6 @@ func doAlterBackupPlan( return writeNewEncryptionInfoToBackup(ctx, encryptionInfo, baseStore, len(opts)) } -func writeNewEncryptionInfoToBackup( - ctx context.Context, opts *jobspb.EncryptionInfo, dest cloud.ExternalStorage, numFiles int, -) error { - // New encryption-info file name is in the format "ENCRYPTION-INFO-" - newEncryptionInfoFile := fmt.Sprintf("%s-%d", backupEncryptionInfoFile, numFiles+1) - - buf, err := protoutil.Marshal(opts) - if err != nil { - return err - } - return cloud.WriteFile(ctx, dest, newEncryptionInfoFile, bytes.NewReader(buf)) -} - func init() { sql.AddPlanHook("alter backup", alterBackupPlanHook) } diff --git a/pkg/ccl/backupccl/backup_destination.go b/pkg/ccl/backupccl/backup_destination.go index aea98a6fdcc9..b2903f939518 100644 --- a/pkg/ccl/backupccl/backup_destination.go +++ b/pkg/ccl/backupccl/backup_destination.go @@ -13,7 +13,6 @@ import ( "net/url" "strings" - "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -251,55 +250,6 @@ func getBackupManifests( return manifests, memMu.total, nil } -// getEncryptionFromBase retrieves the encryption options of a base backup. It -// is expected that incremental backups use the same encryption options as the -// base backups. -func getEncryptionFromBase( - ctx context.Context, - user security.SQLUsername, - makeCloudStorage cloud.ExternalStorageFromURIFactory, - baseBackupURI string, - encryptionParams jobspb.BackupEncryptionOptions, - kmsEnv cloud.KMSEnv, -) (*jobspb.BackupEncryptionOptions, error) { - var encryptionOptions *jobspb.BackupEncryptionOptions - if encryptionParams.Mode != jobspb.EncryptionMode_None { - exportStore, err := makeCloudStorage(ctx, baseBackupURI, user) - if err != nil { - return nil, err - } - defer exportStore.Close() - opts, err := readEncryptionOptions(ctx, exportStore) - if err != nil { - return nil, err - } - - switch encryptionParams.Mode { - case jobspb.EncryptionMode_Passphrase: - encryptionOptions = &jobspb.BackupEncryptionOptions{ - Mode: jobspb.EncryptionMode_Passphrase, - Key: storageccl.GenerateKey([]byte(encryptionParams.RawPassphrae), opts[0].Salt), - } - case jobspb.EncryptionMode_KMS: - var defaultKMSInfo *jobspb.BackupEncryptionOptions_KMSInfo - for _, encFile := range opts { - defaultKMSInfo, err = validateKMSURIsAgainstFullBackup(encryptionParams.RawKmsUris, - newEncryptedDataKeyMapFromProtoMap(encFile.EncryptedDataKeyByKMSMasterKeyID), kmsEnv) - if err == nil { - break - } - } - if err != nil { - return nil, err - } - encryptionOptions = &jobspb.BackupEncryptionOptions{ - Mode: jobspb.EncryptionMode_KMS, - KMSInfo: defaultKMSInfo} - } - } - return encryptionOptions, nil -} - func readLatestFile( ctx context.Context, collectionURI string, diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index 0599a89259a2..d603939bfb20 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -11,8 +11,6 @@ package backupccl import ( "bytes" "context" - "crypto" - cryptorand "crypto/rand" "fmt" "net/url" "sort" @@ -110,51 +108,6 @@ type ( } ) -func newEncryptedDataKeyMap() *encryptedDataKeyMap { - return &encryptedDataKeyMap{make(map[hashedMasterKeyID][]byte)} -} - -func newEncryptedDataKeyMapFromProtoMap(protoDataKeyMap map[string][]byte) *encryptedDataKeyMap { - encMap := &encryptedDataKeyMap{make(map[hashedMasterKeyID][]byte)} - for k, v := range protoDataKeyMap { - encMap.m[hashedMasterKeyID(k)] = v - } - - return encMap -} - -func (e *encryptedDataKeyMap) addEncryptedDataKey( - masterKeyID plaintextMasterKeyID, encryptedDataKey []byte, -) { - // Hash the master key ID before writing to the map. - hasher := crypto.SHA256.New() - hasher.Write([]byte(masterKeyID)) - hash := hasher.Sum(nil) - e.m[hashedMasterKeyID(hash)] = encryptedDataKey -} - -func (e *encryptedDataKeyMap) getEncryptedDataKey( - masterKeyID plaintextMasterKeyID, -) ([]byte, error) { - // Hash the master key ID before reading from the map. - hasher := crypto.SHA256.New() - hasher.Write([]byte(masterKeyID)) - hash := hasher.Sum(nil) - var encDataKey []byte - var ok bool - if encDataKey, ok = e.m[hashedMasterKeyID(hash)]; !ok { - return nil, errors.New("could not find an entry in the encryptedDataKeyMap") - } - - return encDataKey, nil -} - -func (e *encryptedDataKeyMap) rangeOverMap(fn func(masterKeyID hashedMasterKeyID, dataKey []byte)) { - for k, v := range e.m { - fn(k, v) - } -} - // getPublicIndexTableSpans returns all the public index spans of the // provided table. func getPublicIndexTableSpans( @@ -433,54 +386,6 @@ func backupJobDescription( return tree.AsStringWithFQNames(b, ann), nil } -// validateKMSURIsAgainstFullBackup ensures that the KMS URIs provided to an -// incremental BACKUP are a subset of those used during the full BACKUP. It does -// this by ensuring that the KMS master key ID of each KMS URI specified during -// the incremental BACKUP can be found in the map written to `encryption-info` -// during a base BACKUP. -// -// The method also returns the KMSInfo to be used for all subsequent -// encryption/decryption operations during this BACKUP. By default it is the -// first KMS URI passed during the incremental BACKUP. -func validateKMSURIsAgainstFullBackup( - kmsURIs []string, kmsMasterKeyIDToDataKey *encryptedDataKeyMap, kmsEnv cloud.KMSEnv, -) (*jobspb.BackupEncryptionOptions_KMSInfo, error) { - var defaultKMSInfo *jobspb.BackupEncryptionOptions_KMSInfo - for _, kmsURI := range kmsURIs { - kms, err := cloud.KMSFromURI(kmsURI, kmsEnv) - if err != nil { - return nil, err - } - - defer func() { - _ = kms.Close() - }() - - // Depending on the KMS specific implementation, this may or may not contact - // the remote KMS. - id, err := kms.MasterKeyID() - if err != nil { - return nil, err - } - - encryptedDataKey, err := kmsMasterKeyIDToDataKey.getEncryptedDataKey(plaintextMasterKeyID(id)) - if err != nil { - return nil, - errors.Wrap(err, - "one of the provided URIs was not used when encrypting the base BACKUP") - } - - if defaultKMSInfo == nil { - defaultKMSInfo = &jobspb.BackupEncryptionOptions_KMSInfo{ - Uri: kmsURI, - EncryptedDataKey: encryptedDataKey, - } - } - } - - return defaultKMSInfo, nil -} - // annotatedBackupStatement is a tree.Backup, optionally // annotated with the scheduling information. type annotatedBackupStatement struct { @@ -1345,53 +1250,6 @@ func getReintroducedSpans( return tableSpans, nil } -func makeNewEncryptionOptions( - ctx context.Context, encryptionParams jobspb.BackupEncryptionOptions, kmsEnv cloud.KMSEnv, -) (*jobspb.BackupEncryptionOptions, *jobspb.EncryptionInfo, error) { - var encryptionOptions *jobspb.BackupEncryptionOptions - var encryptionInfo *jobspb.EncryptionInfo - switch encryptionParams.Mode { - case jobspb.EncryptionMode_Passphrase: - salt, err := storageccl.GenerateSalt() - if err != nil { - return nil, nil, err - } - - encryptionInfo = &jobspb.EncryptionInfo{Salt: salt} - encryptionOptions = &jobspb.BackupEncryptionOptions{ - Mode: jobspb.EncryptionMode_Passphrase, - Key: storageccl.GenerateKey([]byte(encryptionParams.RawPassphrae), salt), - } - case jobspb.EncryptionMode_KMS: - // Generate a 32 byte/256-bit crypto-random number which will serve as - // the data key for encrypting the BACKUP data and manifest files. - plaintextDataKey := make([]byte, 32) - _, err := cryptorand.Read(plaintextDataKey) - if err != nil { - return nil, nil, errors.Wrap(err, "failed to generate DataKey") - } - - encryptedDataKeyByKMSMasterKeyID, defaultKMSInfo, err := - getEncryptedDataKeyByKMSMasterKeyID(ctx, encryptionParams.RawKmsUris, plaintextDataKey, kmsEnv) - if err != nil { - return nil, nil, err - } - - encryptedDataKeyMapForProto := make(map[string][]byte) - encryptedDataKeyByKMSMasterKeyID.rangeOverMap( - func(masterKeyID hashedMasterKeyID, dataKey []byte) { - encryptedDataKeyMapForProto[string(masterKeyID)] = dataKey - }) - - encryptionInfo = &jobspb.EncryptionInfo{EncryptedDataKeyByKMSMasterKeyID: encryptedDataKeyMapForProto} - encryptionOptions = &jobspb.BackupEncryptionOptions{ - Mode: jobspb.EncryptionMode_KMS, - KMSInfo: defaultKMSInfo, - } - } - return encryptionOptions, encryptionInfo, nil -} - func getProtectedTimestampTargetForBackup(backupManifest BackupManifest) *ptpb.Target { if backupManifest.DescriptorCoverage == tree.AllDescriptors { return ptpb.MakeClusterTarget() @@ -1462,74 +1320,6 @@ func protectTimestampForBackup( return nil } -func getEncryptedDataKeyFromURI( - ctx context.Context, plaintextDataKey []byte, kmsURI string, kmsEnv cloud.KMSEnv, -) (string, []byte, error) { - kms, err := cloud.KMSFromURI(kmsURI, kmsEnv) - if err != nil { - return "", nil, err - } - - defer func() { - _ = kms.Close() - }() - - kmsURL, err := url.ParseRequestURI(kmsURI) - if err != nil { - return "", nil, errors.Wrap(err, "cannot parse KMSURI") - } - encryptedDataKey, err := kms.Encrypt(ctx, plaintextDataKey) - if err != nil { - return "", nil, errors.Wrapf(err, "failed to encrypt data key for KMS scheme %s", - kmsURL.Scheme) - } - - masterKeyID, err := kms.MasterKeyID() - if err != nil { - return "", nil, errors.Wrapf(err, "failed to get master key ID for KMS scheme %s", - kmsURL.Scheme) - } - - return masterKeyID, encryptedDataKey, nil -} - -// getEncryptedDataKeyByKMSMasterKeyID constructs a mapping {MasterKeyID : -// EncryptedDataKey} for each KMS URI provided during a full BACKUP. The -// MasterKeyID is hashed before writing it to the map. -// -// The method also returns the KMSInfo to be used for all subsequent -// encryption/decryption operations during this BACKUP. By default it is the -// first KMS URI. -func getEncryptedDataKeyByKMSMasterKeyID( - ctx context.Context, kmsURIs []string, plaintextDataKey []byte, kmsEnv cloud.KMSEnv, -) (*encryptedDataKeyMap, *jobspb.BackupEncryptionOptions_KMSInfo, error) { - encryptedDataKeyByKMSMasterKeyID := newEncryptedDataKeyMap() - // The coordinator node contacts every KMS and records the encrypted data - // key for each one. - var kmsInfo *jobspb.BackupEncryptionOptions_KMSInfo - for _, kmsURI := range kmsURIs { - masterKeyID, encryptedDataKey, err := getEncryptedDataKeyFromURI(ctx, - plaintextDataKey, kmsURI, kmsEnv) - if err != nil { - return nil, nil, err - } - - // By default we use the first KMS URI and encrypted data key for subsequent - // encryption/decryption operation during a BACKUP. - if kmsInfo == nil { - kmsInfo = &jobspb.BackupEncryptionOptions_KMSInfo{ - Uri: kmsURI, - EncryptedDataKey: encryptedDataKey, - } - } - - encryptedDataKeyByKMSMasterKeyID.addEncryptedDataKey(plaintextMasterKeyID(masterKeyID), - encryptedDataKey) - } - - return encryptedDataKeyByKMSMasterKeyID, kmsInfo, nil -} - // checkForNewDatabases returns an error if any new complete databases were // introduced. func checkForNewCompleteDatabases( diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index e570b95af286..e8a3f857b616 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -6660,12 +6660,12 @@ func TestBackupRestoreInsideTenant(t *testing.T) { tenant10C2.CheckQueryResults(t, `SELECT * FROM foo.bar`, tenant10.QueryStr(t, `SELECT * FROM foo.bar`)) }) t.Run("into-different-tenant-id", func(t *testing.T) { - tenant11C2.ExpectErr(t, `cannot cluster RESTORE backups taken from different tenant: 10`, - `RESTORE FROM $1`, httpAddr) + tenant11C2.Exec(t, `RESTORE FROM $1`, httpAddr) + tenant11C2.CheckQueryResults(t, `SELECT * FROM foo.bar`, tenant10.QueryStr(t, `SELECT * FROM foo.bar`)) }) t.Run("into-system-tenant-id", func(t *testing.T) { - systemDB2.ExpectErr(t, `cannot cluster RESTORE backups taken from different tenant: 10`, - `RESTORE FROM $1`, httpAddr) + systemDB2.Exec(t, `RESTORE FROM $1`, httpAddr) + systemDB2.CheckQueryResults(t, `SELECT * FROM foo.bar`, tenant10.QueryStr(t, `SELECT * FROM foo.bar`)) }) }) @@ -6717,8 +6717,7 @@ func TestBackupRestoreInsideTenant(t *testing.T) { defer cleanupEmptyCluster() emptySystemDB.Exec(t, `BACKUP TO $1`, httpAddrEmpty) - tenant20C2.ExpectErr(t, `cannot cluster RESTORE backups taken from different tenant: system`, - `RESTORE FROM $1`, httpAddrEmpty) + tenant20C2.Exec(t, `RESTORE FROM $1`, httpAddrEmpty) }) }) @@ -6731,6 +6730,50 @@ func TestBackupRestoreInsideTenant(t *testing.T) { }) } +// TestBackupRestoreTenantSettings tests the behaviour of the custom restore function for +// the system.tenant_settings table. +func TestBackupRestoreTenantSettings(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + const numAccounts = 1 + + makeTenant := func(srv serverutils.TestServerInterface, tenant uint64) (*sqlutils.SQLRunner, func()) { + _, conn := serverutils.StartTenant(t, srv, base.TestTenantArgs{TenantID: roachpb.MakeTenantID(tenant)}) + cleanup := func() { conn.Close() } + return sqlutils.MakeSQLRunner(conn), cleanup + } + tc, systemDB, dir, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + _, _ = tc, systemDB + defer cleanupFn() + + // NB: tenant certs for 10, 11, and 20 are embedded. See: + _ = security.EmbeddedTenantIDs() + + // Create another server. + tc2, _, cleanupEmptyCluster := backupRestoreTestSetupEmpty(t, singleNode, dir, InitManualReplication, base.TestClusterArgs{}) + srv2 := tc2.Server(0) + defer cleanupEmptyCluster() + + tenant10C2, cleanupT10C2 := makeTenant(srv2, 10) + defer cleanupT10C2() + + systemDB.Exec(t, `ALTER TENANT ALL SET CLUSTER SETTING sql.notices.enabled = false`) + backup2HttpAddr, backup2Cleanup := makeInsecureHTTPServer(t) + defer backup2Cleanup() + systemDB.Exec(t, `BACKUP TO $1`, backup2HttpAddr) + + t.Run("cluster-restore-into-tenant-with-tenant-settings-succeeds", func(t *testing.T) { + tenant10C2.Exec(t, `RESTORE FROM $1`, backup2HttpAddr) + }) + + _, systemDB2, cleanupDB2 := backupRestoreTestSetupEmpty(t, singleNode, dir, InitManualReplication, base.TestClusterArgs{}) + defer cleanupDB2() + t.Run("cluster-restore-into-cluster-with-tenant-settings-succeeds", func(t *testing.T) { + systemDB2.Exec(t, `RESTORE FROM $1`, backup2HttpAddr) + systemDB2.CheckQueryResults(t, `SELECT * FROM system.tenant_settings`, systemDB.QueryStr(t, `SELECT * FROM system.tenant_settings`)) + }) +} + // TestBackupRestoreInsideMultiPodTenant verifies that backup and restore work // inside tenants with multiple SQL pods. Currently, verification that restore // and backup are distributed to all pods in the multi-pod tests must be done @@ -6797,12 +6840,11 @@ func TestBackupRestoreInsideMultiPodTenant(t *testing.T) { tenant10C2[0].CheckQueryResults(t, `SELECT * FROM foo.bar`, tenant10[0].QueryStr(t, `SELECT * FROM foo.bar`)) }) t.Run("into-different-tenant-id", func(t *testing.T) { - tenant11C2.ExpectErr(t, `cannot cluster RESTORE backups taken from different tenant: 10`, - `RESTORE FROM $1`, httpAddr) + tenant11C2.Exec(t, `RESTORE FROM $1`, httpAddr) + tenant11C2.CheckQueryResults(t, `SELECT * FROM foo.bar`, tenant10[0].QueryStr(t, `SELECT * FROM foo.bar`)) }) t.Run("into-system-tenant-id", func(t *testing.T) { - systemDB2.ExpectErr(t, `cannot cluster RESTORE backups taken from different tenant: 10`, - `RESTORE FROM $1`, httpAddr) + systemDB2.Exec(t, `RESTORE FROM $1`, httpAddr) }) }) @@ -6854,8 +6896,7 @@ func TestBackupRestoreInsideMultiPodTenant(t *testing.T) { defer cleanupEmptyCluster() emptySystemDB.Exec(t, `BACKUP TO $1`, httpAddrEmpty) - tenant20C2.ExpectErr(t, `cannot cluster RESTORE backups taken from different tenant: system`, - `RESTORE FROM $1`, httpAddrEmpty) + tenant20C2.Exec(t, `RESTORE FROM $1`, httpAddrEmpty) }) }) diff --git a/pkg/ccl/backupccl/encryption.go b/pkg/ccl/backupccl/encryption.go new file mode 100644 index 000000000000..2d59945fdb4e --- /dev/null +++ b/pkg/ccl/backupccl/encryption.go @@ -0,0 +1,424 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package backupccl + +import ( + "bytes" + "context" + "crypto" + cryptorand "crypto/rand" + "fmt" + "net/url" + "strings" + + "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" + "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" + "github.com/cockroachdb/cockroach/pkg/security" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/ioctx" + "github.com/cockroachdb/cockroach/pkg/util/protoutil" + "github.com/cockroachdb/errors" +) + +func newEncryptedDataKeyMap() *encryptedDataKeyMap { + return &encryptedDataKeyMap{make(map[hashedMasterKeyID][]byte)} +} + +func newEncryptedDataKeyMapFromProtoMap(protoDataKeyMap map[string][]byte) *encryptedDataKeyMap { + encMap := &encryptedDataKeyMap{make(map[hashedMasterKeyID][]byte)} + for k, v := range protoDataKeyMap { + encMap.m[hashedMasterKeyID(k)] = v + } + + return encMap +} + +func (e *encryptedDataKeyMap) addEncryptedDataKey( + masterKeyID plaintextMasterKeyID, encryptedDataKey []byte, +) { + // Hash the master key ID before writing to the map. + hasher := crypto.SHA256.New() + hasher.Write([]byte(masterKeyID)) + hash := hasher.Sum(nil) + e.m[hashedMasterKeyID(hash)] = encryptedDataKey +} + +func (e *encryptedDataKeyMap) getEncryptedDataKey( + masterKeyID plaintextMasterKeyID, +) ([]byte, error) { + // Hash the master key ID before reading from the map. + hasher := crypto.SHA256.New() + hasher.Write([]byte(masterKeyID)) + hash := hasher.Sum(nil) + var encDataKey []byte + var ok bool + if encDataKey, ok = e.m[hashedMasterKeyID(hash)]; !ok { + return nil, errors.New("could not find an entry in the encryptedDataKeyMap") + } + + return encDataKey, nil +} + +func (e *encryptedDataKeyMap) rangeOverMap(fn func(masterKeyID hashedMasterKeyID, dataKey []byte)) { + for k, v := range e.m { + fn(k, v) + } +} + +// validateKMSURIsAgainstFullBackup ensures that the KMS URIs provided to an +// incremental BACKUP are a subset of those used during the full BACKUP. It does +// this by ensuring that the KMS master key ID of each KMS URI specified during +// the incremental BACKUP can be found in the map written to `encryption-info` +// during a base BACKUP. +// +// The method also returns the KMSInfo to be used for all subsequent +// encryption/decryption operations during this BACKUP. By default it is the +// first KMS URI passed during the incremental BACKUP. +func validateKMSURIsAgainstFullBackup( + kmsURIs []string, kmsMasterKeyIDToDataKey *encryptedDataKeyMap, kmsEnv cloud.KMSEnv, +) (*jobspb.BackupEncryptionOptions_KMSInfo, error) { + var defaultKMSInfo *jobspb.BackupEncryptionOptions_KMSInfo + for _, kmsURI := range kmsURIs { + kms, err := cloud.KMSFromURI(kmsURI, kmsEnv) + if err != nil { + return nil, err + } + + defer func() { + _ = kms.Close() + }() + + // Depending on the KMS specific implementation, this may or may not contact + // the remote KMS. + id, err := kms.MasterKeyID() + if err != nil { + return nil, err + } + + encryptedDataKey, err := kmsMasterKeyIDToDataKey.getEncryptedDataKey(plaintextMasterKeyID(id)) + if err != nil { + return nil, + errors.Wrap(err, + "one of the provided URIs was not used when encrypting the base BACKUP") + } + + if defaultKMSInfo == nil { + defaultKMSInfo = &jobspb.BackupEncryptionOptions_KMSInfo{ + Uri: kmsURI, + EncryptedDataKey: encryptedDataKey, + } + } + } + + return defaultKMSInfo, nil +} + +func makeNewEncryptionOptions( + ctx context.Context, encryptionParams jobspb.BackupEncryptionOptions, kmsEnv cloud.KMSEnv, +) (*jobspb.BackupEncryptionOptions, *jobspb.EncryptionInfo, error) { + var encryptionOptions *jobspb.BackupEncryptionOptions + var encryptionInfo *jobspb.EncryptionInfo + switch encryptionParams.Mode { + case jobspb.EncryptionMode_Passphrase: + salt, err := storageccl.GenerateSalt() + if err != nil { + return nil, nil, err + } + + encryptionInfo = &jobspb.EncryptionInfo{Salt: salt} + encryptionOptions = &jobspb.BackupEncryptionOptions{ + Mode: jobspb.EncryptionMode_Passphrase, + Key: storageccl.GenerateKey([]byte(encryptionParams.RawPassphrae), salt), + } + case jobspb.EncryptionMode_KMS: + // Generate a 32 byte/256-bit crypto-random number which will serve as + // the data key for encrypting the BACKUP data and manifest files. + plaintextDataKey := make([]byte, 32) + _, err := cryptorand.Read(plaintextDataKey) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to generate DataKey") + } + + encryptedDataKeyByKMSMasterKeyID, defaultKMSInfo, err := + getEncryptedDataKeyByKMSMasterKeyID(ctx, encryptionParams.RawKmsUris, plaintextDataKey, kmsEnv) + if err != nil { + return nil, nil, err + } + + encryptedDataKeyMapForProto := make(map[string][]byte) + encryptedDataKeyByKMSMasterKeyID.rangeOverMap( + func(masterKeyID hashedMasterKeyID, dataKey []byte) { + encryptedDataKeyMapForProto[string(masterKeyID)] = dataKey + }) + + encryptionInfo = &jobspb.EncryptionInfo{EncryptedDataKeyByKMSMasterKeyID: encryptedDataKeyMapForProto} + encryptionOptions = &jobspb.BackupEncryptionOptions{ + Mode: jobspb.EncryptionMode_KMS, + KMSInfo: defaultKMSInfo, + } + } + return encryptionOptions, encryptionInfo, nil +} + +func getEncryptedDataKeyFromURI( + ctx context.Context, plaintextDataKey []byte, kmsURI string, kmsEnv cloud.KMSEnv, +) (string, []byte, error) { + kms, err := cloud.KMSFromURI(kmsURI, kmsEnv) + if err != nil { + return "", nil, err + } + + defer func() { + _ = kms.Close() + }() + + kmsURL, err := url.ParseRequestURI(kmsURI) + if err != nil { + return "", nil, errors.Wrap(err, "cannot parse KMSURI") + } + encryptedDataKey, err := kms.Encrypt(ctx, plaintextDataKey) + if err != nil { + return "", nil, errors.Wrapf(err, "failed to encrypt data key for KMS scheme %s", + kmsURL.Scheme) + } + + masterKeyID, err := kms.MasterKeyID() + if err != nil { + return "", nil, errors.Wrapf(err, "failed to get master key ID for KMS scheme %s", + kmsURL.Scheme) + } + + return masterKeyID, encryptedDataKey, nil +} + +// getEncryptedDataKeyByKMSMasterKeyID constructs a mapping {MasterKeyID : +// EncryptedDataKey} for each KMS URI provided during a full BACKUP. The +// MasterKeyID is hashed before writing it to the map. +// +// The method also returns the KMSInfo to be used for all subsequent +// encryption/decryption operations during this BACKUP. By default it is the +// first KMS URI. +func getEncryptedDataKeyByKMSMasterKeyID( + ctx context.Context, kmsURIs []string, plaintextDataKey []byte, kmsEnv cloud.KMSEnv, +) (*encryptedDataKeyMap, *jobspb.BackupEncryptionOptions_KMSInfo, error) { + encryptedDataKeyByKMSMasterKeyID := newEncryptedDataKeyMap() + // The coordinator node contacts every KMS and records the encrypted data + // key for each one. + var kmsInfo *jobspb.BackupEncryptionOptions_KMSInfo + for _, kmsURI := range kmsURIs { + masterKeyID, encryptedDataKey, err := getEncryptedDataKeyFromURI(ctx, + plaintextDataKey, kmsURI, kmsEnv) + if err != nil { + return nil, nil, err + } + + // By default we use the first KMS URI and encrypted data key for subsequent + // encryption/decryption operation during a BACKUP. + if kmsInfo == nil { + kmsInfo = &jobspb.BackupEncryptionOptions_KMSInfo{ + Uri: kmsURI, + EncryptedDataKey: encryptedDataKey, + } + } + + encryptedDataKeyByKMSMasterKeyID.addEncryptedDataKey(plaintextMasterKeyID(masterKeyID), + encryptedDataKey) + } + + return encryptedDataKeyByKMSMasterKeyID, kmsInfo, nil +} + +func writeNewEncryptionInfoToBackup( + ctx context.Context, opts *jobspb.EncryptionInfo, dest cloud.ExternalStorage, numFiles int, +) error { + // New encryption-info file name is in the format "ENCRYPTION-INFO-" + newEncryptionInfoFile := fmt.Sprintf("%s-%d", backupEncryptionInfoFile, numFiles+1) + + buf, err := protoutil.Marshal(opts) + if err != nil { + return err + } + return cloud.WriteFile(ctx, dest, newEncryptionInfoFile, bytes.NewReader(buf)) +} + +// getEncryptionFromBase retrieves the encryption options of a base backup. It +// is expected that incremental backups use the same encryption options as the +// base backups. +func getEncryptionFromBase( + ctx context.Context, + user security.SQLUsername, + makeCloudStorage cloud.ExternalStorageFromURIFactory, + baseBackupURI string, + encryptionParams jobspb.BackupEncryptionOptions, + kmsEnv cloud.KMSEnv, +) (*jobspb.BackupEncryptionOptions, error) { + var encryptionOptions *jobspb.BackupEncryptionOptions + if encryptionParams.Mode != jobspb.EncryptionMode_None { + exportStore, err := makeCloudStorage(ctx, baseBackupURI, user) + if err != nil { + return nil, err + } + defer exportStore.Close() + opts, err := readEncryptionOptions(ctx, exportStore) + if err != nil { + return nil, err + } + + switch encryptionParams.Mode { + case jobspb.EncryptionMode_Passphrase: + encryptionOptions = &jobspb.BackupEncryptionOptions{ + Mode: jobspb.EncryptionMode_Passphrase, + Key: storageccl.GenerateKey([]byte(encryptionParams.RawPassphrae), opts[0].Salt), + } + case jobspb.EncryptionMode_KMS: + var defaultKMSInfo *jobspb.BackupEncryptionOptions_KMSInfo + for _, encFile := range opts { + defaultKMSInfo, err = validateKMSURIsAgainstFullBackup(encryptionParams.RawKmsUris, + newEncryptedDataKeyMapFromProtoMap(encFile.EncryptedDataKeyByKMSMasterKeyID), kmsEnv) + if err == nil { + break + } + } + if err != nil { + return nil, err + } + encryptionOptions = &jobspb.BackupEncryptionOptions{ + Mode: jobspb.EncryptionMode_KMS, + KMSInfo: defaultKMSInfo} + } + } + return encryptionOptions, nil +} + +func getEncryptionKey( + ctx context.Context, + encryption *jobspb.BackupEncryptionOptions, + settings *cluster.Settings, + ioConf base.ExternalIODirConfig, +) ([]byte, error) { + if encryption == nil { + return nil, errors.New("FileEncryptionOptions is nil when retrieving encryption key") + } + switch encryption.Mode { + case jobspb.EncryptionMode_Passphrase: + return encryption.Key, nil + case jobspb.EncryptionMode_KMS: + // Contact the selected KMS to derive the decrypted data key. + // TODO(pbardea): Add a check here if encryption.KMSInfo is unexpectedly nil + // here to avoid a panic, and return an error instead. + kms, err := cloud.KMSFromURI(encryption.KMSInfo.Uri, &backupKMSEnv{ + settings: settings, + conf: &ioConf, + }) + if err != nil { + return nil, err + } + + defer func() { + _ = kms.Close() + }() + + plaintextDataKey, err := kms.Decrypt(ctx, encryption.KMSInfo.EncryptedDataKey) + if err != nil { + return nil, errors.Wrap(err, "failed to decrypt data key") + } + + return plaintextDataKey, nil + } + + return nil, errors.New("invalid encryption mode") +} + +// readEncryptionOptions takes in a backup location and tries to find +// and return all encryption option files in the backup. A backup +// normally only creates one encryption option file, but if the user +// uses ALTER BACKUP to add new keys, a new encryption option file +// will be placed side by side with the old one. Since the old file +// is still valid, as we never want to modify or delete an existing +// backup, we return both new and old files. +func readEncryptionOptions( + ctx context.Context, src cloud.ExternalStorage, +) ([]jobspb.EncryptionInfo, error) { + const encryptionReadErrorMsg = `could not find or read encryption information` + + files, err := getEncryptionInfoFiles(ctx, src) + if err != nil { + return nil, errors.Mark(errors.Wrap(err, encryptionReadErrorMsg), errEncryptionInfoRead) + } + var encInfo []jobspb.EncryptionInfo + // The user is more likely to pass in a KMS URI that was used to + // encrypt the backup recently, so we iterate the ENCRYPTION-INFO + // files from latest to oldest. + for i := len(files) - 1; i >= 0; i-- { + r, err := src.ReadFile(ctx, files[i]) + if err != nil { + return nil, errors.Wrap(err, encryptionReadErrorMsg) + } + defer r.Close(ctx) + + encInfoBytes, err := ioctx.ReadAll(ctx, r) + if err != nil { + return nil, errors.Wrap(err, encryptionReadErrorMsg) + } + var currentEncInfo jobspb.EncryptionInfo + if err := protoutil.Unmarshal(encInfoBytes, ¤tEncInfo); err != nil { + return nil, err + } + encInfo = append(encInfo, currentEncInfo) + } + return encInfo, nil +} + +func getEncryptionInfoFiles(ctx context.Context, dest cloud.ExternalStorage) ([]string, error) { + var files []string + // Look for all files in dest that start with "/ENCRYPTION-INFO" + // and return them. + err := dest.List(ctx, "", "", func(p string) error { + paths := strings.Split(p, "/") + p = paths[len(paths)-1] + if match := strings.HasPrefix(p, backupEncryptionInfoFile); match { + files = append(files, p) + } + + return nil + }) + if len(files) < 1 { + return nil, errors.New("no ENCRYPTION-INFO files found") + } + + return files, err +} + +func writeEncryptionInfoIfNotExists( + ctx context.Context, opts *jobspb.EncryptionInfo, dest cloud.ExternalStorage, +) error { + r, err := dest.ReadFile(ctx, backupEncryptionInfoFile) + if err == nil { + r.Close(ctx) + // If the file already exists, then we don't need to create a new one. + return nil + } + + if !errors.Is(err, cloud.ErrFileDoesNotExist) { + return errors.Wrapf(err, + "returned an unexpected error when checking for the existence of %s file", + backupEncryptionInfoFile) + } + + buf, err := protoutil.Marshal(opts) + if err != nil { + return err + } + if err := cloud.WriteFile(ctx, dest, backupEncryptionInfoFile, bytes.NewReader(buf)); err != nil { + return err + } + return nil +} diff --git a/pkg/ccl/backupccl/manifest_handling.go b/pkg/ccl/backupccl/manifest_handling.go index f8325c7691b5..64ead25eb95f 100644 --- a/pkg/ccl/backupccl/manifest_handling.go +++ b/pkg/ccl/backupccl/manifest_handling.go @@ -20,7 +20,6 @@ import ( "sort" "strings" - "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -533,45 +532,6 @@ func getChecksum(data []byte) ([]byte, error) { return hash.Sum(nil)[:checksumSizeBytes], nil } -func getEncryptionKey( - ctx context.Context, - encryption *jobspb.BackupEncryptionOptions, - settings *cluster.Settings, - ioConf base.ExternalIODirConfig, -) ([]byte, error) { - if encryption == nil { - return nil, errors.New("FileEncryptionOptions is nil when retrieving encryption key") - } - switch encryption.Mode { - case jobspb.EncryptionMode_Passphrase: - return encryption.Key, nil - case jobspb.EncryptionMode_KMS: - // Contact the selected KMS to derive the decrypted data key. - // TODO(pbardea): Add a check here if encryption.KMSInfo is unexpectedly nil - // here to avoid a panic, and return an error instead. - kms, err := cloud.KMSFromURI(encryption.KMSInfo.Uri, &backupKMSEnv{ - settings: settings, - conf: &ioConf, - }) - if err != nil { - return nil, err - } - - defer func() { - _ = kms.Close() - }() - - plaintextDataKey, err := kms.Decrypt(ctx, encryption.KMSInfo.EncryptedDataKey) - if err != nil { - return nil, errors.Wrap(err, "failed to decrypt data key") - } - - return plaintextDataKey, nil - } - - return nil, errors.New("invalid encryption mode") -} - // writeBackupPartitionDescriptor writes metadata (containing a locality KV and // partial file listing) for a partitioned BACKUP to one of the stores in the // backup. @@ -1102,92 +1062,6 @@ func sanitizeLocalityKV(kv string) string { return string(sanitizedKV) } -// readEncryptionOptions takes in a backup location and tries to find -// and return all encryption option files in the backup. A backup -// normally only creates one encryption option file, but if the user -// uses ALTER BACKUP to add new keys, a new encryption option file -// will be placed side by side with the old one. Since the old file -// is still valid, as we never want to modify or delete an existing -// backup, we return both new and old files. -func readEncryptionOptions( - ctx context.Context, src cloud.ExternalStorage, -) ([]jobspb.EncryptionInfo, error) { - const encryptionReadErrorMsg = `could not find or read encryption information` - - files, err := getEncryptionInfoFiles(ctx, src) - if err != nil { - return nil, errors.Mark(errors.Wrap(err, encryptionReadErrorMsg), errEncryptionInfoRead) - } - var encInfo []jobspb.EncryptionInfo - // The user is more likely to pass in a KMS URI that was used to - // encrypt the backup recently, so we iterate the ENCRYPTION-INFO - // files from latest to oldest. - for i := len(files) - 1; i >= 0; i-- { - r, err := src.ReadFile(ctx, files[i]) - if err != nil { - return nil, errors.Wrap(err, encryptionReadErrorMsg) - } - defer r.Close(ctx) - - encInfoBytes, err := ioctx.ReadAll(ctx, r) - if err != nil { - return nil, errors.Wrap(err, encryptionReadErrorMsg) - } - var currentEncInfo jobspb.EncryptionInfo - if err := protoutil.Unmarshal(encInfoBytes, ¤tEncInfo); err != nil { - return nil, err - } - encInfo = append(encInfo, currentEncInfo) - } - return encInfo, nil -} - -func getEncryptionInfoFiles(ctx context.Context, dest cloud.ExternalStorage) ([]string, error) { - var files []string - // Look for all files in dest that start with "/ENCRYPTION-INFO" - // and return them. - err := dest.List(ctx, "", "", func(p string) error { - paths := strings.Split(p, "/") - p = paths[len(paths)-1] - if match := strings.HasPrefix(p, backupEncryptionInfoFile); match { - files = append(files, p) - } - - return nil - }) - if len(files) < 1 { - return nil, errors.New("no ENCRYPTION-INFO files found") - } - - return files, err -} - -func writeEncryptionInfoIfNotExists( - ctx context.Context, opts *jobspb.EncryptionInfo, dest cloud.ExternalStorage, -) error { - r, err := dest.ReadFile(ctx, backupEncryptionInfoFile) - if err == nil { - r.Close(ctx) - // If the file already exists, then we don't need to create a new one. - return nil - } - - if !errors.Is(err, cloud.ErrFileDoesNotExist) { - return errors.Wrapf(err, - "returned an unexpected error when checking for the existence of %s file", - backupEncryptionInfoFile) - } - - buf, err := protoutil.Marshal(opts) - if err != nil { - return err - } - if err := cloud.WriteFile(ctx, dest, backupEncryptionInfoFile, bytes.NewReader(buf)); err != nil { - return err - } - return nil -} - // RedactURIForErrorMessage redacts any storage secrets before returning a URI which is safe to // return to the client in an error message. func RedactURIForErrorMessage(uri string) string { diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index ba24eb7271a8..039f55b342c4 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -50,7 +50,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" - "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/interval" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -1279,13 +1278,6 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro } if !backupCodec.TenantPrefix().Equal(p.ExecCfg().Codec.TenantPrefix()) { - // Disallow cluster restores until values like jobs are relocatable. - if details.DescriptorCoverage == tree.AllDescriptors { - return unimplemented.NewWithIssuef(62277, - "cannot cluster RESTORE backups taken from different tenant: %s", - backupTenantID.String()) - } - // Ensure old processors fail if this is a previously unsupported restore of // a tenant backup by the system tenant, which the old rekey processor would // mishandle since it assumed the system tenant always restored tenant keys diff --git a/pkg/ccl/backupccl/system_schema.go b/pkg/ccl/backupccl/system_schema.go index d013bb4bc14d..530154b9b711 100644 --- a/pkg/ccl/backupccl/system_schema.go +++ b/pkg/ccl/backupccl/system_schema.go @@ -113,6 +113,43 @@ func defaultSystemTableRestoreFunc( // Custom restore functions for different system tables. +// tenantSettingsTableRestoreFunc restores the system.tenant_settings table. It +// returns an error when trying to restore a non-empty tenant_settings table +// into a non-system tenant. +func tenantSettingsTableRestoreFunc( + ctx context.Context, + execCfg *sql.ExecutorConfig, + txn *kv.Txn, + systemTableName, tempTableName string, +) error { + if execCfg.Codec.ForSystemTenant() { + return defaultSystemTableRestoreFunc(ctx, execCfg, txn, systemTableName, tempTableName) + } + + if count, err := queryTableRowCount(ctx, execCfg.InternalExecutor, txn, tempTableName); err == nil && count > 0 { + log.Warningf(ctx, "skipping restore of %d entries in system.tenant_settings table", count) + } else if err != nil { + log.Warningf(ctx, "skipping restore of entries in system.tenant_settings table (count failed: %s)", err.Error()) + } + return nil +} + +func queryTableRowCount( + ctx context.Context, ie *sql.InternalExecutor, txn *kv.Txn, tableName string, +) (int64, error) { + countQuery := fmt.Sprintf("SELECT count(1) FROM %s", tableName) + row, err := ie.QueryRow(ctx, fmt.Sprintf("count-%s", tableName), txn, countQuery) + if err != nil { + return 0, errors.Wrapf(err, "counting rows in %q", tableName) + } + + count, ok := row[0].(*tree.DInt) + if !ok { + return 0, errors.AssertionFailedf("failed to read count as DInt (was %T)", row[0]) + } + return int64(*count), nil +} + // jobsMigrationFunc resets the progress on schema change jobs, and marks all // other jobs as reverting. func jobsMigrationFunc( @@ -360,6 +397,7 @@ var systemTableBackupConfiguration = map[string]systemBackupConfiguration{ }, systemschema.TenantSettingsTable.GetName(): { shouldIncludeInClusterBackup: optInToClusterBackup, + customRestoreFunc: tenantSettingsTableRestoreFunc, }, } diff --git a/pkg/ccl/sqlproxyccl/balancer/BUILD.bazel b/pkg/ccl/sqlproxyccl/balancer/BUILD.bazel index 3739e2a8e8b8..c913339ec156 100644 --- a/pkg/ccl/sqlproxyccl/balancer/BUILD.bazel +++ b/pkg/ccl/sqlproxyccl/balancer/BUILD.bazel @@ -15,9 +15,11 @@ go_library( "//pkg/roachpb", "//pkg/util/log", "//pkg/util/randutil", + "//pkg/util/stop", "//pkg/util/syncutil", "@com_github_cockroachdb_errors//:errors", "@com_github_cockroachdb_logtags//:logtags", + "@com_github_marusama_semaphore//:semaphore", ], ) @@ -33,6 +35,10 @@ go_test( "//pkg/ccl/sqlproxyccl/tenant", "//pkg/roachpb", "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/stop", + "//pkg/util/timeutil", + "@com_github_cockroachdb_errors//:errors", "@com_github_stretchr_testify//require", ], ) diff --git a/pkg/ccl/sqlproxyccl/balancer/balancer.go b/pkg/ccl/sqlproxyccl/balancer/balancer.go index d0b166f797ae..ad9a15eb5845 100644 --- a/pkg/ccl/sqlproxyccl/balancer/balancer.go +++ b/pkg/ccl/sqlproxyccl/balancer/balancer.go @@ -9,18 +9,56 @@ package balancer import ( + "container/list" + "context" + "math" "math/rand" + "time" "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl/tenant" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/randutil" + "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" + "github.com/marusama/semaphore" ) // ErrNoAvailablePods is an error that indicates that no pods are available // for selection. var ErrNoAvailablePods = errors.New("no available pods") +// defaultMaxConcurrentRebalances represents the maximum number of concurrent +// rebalance requests that are being processed. This effectively limits the +// number of concurrent transfers per proxy. +const defaultMaxConcurrentRebalances = 100 + +// maxTransferAttempts represents the maximum number of transfer attempts per +// rebalance requests when the previous attempts failed (possibly due to an +// unsafe transfer point). Note that each transfer attempt currently has a +// timeout of 15 seconds, so retrying up to 3 times may hold onto processSem +// up to 45 seconds for each rebalance request. +// +// TODO(jaylim-crl): Reduce transfer timeout to 5 seconds. +const maxTransferAttempts = 3 + +// balancerOptions controls the behavior of the balancer component. +type balancerOptions struct { + maxConcurrentRebalances int +} + +// Option defines an option that can be passed to NewBalancer in order to +// control its behavior. +type Option func(opts *balancerOptions) + +// MaxConcurrentRebalances defines the maximum number of concurrent rebalance +// operations for the balancer. This defaults to defaultMaxConcurrentRebalances. +func MaxConcurrentRebalances(max int) Option { + return func(opts *balancerOptions) { + opts.maxConcurrentRebalances = max + } +} + // Balancer handles load balancing of SQL connections within the proxy. // All methods on the Balancer instance are thread-safe. type Balancer struct { @@ -32,16 +70,54 @@ type Balancer struct { // be used for load balancing. rng *rand.Rand } + + // stopper is used to start async tasks (e.g. transfer requests) within the + // balancer. + stopper *stop.Stopper + + // queue represents the rebalancer queue. All transfer requests should be + // enqueued to this queue instead of calling the transfer API directly. + queue *rebalancerQueue + + // processSem is used to limit the number of concurrent rebalance requests + // that are being processed. + processSem semaphore.Semaphore } // NewBalancer constructs a new Balancer instance that is responsible for // load balancing SQL connections within the proxy. // // TODO(jaylim-crl): Update Balancer to take in a ConnTracker object. -func NewBalancer() *Balancer { - b := &Balancer{} +func NewBalancer(ctx context.Context, stopper *stop.Stopper, opts ...Option) (*Balancer, error) { + // Handle options. + options := &balancerOptions{} + for _, opt := range opts { + opt(options) + } + if options.maxConcurrentRebalances == 0 { + options.maxConcurrentRebalances = defaultMaxConcurrentRebalances + } + + // Ensure that ctx gets cancelled on stopper's quiescing. + ctx, _ = stopper.WithCancelOnQuiesce(ctx) + + q, err := newRebalancerQueue(ctx) + if err != nil { + return nil, err + } + + b := &Balancer{ + stopper: stopper, + queue: q, + processSem: semaphore.New(options.maxConcurrentRebalances), + } b.mu.rng, _ = randutil.NewPseudoRand() - return b + + if err := b.stopper.RunAsyncTask(ctx, "processQueue", b.processQueue); err != nil { + return nil, err + } + + return b, nil } // SelectTenantPod selects a tenant pod from the given list based on a weighted @@ -62,3 +138,141 @@ func (b *Balancer) randFloat32() float32 { defer b.mu.Unlock() return b.mu.rng.Float32() } + +// processQueue runs on a background goroutine, and invokes TransferConnection +// for each rebalance request. +func (b *Balancer) processQueue(ctx context.Context) { + // processOneReq processors a request from the balancer queue. If the queue + // is empty, this blocks. This returns true if processing should continue, + // or false otherwise. + processOneReq := func() (canContinue bool) { + if err := b.processSem.Acquire(ctx, 1); err != nil { + log.Errorf(ctx, "could not acquire processSem: %v", err.Error()) + return false + } + + req, err := b.queue.dequeue(ctx) + if err != nil { + // Context is cancelled. + log.Errorf(ctx, "could not dequeue from rebalancer queue: %v", err.Error()) + return false + } + + // TODO(jaylim-crl): implement enhancements: + // 1. Add metrics to track the number of active transfers. + // 2. Rate limit the number of transfers per connection (e.g. once + // every 5 minutes). This ensures that the connection isn't + // ping-ponged between pods within a short interval. However, for + // draining ones, we may want to move right away (or after 60 secs), + // even if the connection was recently transferred to the draining + // pod. + if err := b.stopper.RunAsyncTask(ctx, "processQueue-item", func(ctx context.Context) { + defer b.processSem.Release(1) + + // Each request is retried up to maxTransferAttempts. + for i := 0; i < maxTransferAttempts && ctx.Err() == nil; i++ { + // TODO(jaylim-crl): Once the TransferConnection API accepts a + // destination, we could update this code, and pass along dst. + err := req.conn.TransferConnection( /* req.dst */ ) + if err == nil || errors.Is(err, context.Canceled) || + req.dst == req.conn.ServerRemoteAddr() { + break + } + + // Retry again if the connection hasn't been closed or + // transferred to the destination. + time.Sleep(250 * time.Millisecond) + } + }); err != nil { + // We should not hit this case, but if we did, log and abandon the + // transfer. + log.Errorf(ctx, "could not run async task for processQueue-item: %v", err.Error()) + } + return true + } + for ctx.Err() == nil && processOneReq() { + } +} + +// rebalanceRequest corresponds to a rebalance request. For now, this only +// indicates where the connection should be transferred to through dst. +type rebalanceRequest struct { + createdAt time.Time + conn ConnectionHandle + dst string +} + +// balancerQueue represents the balancer's internal queue which is used for +// rebalancing requests. All methods on the queue are thread-safe. +type rebalancerQueue struct { + mu syncutil.Mutex + sem semaphore.Semaphore + queue *list.List + elements map[ConnectionHandle]*list.Element +} + +// newRebalancerQueue returns a new instance of rebalancerQueue. +func newRebalancerQueue(ctx context.Context) (*rebalancerQueue, error) { + q := &rebalancerQueue{ + sem: semaphore.New(math.MaxInt32), + queue: list.New(), + elements: make(map[ConnectionHandle]*list.Element), + } + // sem represents the number of items in the queue, so we'll acquire + // everything to denote an empty queue. + if err := q.sem.Acquire(ctx, math.MaxInt32); err != nil { + return nil, err + } + return q, nil +} + +// enqueue puts the rebalance request into the queue. If a request for the +// connection already exists, the newer of the two will be used. This returns +// nil if the operation succeeded. +// +// NOTE: req should not be nil. +func (q *rebalancerQueue) enqueue(req *rebalanceRequest) { + q.mu.Lock() + defer q.mu.Unlock() + + e, ok := q.elements[req.conn] + if ok { + // Use the newer request of the two. + if e.Value.(*rebalanceRequest).createdAt.Before(req.createdAt) { + e.Value = req + } + } else { + e = q.queue.PushBack(req) + q.elements[req.conn] = e + } + q.sem.Release(1) +} + +// dequeue removes a request at the front of the queue, and returns that. If the +// queue has no items, dequeue will block until the queue is non-empty. +// +// NOTE: It is unsafe to continue using the queue if dequeue returns an error. +func (q *rebalancerQueue) dequeue(ctx context.Context) (*rebalanceRequest, error) { + // Block until there is an item in the queue. There is a possibility where + // Acquire returns an error AND obtains the semaphore. It is unsafe to + // continue using the queue when that happens. + // + // It is deliberate to block on acquiring the semaphore before obtaining + // the mu lock. We need that lock to enqueue items. + if err := q.sem.Acquire(ctx, 1); err != nil { + return nil, err + } + + q.mu.Lock() + defer q.mu.Unlock() + + e := q.queue.Front() + if e == nil { + // The queue cannot be empty here. + return nil, errors.AssertionFailedf("unexpected empty queue") + } + + req := q.queue.Remove(e).(*rebalanceRequest) + delete(q.elements, req.conn) + return req, nil +} diff --git a/pkg/ccl/sqlproxyccl/balancer/balancer_test.go b/pkg/ccl/sqlproxyccl/balancer/balancer_test.go index 9c687a6be868..5fe503e9c157 100644 --- a/pkg/ccl/sqlproxyccl/balancer/balancer_test.go +++ b/pkg/ccl/sqlproxyccl/balancer/balancer_test.go @@ -6,25 +6,37 @@ // // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package balancer_test +package balancer import ( + "context" + "fmt" + "sync" + "sync/atomic" "testing" + "time" - "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl/balancer" "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl/tenant" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/stop" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" + "github.com/cockroachdb/errors" "github.com/stretchr/testify/require" ) -func TestBalancer(t *testing.T) { +func TestBalancer_SelectTenantPod(t *testing.T) { defer leaktest.AfterTest(t)() - b := balancer.NewBalancer() + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + b, err := NewBalancer(ctx, stopper) + require.NoError(t, err) t.Run("no pods", func(t *testing.T) { pod, err := b.SelectTenantPod([]*tenant.Pod{}) - require.EqualError(t, err, balancer.ErrNoAvailablePods.Error()) + require.EqualError(t, err, ErrNoAvailablePods.Error()) require.Nil(t, pod) }) @@ -34,3 +46,302 @@ func TestBalancer(t *testing.T) { require.Contains(t, []string{"1", "2"}, pod.Addr) }) } + +func TestRebalancer_processQueue(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + b, err := NewBalancer(ctx, stopper, MaxConcurrentRebalances(1)) + require.NoError(t, err) + + // Use a custom time source for testing. + t0 := time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC) + timeSource := timeutil.NewManualTime(t0) + + // syncReq is used to wait until the test has completed processing the + // items that are of concern. + syncCh := make(chan struct{}) + syncReq := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: &testBalancerConnHandle{ + onTransferConnection: func() error { + syncCh <- struct{}{} + return nil + }, + }, + dst: "foo", + } + + t.Run("retries_up_to_maxTransferAttempts", func(t *testing.T) { + count := 0 + req := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: &testBalancerConnHandle{ + onTransferConnection: func() error { + count++ + return errors.New("cannot transfer") + }, + }, + dst: "foo", + } + b.queue.enqueue(req) + + // Wait until the item has been processed. + b.queue.enqueue(syncReq) + <-syncCh + + // Ensure that we only retried up to 3 times. + require.Equal(t, 3, count) + }) + + t.Run("conn_was_transferred_by_other", func(t *testing.T) { + count := 0 + conn := &testBalancerConnHandle{} + conn.onTransferConnection = func() error { + count++ + // Simulate that connection was transferred by someone else. + conn.remoteAddr = "foo" + return errors.New("cannot transfer") + } + req := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: conn, + dst: "foo", + } + b.queue.enqueue(req) + + // Wait until the item has been processed. + b.queue.enqueue(syncReq) + <-syncCh + + // We should only retry once. + require.Equal(t, 1, count) + }) + + t.Run("conn_was_transferred", func(t *testing.T) { + count := 0 + conn := &testBalancerConnHandle{} + conn.onTransferConnection = func() error { + count++ + conn.remoteAddr = "foo" + return nil + } + req := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: conn, + dst: "foo", + } + b.queue.enqueue(req) + + // Wait until the item has been processed. + b.queue.enqueue(syncReq) + <-syncCh + + // We should only retry once. + require.Equal(t, 1, count) + }) + + t.Run("conn_was_closed", func(t *testing.T) { + count := 0 + conn := &testBalancerConnHandle{} + conn.onTransferConnection = func() error { + count++ + return context.Canceled + } + req := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: conn, + dst: "foo", + } + b.queue.enqueue(req) + + // Wait until the item has been processed. + b.queue.enqueue(syncReq) + <-syncCh + + // We should only retry once. + require.Equal(t, 1, count) + }) + + t.Run("limit_concurrent_rebalances", func(t *testing.T) { + const reqCount = 100 + + // Allow up to 2 concurrent rebalances. + b.processSem.SetLimit(2) + + // wg is used to wait until all transfers have completed. + var wg sync.WaitGroup + wg.Add(reqCount) + + // waitCh is used to wait until all items have fully been enqueued. + waitCh := make(chan struct{}) + + var count int32 + for i := 0; i < reqCount; i++ { + req := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: &testBalancerConnHandle{ + onTransferConnection: func() error { + // Block until all requests are enqueued. + <-waitCh + + defer func() { + newCount := atomic.AddInt32(&count, -1) + require.True(t, newCount >= 0) + wg.Done() + }() + + // Count should not exceed the maximum number of + // concurrent rebalances defined. + newCount := atomic.AddInt32(&count, 1) + require.True(t, newCount <= 2) + return nil + }, + }, + dst: "foo", + } + b.queue.enqueue(req) + } + + // Close the channel to unblock. + close(waitCh) + + // Wait until all transfers have completed. + wg.Wait() + + // We should only transfer once for every connection. + require.Equal(t, int32(0), count) + }) +} + +func TestRebalancerQueue(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + q, err := newRebalancerQueue(ctx) + require.NoError(t, err) + + // Use a custom time source for testing. + t0 := time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC) + timeSource := timeutil.NewManualTime(t0) + + // Create rebalance requests for the same connection handle. + conn1 := &testBalancerConnHandle{} + req1 := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: conn1, + dst: "foo1", + } + timeSource.Advance(5 * time.Second) + req2 := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: conn1, + dst: "foo2", + } + timeSource.Advance(5 * time.Second) + req3 := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: conn1, + dst: "foo3", + } + + // Enqueue in a specific order. req3 overrides req1; req2 is a no-op. + q.enqueue(req1) + q.enqueue(req3) + q.enqueue(req2) + require.Len(t, q.elements, 1) + require.Equal(t, 1, q.queue.Len()) + + // Create another request. + conn2 := &testBalancerConnHandle{} + req4 := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: conn2, + dst: "bar1", + } + q.enqueue(req4) + require.Len(t, q.elements, 2) + require.Equal(t, 2, q.queue.Len()) + + // Dequeue the items. + item, err := q.dequeue(ctx) + require.NoError(t, err) + require.Equal(t, req3, item) + item, err = q.dequeue(ctx) + require.NoError(t, err) + require.Equal(t, req4, item) + require.Empty(t, q.elements) + require.Equal(t, 0, q.queue.Len()) + + // Cancel the context. Dequeue should return immediately with an error. + cancel() + req4, err = q.dequeue(ctx) + require.EqualError(t, err, context.Canceled.Error()) + require.Nil(t, req4) +} + +func TestRebalancerQueueBlocking(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + q, err := newRebalancerQueue(ctx) + require.NoError(t, err) + + reqCh := make(chan *rebalanceRequest, 10) + go func() { + for { + req, err := q.dequeue(ctx) + if err != nil { + break + } + reqCh <- req + } + }() + + // Use a custom time source for testing. + t0 := time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC) + timeSource := timeutil.NewManualTime(t0) + + const reqCount = 100 + for i := 0; i < reqCount; i++ { + req := &rebalanceRequest{ + createdAt: timeSource.Now(), + conn: &testBalancerConnHandle{}, + dst: fmt.Sprint(i), + } + q.enqueue(req) + timeSource.Advance(1 * time.Second) + } + + for i := 0; i < reqCount; i++ { + req := <-reqCh + require.Equal(t, fmt.Sprint(i), req.dst) + } +} + +// testBalancerConnHandle is a test connection handle that is used for testing +// the balancer. +type testBalancerConnHandle struct { + ConnectionHandle + remoteAddr string + onTransferConnection func() error +} + +var _ ConnectionHandle = &testBalancerConnHandle{} + +// TransferConnection implements the ConnectionHandle interface. +func (h *testBalancerConnHandle) TransferConnection() error { + return h.onTransferConnection() +} + +// ServerRemoteAddr implements the ConnectionHandle interface. +func (h *testBalancerConnHandle) ServerRemoteAddr() string { + return h.remoteAddr +} diff --git a/pkg/ccl/sqlproxyccl/balancer/conn_tracker_test.go b/pkg/ccl/sqlproxyccl/balancer/conn_tracker_test.go index 8f5072f163ec..f5fbfec8bec6 100644 --- a/pkg/ccl/sqlproxyccl/balancer/conn_tracker_test.go +++ b/pkg/ccl/sqlproxyccl/balancer/conn_tracker_test.go @@ -18,15 +18,17 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/stretchr/testify/require" ) func TestConnTracker(t *testing.T) { defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) tracker := NewConnTracker() - makeConn := func(tenantID int, podAddr string) (roachpb.TenantID, *testConnectionHandle) { - return roachpb.MakeTenantID(uint64(tenantID)), newTestConnectionHandle(podAddr) + makeConn := func(tenantID int, podAddr string) (roachpb.TenantID, *testTrackerConnHandle) { + return roachpb.MakeTenantID(uint64(tenantID)), newTestTrackerConnHandle(podAddr) } tenantID, handle := makeConn(20, "127.0.0.10:8090") @@ -69,7 +71,7 @@ func TestTenantEntry(t *testing.T) { entry := newTenantEntry() - h1 := newTestConnectionHandle("10.0.0.1:12345") + h1 := newTestTrackerConnHandle("10.0.0.1:12345") require.True(t, entry.addHandle(h1)) require.False(t, entry.addHandle(h1)) @@ -83,25 +85,25 @@ func TestTenantEntry(t *testing.T) { require.Len(t, conns, 1) } -// testConnectionHandle is a test connection handle that only implements a -// small subset of methods. -type testConnectionHandle struct { +// testTrackerConnHandle is a test connection handle that only implements a +// small subset of methods used for testing the connection tracker. +type testTrackerConnHandle struct { ConnectionHandle remoteAddr string } -var _ ConnectionHandle = &testConnectionHandle{} +var _ ConnectionHandle = &testTrackerConnHandle{} -func newTestConnectionHandle(remoteAddr string) *testConnectionHandle { - return &testConnectionHandle{remoteAddr: remoteAddr} +func newTestTrackerConnHandle(remoteAddr string) *testTrackerConnHandle { + return &testTrackerConnHandle{remoteAddr: remoteAddr} } // Context implements the ConnectionHandle interface. -func (h *testConnectionHandle) Context() context.Context { +func (h *testTrackerConnHandle) Context() context.Context { return context.Background() } // ServerRemoteAddr implements the ConnectionHandle interface. -func (h *testConnectionHandle) ServerRemoteAddr() string { +func (h *testTrackerConnHandle) ServerRemoteAddr() string { return h.remoteAddr } diff --git a/pkg/ccl/sqlproxyccl/connector.go b/pkg/ccl/sqlproxyccl/connector.go index b5dd5dfe075c..4971959473f8 100644 --- a/pkg/ccl/sqlproxyccl/connector.go +++ b/pkg/ccl/sqlproxyccl/connector.go @@ -275,14 +275,17 @@ func (c *connector) lookupAddr(ctx context.Context) (string, error) { pods, err := c.DirectoryCache.LookupTenantPods(ctx, c.TenantID, c.ClusterName) switch { case err == nil: - // Note that LookupTenantPods will always return RUNNING pods, so this - // is fine for now. If we start changing that to also return DRAINING - // pods, we'd have to filter accordingly. - pod, err := c.Balancer.SelectTenantPod(pods) + runningPods := make([]*tenant.Pod, 0, len(pods)) + for _, pod := range pods { + if pod.State == tenant.RUNNING { + runningPods = append(runningPods, pod) + } + } + pod, err := c.Balancer.SelectTenantPod(runningPods) if err != nil { // This should never happen because LookupTenantPods ensured that - // len(pods) should never be 0. Mark it as a retriable connection - // anyway. + // there should be at least one RUNNING pod. Mark it as a retriable + // connection anyway. return "", markAsRetriableConnectorError(err) } return pod.Addr, nil diff --git a/pkg/ccl/sqlproxyccl/connector_test.go b/pkg/ccl/sqlproxyccl/connector_test.go index f27ca95bee93..1637d27bf651 100644 --- a/pkg/ccl/sqlproxyccl/connector_test.go +++ b/pkg/ccl/sqlproxyccl/connector_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/stop" "github.com/cockroachdb/cockroach/pkg/util/syncutil" "github.com/cockroachdb/errors" "github.com/jackc/pgproto3/v2" @@ -480,13 +481,19 @@ func TestConnector_dialTenantCluster(t *testing.T) { func TestConnector_lookupAddr(t *testing.T) { defer leaktest.AfterTest(t)() ctx := context.Background() + stopper := stop.NewStopper() + defer stopper.Stop(ctx) + + balancer, err := balancer.NewBalancer(ctx, stopper) + require.NoError(t, err) t.Run("successful", func(t *testing.T) { var lookupTenantPodsFnCount int + c := &connector{ ClusterName: "my-foo", TenantID: roachpb.MakeTenantID(10), - Balancer: balancer.NewBalancer(), + Balancer: balancer, } c.DirectoryCache = &testTenantDirectoryCache{ lookupTenantPodsFn: func( @@ -496,7 +503,10 @@ func TestConnector_lookupAddr(t *testing.T) { require.Equal(t, ctx, fnCtx) require.Equal(t, c.TenantID, tenantID) require.Equal(t, c.ClusterName, clusterName) - return []*tenant.Pod{{Addr: "127.0.0.10:80"}}, nil + return []*tenant.Pod{ + {Addr: "127.0.0.10:70", State: tenant.DRAINING}, + {Addr: "127.0.0.10:80", State: tenant.RUNNING}, + }, nil }, } @@ -525,7 +535,7 @@ func TestConnector_lookupAddr(t *testing.T) { c := &connector{ ClusterName: "my-foo", TenantID: roachpb.MakeTenantID(10), - Balancer: balancer.NewBalancer(), + Balancer: balancer, } c.DirectoryCache = &testTenantDirectoryCache{ lookupTenantPodsFn: func( @@ -536,7 +546,11 @@ func TestConnector_lookupAddr(t *testing.T) { pods := make([]*tenant.Pod, 0, len(mu.pods)) for addr, load := range mu.pods { - pods = append(pods, &tenant.Pod{Addr: addr, Load: load}) + pods = append(pods, &tenant.Pod{ + Addr: addr, + Load: load, + State: tenant.RUNNING, + }) } return pods, nil }, @@ -588,7 +602,7 @@ func TestConnector_lookupAddr(t *testing.T) { c := &connector{ ClusterName: "my-foo", TenantID: roachpb.MakeTenantID(10), - Balancer: balancer.NewBalancer(), + Balancer: balancer, } c.DirectoryCache = &testTenantDirectoryCache{ lookupTenantPodsFn: func( @@ -613,7 +627,7 @@ func TestConnector_lookupAddr(t *testing.T) { c := &connector{ ClusterName: "my-foo", TenantID: roachpb.MakeTenantID(10), - Balancer: balancer.NewBalancer(), + Balancer: balancer, } c.DirectoryCache = &testTenantDirectoryCache{ lookupTenantPodsFn: func( @@ -638,7 +652,7 @@ func TestConnector_lookupAddr(t *testing.T) { c := &connector{ ClusterName: "my-foo", TenantID: roachpb.MakeTenantID(10), - Balancer: balancer.NewBalancer(), + Balancer: balancer, } c.DirectoryCache = &testTenantDirectoryCache{ lookupTenantPodsFn: func( diff --git a/pkg/ccl/sqlproxyccl/proxy_handler.go b/pkg/ccl/sqlproxyccl/proxy_handler.go index 730d3bb135ee..1a93e793dfde 100644 --- a/pkg/ccl/sqlproxyccl/proxy_handler.go +++ b/pkg/ccl/sqlproxyccl/proxy_handler.go @@ -165,21 +165,26 @@ var throttledError = errors.WithHint( func newProxyHandler( ctx context.Context, stopper *stop.Stopper, proxyMetrics *metrics, options ProxyOptions, ) (*proxyHandler, error) { + ctx, _ = stopper.WithCancelOnQuiesce(ctx) + handler := proxyHandler{ stopper: stopper, metrics: proxyMetrics, ProxyOptions: options, certManager: certmgr.NewCertManager(ctx), - balancer: balancer.NewBalancer(), - connTracker: balancer.NewConnTracker(), } - err := handler.setupIncomingCert() + err := handler.setupIncomingCert(ctx) if err != nil { return nil, err } - ctx, _ = stopper.WithCancelOnQuiesce(ctx) + // Create the balancer component. + handler.balancer, err = balancer.NewBalancer(ctx, stopper) + if err != nil { + return nil, err + } + handler.connTracker = balancer.NewConnTracker() // If denylist functionality is requested, create the denylist service. if options.Denylist != "" { @@ -469,7 +474,7 @@ func (handler *proxyHandler) incomingTLSConfig() *tls.Config { // using self-signed, runtime generated cert (if cert is set to *) or // using file based cert where the cert/key values refer to file names // containing the information. -func (handler *proxyHandler) setupIncomingCert() error { +func (handler *proxyHandler) setupIncomingCert(ctx context.Context) error { if (handler.ListenKey == "") != (handler.ListenCert == "") { return errors.New("must specify either both or neither of cert and key") } @@ -479,7 +484,6 @@ func (handler *proxyHandler) setupIncomingCert() error { } // TODO(darin): change the cert manager so it uses the stopper. - ctx, _ := handler.stopper.WithCancelOnQuiesce(context.Background()) certMgr := certmgr.NewCertManager(ctx) var cert certmgr.Cert if handler.ListenCert == "*" { diff --git a/pkg/ccl/sqlproxyccl/tenant/BUILD.bazel b/pkg/ccl/sqlproxyccl/tenant/BUILD.bazel index 0e2e01dcce5a..7b570b7d2d25 100644 --- a/pkg/ccl/sqlproxyccl/tenant/BUILD.bazel +++ b/pkg/ccl/sqlproxyccl/tenant/BUILD.bazel @@ -8,7 +8,10 @@ proto_library( srcs = ["directory.proto"], strip_import_prefix = "/pkg", visibility = ["//visibility:public"], - deps = ["@com_github_gogo_protobuf//gogoproto:gogo_proto"], + deps = [ + "@com_github_gogo_protobuf//gogoproto:gogo_proto", + "@com_google_protobuf//:timestamp_proto", + ], ) go_proto_library( diff --git a/pkg/ccl/sqlproxyccl/tenant/directory.proto b/pkg/ccl/sqlproxyccl/tenant/directory.proto index d22edd18ffe3..e56c44afd606 100644 --- a/pkg/ccl/sqlproxyccl/tenant/directory.proto +++ b/pkg/ccl/sqlproxyccl/tenant/directory.proto @@ -11,9 +11,7 @@ package cockroach.ccl.sqlproxyccl.tenant; option go_package="tenant"; import "gogoproto/gogo.proto"; - -// WatchPodsRequest is empty as we want to get all notifications. -message WatchPodsRequest {} +import "google/protobuf/timestamp.proto"; // PodState gives the current state of a tenant pod, so that the proxy knows // how/where to route traffic. @@ -21,13 +19,13 @@ message WatchPodsRequest {} enum PodState { option (gogoproto.goproto_enum_prefix) = false; - // UNKNOWN indicates that the pod values being reported are from a - // potentially out of date source. UNKNOWN may be used to notify updates to - // pod values when the pod's state may be out of date by the time the update - // is processed. + // UNKNOWN indicates that the pod values being reported are from a potentially + // out of date source. UNKNOWN may be used to notify updates to pod values + // when the pod's state may be out of date by the time the update is processed. UNKNOWN = 0; // RUNNING indicates the pod may have active SQL connections and is ready to // accept new SQL connections. + // // NOTE: The proxy must still be prepared to retry connections against a // running pod in case of transient failures. RUNNING = 1; @@ -46,17 +44,38 @@ enum PodState { // location, and state. message Pod { // TenantID is the tenant that owns the pod. - uint64 tenant_id = 2[(gogoproto.customname) = "TenantID"]; - // Addr is the ip and port combo identifying the tenant pod, (e.g. + uint64 tenant_id = 2 [(gogoproto.customname) = "TenantID"]; + // Addr is the ip and port combination identifying the tenant pod, (e.g. // 132.130.1.11:34576). - string Addr = 1; - // PodState gives the current status of the tenant pod. - PodState State = 3; + string addr = 1; + // State gives the current status of the tenant pod. + PodState state = 3; // Load is a number in the range [0, 1] indicating the current amount of load // experienced by this tenant pod. float Load = 4; + // StateTimestamp represents the timestamp that the state was last updated. + google.protobuf.Timestamp stateTimestamp = 5 [(gogoproto.nullable) = false, (gogoproto.stdtime) = true]; +} + +// ListPodsRequest is used to query the server for the list of current pods of +// a given tenant. +message ListPodsRequest { + // TenantID identifies the tenant for which the client is requesting a list of + // the pods. + uint64 tenant_id = 1 [(gogoproto.customname) = "TenantID"]; +} + +// ListPodsResponse is sent back as a result of requesting the list of pods for +// a given tenant. +message ListPodsResponse { + // Pods is the list of RUNNING and/or DRAINING pods for the requested tenant. + // It does not include DELETING pods. + repeated Pod pods = 1; } +// WatchPodsRequest is empty as we want to get all notifications. +message WatchPodsRequest {} + // WatchPodsResponse represents the notifications that the server sends to // its clients when clients want to monitor the directory server activity. message WatchPodsResponse { @@ -64,19 +83,11 @@ message WatchPodsResponse { Pod pod = 1; } -// ListPodsRequest is used to query the server for the list of current -// pods of a given tenant. -message ListPodsRequest { - // TenantID identifies the tenant for which the client is requesting a list of - // the pods. - uint64 tenant_id = 1[(gogoproto.customname) = "TenantID"]; -} - // EnsurePodRequest is used to ensure that at least one tenant pod is in the // RUNNING state. message EnsurePodRequest { // TenantID is the id of the tenant for which a RUNNING pod is requested. - uint64 tenant_id = 1[(gogoproto.customname) = "TenantID"]; + uint64 tenant_id = 1 [(gogoproto.customname) = "TenantID"]; } // EnsurePodResponse is empty and indicates that the server processed the @@ -84,19 +95,11 @@ message EnsurePodRequest { message EnsurePodResponse { } -// ListPodsResponse is sent back as a result of requesting the list of pods for -// a given tenant. -message ListPodsResponse { - // Pods is the list of RUNNING and/or DRAINING pods for the requested tenant. - // It does not include DELETING pods. - repeated Pod pods = 1; -} - // GetTenantRequest is used by a client to request from the sever metadata // related to a given tenant. message GetTenantRequest { // TenantID identifies the tenant for which the metadata is being requested. - uint64 tenant_id = 1[(gogoproto.customname) = "TenantID"]; + uint64 tenant_id = 1 [(gogoproto.customname) = "TenantID"]; } // GetTenantResponse is sent back when a client requests metadata for a tenant. diff --git a/pkg/ccl/sqlproxyccl/tenant/directory_cache.go b/pkg/ccl/sqlproxyccl/tenant/directory_cache.go index f01353a0c08e..738a292119a8 100644 --- a/pkg/ccl/sqlproxyccl/tenant/directory_cache.go +++ b/pkg/ccl/sqlproxyccl/tenant/directory_cache.go @@ -154,10 +154,11 @@ func NewDirectoryCache( return dir, nil } -// LookupTenantPods returns a list of SQL pods in the RUNNING state for the -// given tenant. If the tenant was just created or is suspended, such that there -// are no available processes, then LookupTenantPods will trigger resumption of a -// new instance and block until the process is ready. +// LookupTenantPods returns a list of SQL pods in the RUNNING and DRAINING +// states for the given tenant. If the tenant was just created or is suspended, +// such that there are no available RUNNING processes, then LookupTenantPods +// will trigger resumption of a new instance (or a conversion of a DRAINING pod +// to a RUNNING one) and block until that happens. // // If clusterName is non-empty, then a GRPC NotFound error is returned if no // pods match the cluster name. This can be used to ensure that the incoming SQL @@ -191,7 +192,16 @@ func (d *directoryCache) LookupTenantPods( ctx, _ = d.stopper.WithCancelOnQuiesce(ctx) tenantPods := entry.GetPods() - if len(tenantPods) == 0 { + + // Trigger resumption if there are no RUNNING pods. + hasRunningPod := false + for _, pod := range tenantPods { + if pod.State == RUNNING { + hasRunningPod = true + break + } + } + if !hasRunningPod { // There are no known pod IP addresses, so fetch pod information from // the directory server. Resume the tenant if it is suspended; that // will always result in at least one pod IP address (or an error). @@ -206,12 +216,12 @@ func (d *directoryCache) LookupTenantPods( return tenantPods, nil } -// TryLookupTenantPods returns a list of SQL pods in the RUNNING state for the -// given tenant. It returns a GRPC NotFound error if the tenant does not exist -// (e.g. it has not yet been created) or if it has not yet been fetched into the -// directory's cache (TryLookupTenantPods will never attempt to fetch it). If no -// processes are available for the tenant, TryLookupTenantPods will return the -// empty set (unlike LookupTenantPod). +// TryLookupTenantPods returns a list of SQL pods in the RUNNING and DRAINING +// states for thegiven tenant. It returns a GRPC NotFound error if the tenant +// does not exist (e.g. it has not yet been created) or if it has not yet been +// fetched into the directory's cache (TryLookupTenantPods will never attempt to +// fetch it). If no processes are available for the tenant, TryLookupTenantPods +// will return the empty set (unlike LookupTenantPod). // // WARNING: Callers should never attempt to modify values returned by this // method, or else they may be a race. Other instances may be reading from the @@ -444,8 +454,8 @@ func (d *directoryCache) updateTenantEntry(ctx context.Context, pod *Pod) { } switch pod.State { - case RUNNING: - // Add entries of RUNNING pods if they are not already present. + case RUNNING, DRAINING: + // Add entries of RUNNING and DRAINING pods if they are not already present. if entry.AddPod(pod) { log.Infof(ctx, "added IP address %s with load %.3f for tenant %d", pod.Addr, pod.Load, pod.TenantID) } else { @@ -457,7 +467,7 @@ func (d *directoryCache) updateTenantEntry(ctx context.Context, pod *Pod) { log.Infof(ctx, "updated IP address %s with load %.3f for tenant %d", pod.Addr, pod.Load, pod.TenantID) } default: - // Remove addresses of DRAINING and DELETING pods. + // Remove addresses of DELETING pods. if entry.RemovePodByAddr(pod.Addr) { log.Infof(ctx, "deleted IP address %s for tenant %d", pod.Addr, pod.TenantID) } diff --git a/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go b/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go index c1618c5d8320..380df7ca2522 100644 --- a/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go +++ b/pkg/ccl/sqlproxyccl/tenant/directory_cache_test.go @@ -92,6 +92,7 @@ func TestWatchPods(t *testing.T) { require.Equal(t, tenantID.ToUint64(), pod.TenantID) require.Equal(t, addr, pod.Addr) require.Equal(t, tenant.RUNNING, pod.State) + require.False(t, pod.StateTimestamp.IsZero()) // Trigger drain of pod. tds.Drain() @@ -99,13 +100,7 @@ func TestWatchPods(t *testing.T) { require.Equal(t, tenantID.ToUint64(), pod.TenantID) require.Equal(t, addr, pod.Addr) require.Equal(t, tenant.DRAINING, pod.State) - - // Ensure that all addresses have been cleared from the directory, since - // it should only return RUNNING addresses. - require.Eventually(t, func() bool { - tenantPods, _ := dir.TryLookupTenantPods(ctx, tenantID) - return len(tenantPods) == 0 - }, 10*time.Second, 100*time.Millisecond) + require.False(t, pod.StateTimestamp.IsZero()) // Now shut the tenant directory down. processes := tds.Get(tenantID) @@ -121,13 +116,14 @@ func TestWatchPods(t *testing.T) { require.Equal(t, tenantID.ToUint64(), pod.TenantID) require.Equal(t, addr, pod.Addr) require.Equal(t, tenant.DELETING, pod.State) + require.False(t, pod.StateTimestamp.IsZero()) - // We know that the directory should have been emptied earlier since we - // don't add DRAINING pods to the directory, so putting the pod into the - // DELETING state should not make a difference. - pods, err = dir.TryLookupTenantPods(ctx, tenantID) - require.NoError(t, err) - require.Empty(t, pods) + // Ensure that all addresses have been cleared from the directory, since + // it should only return RUNNING or DRAINING addresses. + require.Eventually(t, func() bool { + tenantPods, _ := dir.TryLookupTenantPods(ctx, tenantID) + return len(tenantPods) == 0 + }, 10*time.Second, 100*time.Millisecond) // Resume tenant again by a direct call to the directory server _, err = tds.EnsurePod(ctx, &tenant.EnsurePodRequest{tenantID.ToUint64()}) @@ -148,6 +144,7 @@ func TestWatchPods(t *testing.T) { require.Equal(t, tenantID.ToUint64(), pod.TenantID) require.Equal(t, addr, pod.Addr) require.Equal(t, tenant.RUNNING, pod.State) + require.False(t, pod.StateTimestamp.IsZero()) // Verify that LookupTenantPods returns the pod's IP address. pods, err = dir.LookupTenantPods(ctx, tenantID, "") @@ -177,6 +174,7 @@ func TestWatchPods(t *testing.T) { require.Equal(t, tenantID.ToUint64(), pod.TenantID) require.Equal(t, addr, pod.Addr) require.Equal(t, tenant.DELETING, pod.State) + require.False(t, pod.StateTimestamp.IsZero()) // Verify that a new call to LookupTenantPods will resume again the tenant. pods, err = dir.LookupTenantPods(ctx, tenantID, "") @@ -189,6 +187,7 @@ func TestWatchPods(t *testing.T) { require.Equal(t, tenantID.ToUint64(), pod.TenantID) require.Equal(t, addr, pod.Addr) require.Equal(t, tenant.RUNNING, pod.State) + require.False(t, pod.StateTimestamp.IsZero()) } func TestCancelLookups(t *testing.T) { @@ -352,6 +351,10 @@ func TestRefreshThrottling(t *testing.T) { require.NoError(t, dir.ReportFailure(ctx, tenantID, addr)) pods, err = dir.TryLookupTenantPods(ctx, tenantID) require.NoError(t, err) + require.NotEmpty(t, pods) + + // Reset StateTimestamp for deterministic comparison. + pods[0].StateTimestamp = time.Time{} require.Equal(t, []*tenant.Pod{{ TenantID: tenantID.ToUint64(), Addr: addr, @@ -364,6 +367,10 @@ func TestRefreshThrottling(t *testing.T) { require.NoError(t, dir.ReportFailure(ctx, tenantID, addr)) pods, err = dir.TryLookupTenantPods(ctx, tenantID) require.NoError(t, err) + require.NotEmpty(t, pods) + + // Reset StateTimestamp for deterministic comparison. + pods[0].StateTimestamp = time.Time{} require.Equal(t, []*tenant.Pod{{ TenantID: tenantID.ToUint64(), Addr: addr, diff --git a/pkg/ccl/sqlproxyccl/tenant/entry.go b/pkg/ccl/sqlproxyccl/tenant/entry.go index 8e69164f7841..8d4fd6c5aa33 100644 --- a/pkg/ccl/sqlproxyccl/tenant/entry.go +++ b/pkg/ccl/sqlproxyccl/tenant/entry.go @@ -247,26 +247,17 @@ func (e *tenantEntry) fetchPodsLocked( return nil, err } - // Get updated list of RUNNING pod IP addresses and save it to the entry. - tenantPods = make([]*Pod, 0, len(list.Pods)) - for i := range list.Pods { - pod := list.Pods[i] - if pod.State == RUNNING { - tenantPods = append(tenantPods, pod) - } - } - // Need to lock in case another thread is reading the IP addresses (e.g. in // ChoosePodAddr). e.pods.Lock() defer e.pods.Unlock() - e.pods.pods = tenantPods + e.pods.pods = list.Pods - if len(tenantPods) != 0 { - log.Infof(ctx, "fetched IP addresses: %v", tenantPods) + if len(e.pods.pods) != 0 { + log.Infof(ctx, "fetched IP addresses: %v", e.pods.pods) } - return tenantPods, nil + return e.pods.pods, nil } // canRefreshLocked returns true if it's been at least X milliseconds since the diff --git a/pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go b/pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go index 97893e0f6172..857f8cb4193a 100644 --- a/pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go +++ b/pkg/ccl/sqlproxyccl/tenantdirsvr/test_directory_svr.go @@ -180,10 +180,11 @@ func (s *TestDirectoryServer) SetFakeLoad(id roachpb.TenantID, addr net.Addr, fa defer s.listen.RUnlock() s.notifyEventListenersLocked(&tenant.WatchPodsResponse{ Pod: &tenant.Pod{ - Addr: addr.String(), - TenantID: id.ToUint64(), - Load: fakeLoad, - State: tenant.UNKNOWN, + Addr: addr.String(), + TenantID: id.ToUint64(), + Load: fakeLoad, + State: tenant.UNKNOWN, + StateTimestamp: timeutil.Now(), }, }) } @@ -266,9 +267,10 @@ func (s *TestDirectoryServer) Drain() { defer s.listen.RUnlock() s.notifyEventListenersLocked(&tenant.WatchPodsResponse{ Pod: &tenant.Pod{ - TenantID: tenantID, - Addr: addr.String(), - State: tenant.DRAINING, + TenantID: tenantID, + Addr: addr.String(), + State: tenant.DRAINING, + StateTimestamp: timeutil.Now(), }, }) } @@ -341,10 +343,11 @@ func (s *TestDirectoryServer) listLocked( resp := tenant.ListPodsResponse{} for addr, proc := range processByAddr { resp.Pods = append(resp.Pods, &tenant.Pod{ - TenantID: req.TenantID, - Addr: addr.String(), - State: tenant.RUNNING, - Load: proc.FakeLoad, + TenantID: req.TenantID, + Addr: addr.String(), + State: tenant.RUNNING, + Load: proc.FakeLoad, + StateTimestamp: timeutil.Now(), }) } return &resp, nil @@ -362,10 +365,11 @@ func (s *TestDirectoryServer) registerInstanceLocked(tenantID uint64, process *P defer s.listen.RUnlock() s.notifyEventListenersLocked(&tenant.WatchPodsResponse{ Pod: &tenant.Pod{ - TenantID: tenantID, - Addr: process.SQL.String(), - State: tenant.RUNNING, - Load: process.FakeLoad, + TenantID: tenantID, + Addr: process.SQL.String(), + State: tenant.RUNNING, + Load: process.FakeLoad, + StateTimestamp: timeutil.Now(), }, }) } @@ -385,9 +389,10 @@ func (s *TestDirectoryServer) deregisterInstance(tenantID uint64, sql net.Addr) defer s.listen.RUnlock() s.notifyEventListenersLocked(&tenant.WatchPodsResponse{ Pod: &tenant.Pod{ - TenantID: tenantID, - Addr: sql.String(), - State: tenant.DELETING, + TenantID: tenantID, + Addr: sql.String(), + State: tenant.DELETING, + StateTimestamp: timeutil.Now(), }, }) } diff --git a/pkg/ccl/sqlproxyccl/tenantdirsvr/test_simple_directory_svr.go b/pkg/ccl/sqlproxyccl/tenantdirsvr/test_simple_directory_svr.go index dc178028d07f..6961a84407cf 100644 --- a/pkg/ccl/sqlproxyccl/tenantdirsvr/test_simple_directory_svr.go +++ b/pkg/ccl/sqlproxyccl/tenantdirsvr/test_simple_directory_svr.go @@ -14,6 +14,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl/tenant" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/util/syncutil" + "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/gogo/status" "google.golang.org/grpc" "google.golang.org/grpc/codes" @@ -67,9 +68,10 @@ func (d *TestSimpleDirectoryServer) ListPods( return &tenant.ListPodsResponse{ Pods: []*tenant.Pod{ { - TenantID: req.TenantID, - Addr: d.podAddr, - State: tenant.RUNNING, + TenantID: req.TenantID, + Addr: d.podAddr, + State: tenant.RUNNING, + StateTimestamp: timeutil.Now(), }, }, }, nil diff --git a/pkg/ccl/testccl/sqlccl/session_revival_test.go b/pkg/ccl/testccl/sqlccl/session_revival_test.go index 97861122eaaf..c56ac5bf9794 100644 --- a/pkg/ccl/testccl/sqlccl/session_revival_test.go +++ b/pkg/ccl/testccl/sqlccl/session_revival_test.go @@ -43,9 +43,7 @@ func TestAuthenticateWithSessionRevivalToken(t *testing.T) { _, err := tenantDB.Exec("CREATE USER testuser WITH PASSWORD 'hunter2'") require.NoError(t, err) - // TODO(rafi): use ALTER TENANT ALL when available. - _, err = mainDB.Exec(`INSERT INTO system.tenant_settings (tenant_id, name, value, value_type) VALUES - (0, 'server.user_login.session_revival_token.enabled', 'true', 'b')`) + _, err = mainDB.Exec(`ALTER TENANT ALL SET CLUSTER SETTING server.user_login.session_revival_token.enabled = true`) require.NoError(t, err) var token string diff --git a/pkg/cli/node.go b/pkg/cli/node.go index 1ddf911f62ed..3fc671a2eaa1 100644 --- a/pkg/cli/node.go +++ b/pkg/cli/node.go @@ -492,7 +492,7 @@ func runDecommissionNodeImpl( } if numReplicaReport > 0 { - printDecommissionReplicas(ctx, *resp) + printDecommissionReplicas(*resp) } if !reflect.DeepEqual(&prevResponse, resp) { @@ -622,13 +622,13 @@ func printDecommissionStatus(resp serverpb.DecommissionStatusResponse) error { clisqlexec.NewRowSliceIter(decommissionResponseValueToRows(resp.Status), decommissionResponseAlignment())) } -func printDecommissionReplicas(ctx context.Context, resp serverpb.DecommissionStatusResponse) { - fmt.Fprintln(stderr, "\npossible decommission stall detected; reporting decommissioning replicas") +func printDecommissionReplicas(resp serverpb.DecommissionStatusResponse) { + fmt.Fprintln(stderr, "\npossible decommission stall detected") for _, nodeStatus := range resp.Status { for _, replica := range nodeStatus.ReportedReplicas { fmt.Fprintf(stderr, - "n%d still has replica id %d for range r%d", + "n%d still has replica id %d for range r%d\n", nodeStatus.NodeID, replica.ReplicaID, replica.RangeID, diff --git a/pkg/cmd/roachtest/tests/follower_reads.go b/pkg/cmd/roachtest/tests/follower_reads.go index 703866653c16..ffed18cd5fb5 100644 --- a/pkg/cmd/roachtest/tests/follower_reads.go +++ b/pkg/cmd/roachtest/tests/follower_reads.go @@ -269,20 +269,24 @@ func runFollowerReadsTest( t.Fatalf("failed to get follower read counts: %v", err) } - // Perform reads on each node and ensure we get the expected value. Do so a - // few times on each follower to give caches time to warm up. + // Perform reads on each node and ensure we get the expected value. Do so for + // 15 seconds to give closed timestamps a chance to propagate and caches time + // to warm up. t.L().Printf("warming up reads") g, gCtx := errgroup.WithContext(ctx) k, v := chooseKV() + until := timeutil.Now().Add(15 * time.Second) for i := 1; i <= c.Spec().NodeCount; i++ { fn := verifySelect(gCtx, i, k, v) g.Go(func() error { - for j := 0; j < 100; j++ { + for { + if timeutil.Now().After(until) { + return nil + } if err := fn(); err != nil { return err } } - return nil }) } if err := g.Wait(); err != nil { diff --git a/pkg/col/coldata/BUILD.bazel b/pkg/col/coldata/BUILD.bazel index be01781363c8..aaec8a0b3b57 100644 --- a/pkg/col/coldata/BUILD.bazel +++ b/pkg/col/coldata/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "coldata", @@ -35,7 +36,6 @@ go_test( srcs = [ "batch_test.go", "bytes_test.go", - "dep_test.go", "main_test.go", "nulls_test.go", "vec_test.go", @@ -47,7 +47,6 @@ go_test( "//pkg/sql/colconv", "//pkg/sql/randgen", "//pkg/sql/types", - "//pkg/testutils/buildutil", "//pkg/util/leaktest", "//pkg/util/randutil", "@com_github_cockroachdb_errors//:errors", @@ -74,3 +73,11 @@ genrule( "//pkg/gen:__pkg__", ], ) + +disallowed_imports_test( + "coldata", + [ + "//pkg/sql/catalog", + "//pkg/sql/sem/tree", + ], +) diff --git a/pkg/col/coldata/dep_test.go b/pkg/col/coldata/dep_test.go deleted file mode 100644 index 5d38d9addc35..000000000000 --- a/pkg/col/coldata/dep_test.go +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package coldata - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" -) - -func TestNoLinkForbidden(t *testing.T) { - defer leaktest.AfterTest(t)() - - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/col/coldata", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/catalog", - "github.com/cockroachdb/cockroach/pkg/sql/sem/tree", - }, nil, - ) -} diff --git a/pkg/jobs/metrics.go b/pkg/jobs/metrics.go index eec66c8e10b1..4ff9399b6e87 100644 --- a/pkg/jobs/metrics.go +++ b/pkg/jobs/metrics.go @@ -24,6 +24,8 @@ import ( // Metrics are for production monitoring of each job type. type Metrics struct { JobMetrics [jobspb.NumJobTypes]*JobTypeMetrics + // RunningNonIdleJobs is the total number of running jobs that are not idle. + RunningNonIdleJobs *metric.Gauge RowLevelTTL metric.Struct Changefeed metric.Struct @@ -173,6 +175,16 @@ var ( Unit: metric.Unit_COUNT, MetricType: io_prometheus_client.MetricType_GAUGE, } + + // MetaRunningNonIdleJobs is the count of currently running jobs that are not + // reporting as being idle. + MetaRunningNonIdleJobs = metric.Metadata{ + Name: "jobs.running_non_idle", + Help: "number of running jobs that are not idle", + Measurement: "jobs", + Unit: metric.Unit_COUNT, + MetricType: io_prometheus_client.MetricType_GAUGE, + } ) // MetricStruct implements the metric.Struct interface. @@ -192,6 +204,7 @@ func (m *Metrics) init(histogramWindowInterval time.Duration) { m.AdoptIterations = metric.NewCounter(metaAdoptIterations) m.ClaimedJobs = metric.NewCounter(metaClaimedJobs) m.ResumedJobs = metric.NewCounter(metaResumedClaimedJobs) + m.RunningNonIdleJobs = metric.NewGauge(MetaRunningNonIdleJobs) for i := 0; i < jobspb.NumJobTypes; i++ { jt := jobspb.Type(i) if jt == jobspb.TypeUnspecified { // do not track TypeUnspecified diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go index c879feb5650d..e06168832ae0 100644 --- a/pkg/jobs/registry.go +++ b/pkg/jobs/registry.go @@ -1125,7 +1125,11 @@ func (r *Registry) stepThroughStateMachine( var err error func() { jm.CurrentlyRunning.Inc(1) - defer jm.CurrentlyRunning.Dec(1) + r.metrics.RunningNonIdleJobs.Inc(1) + defer func() { + jm.CurrentlyRunning.Dec(1) + r.metrics.RunningNonIdleJobs.Dec(1) + }() err = resumer.Resume(resumeCtx, execCtx) }() @@ -1219,7 +1223,11 @@ func (r *Registry) stepThroughStateMachine( var err error func() { jm.CurrentlyRunning.Inc(1) - defer jm.CurrentlyRunning.Dec(1) + r.metrics.RunningNonIdleJobs.Inc(1) + defer func() { + jm.CurrentlyRunning.Dec(1) + r.metrics.RunningNonIdleJobs.Dec(1) + }() err = resumer.OnFailOrCancel(onFailOrCancelCtx, execCtx) }() if successOnFailOrCancel := err == nil; successOnFailOrCancel { @@ -1307,8 +1315,10 @@ func (r *Registry) MarkIdle(job *Job, isIdle bool) { if aj.isIdle != isIdle { log.Infof(r.serverCtx, "%s job %d: toggling idleness to %+v", jobType, job.ID(), isIdle) if isIdle { + r.metrics.RunningNonIdleJobs.Dec(1) jm.CurrentlyIdle.Inc(1) } else { + r.metrics.RunningNonIdleJobs.Inc(1) jm.CurrentlyIdle.Dec(1) } aj.isIdle = isIdle diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go index 49a6336acc2b..d59fb5648e6f 100644 --- a/pkg/jobs/registry_test.go +++ b/pkg/jobs/registry_test.go @@ -1038,22 +1038,26 @@ func TestJobIdleness(t *testing.T) { job2 := createJob() require.False(t, r.TestingIsJobIdle(job1.ID())) - + require.EqualValues(t, 2, r.metrics.RunningNonIdleJobs.Value()) r.MarkIdle(job1, true) r.MarkIdle(job2, true) require.True(t, r.TestingIsJobIdle(job1.ID())) require.Equal(t, int64(2), currentlyIdle.Value()) + require.EqualValues(t, 0, r.metrics.RunningNonIdleJobs.Value()) // Repeated calls should not increase metric r.MarkIdle(job1, true) r.MarkIdle(job1, true) require.Equal(t, int64(2), currentlyIdle.Value()) + require.EqualValues(t, 0, r.metrics.RunningNonIdleJobs.Value()) r.MarkIdle(job1, false) require.Equal(t, int64(1), currentlyIdle.Value()) require.False(t, r.TestingIsJobIdle(job1.ID())) + require.EqualValues(t, 1, r.metrics.RunningNonIdleJobs.Value()) r.MarkIdle(job2, false) require.Equal(t, int64(0), currentlyIdle.Value()) + require.EqualValues(t, 2, r.metrics.RunningNonIdleJobs.Value()) // Let the jobs complete resumeErrChan <- nil diff --git a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go index 01bfa8bdd79a..352c7b1807bd 100644 --- a/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go +++ b/pkg/kv/kvserver/batcheval/cmd_add_sstable_test.go @@ -1440,7 +1440,7 @@ func TestAddSSTableSSTTimestampToRequestTimestampRespectsClosedTS(t *testing.T) require.NoError(t, err) r, store, err := s.GetStores().(*kvserver.Stores).GetReplicaForRangeID(ctx, rd.RangeID) require.NoError(t, err) - closedTS := r.GetClosedTimestamp(ctx) + closedTS := r.GetCurrentClosedTimestamp(ctx) require.NotZero(t, closedTS) // Add an SST writing below the closed timestamp. It should get pushed above it. diff --git a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go index ecbf7b80754d..eea2c6e0e8a8 100644 --- a/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go +++ b/pkg/kv/kvserver/batcheval/cmd_query_resolved_timestamp.go @@ -56,7 +56,7 @@ func QueryResolvedTimestamp( // because QueryResolvedTimestamp requests are often run without acquiring // latches (see roachpb.INCONSISTENT) and often also on follower replicas, // so latches won't help them to synchronize with writes. - closedTS := cArgs.EvalCtx.GetClosedTimestamp(ctx) + closedTS := cArgs.EvalCtx.GetClosedTimestampOlderThanStorageSnapshot() // Compute the minimum timestamp of any intent in the request's key span, // which may span the entire range, but does not need to. diff --git a/pkg/kv/kvserver/batcheval/cmd_subsume.go b/pkg/kv/kvserver/batcheval/cmd_subsume.go index cfd51563854b..5818c29fa318 100644 --- a/pkg/kv/kvserver/batcheval/cmd_subsume.go +++ b/pkg/kv/kvserver/batcheval/cmd_subsume.go @@ -152,7 +152,7 @@ func Subsume( // think about. priorReadSum.Merge(rspb.FromTimestamp(reply.FreezeStart.ToTimestamp())) reply.ReadSummary = &priorReadSum - reply.ClosedTimestamp = cArgs.EvalCtx.GetClosedTimestamp(ctx) + reply.ClosedTimestamp = cArgs.EvalCtx.GetCurrentClosedTimestamp(ctx) return result.Result{}, nil } diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go index d8536d9781a0..ff14661a6dc6 100644 --- a/pkg/kv/kvserver/batcheval/eval_context.go +++ b/pkg/kv/kvserver/batcheval/eval_context.go @@ -49,6 +49,8 @@ type Limiters struct { // underlying state. type EvalContext interface { fmt.Stringer + ImmutableEvalContext + ClusterSettings() *cluster.Settings EvalKnobs() kvserverbase.BatchEvalTestingKnobs @@ -111,12 +113,6 @@ type EvalContext interface { // requests on the range. GetCurrentReadSummary(ctx context.Context) rspb.ReadSummary - // GetClosedTimestamp returns the current closed timestamp on the range. - // It is expected that a caller will have performed some action (either - // calling RevokeLease or WatchForMerge) to freeze further progression of - // the closed timestamp before calling this method. - GetClosedTimestamp(ctx context.Context) hlc.Timestamp - GetExternalStorage(ctx context.Context, dest roachpb.ExternalStorage) (cloud.ExternalStorage, error) GetExternalStorageFromURI(ctx context.Context, uri string, user security.SQLUsername) (cloud.ExternalStorage, error) @@ -141,6 +137,24 @@ type EvalContext interface { // GetEngineCapacity returns the store's underlying engine capacity; other // StoreCapacity fields not related to engine capacity are not populated. GetEngineCapacity() (roachpb.StoreCapacity, error) + + // GetCurrentClosedTimestamp returns the current closed timestamp on the + // range. It is expected that a caller will have performed some action (either + // calling RevokeLease or WatchForMerge) to freeze further progression of the + // closed timestamp before calling this method. + GetCurrentClosedTimestamp(ctx context.Context) hlc.Timestamp + + // Release returns the memory allocated by the EvalContext implementation to a + // sync.Pool. + Release() +} + +// ImmutableEvalContext is like EvalContext, but it encapsulates state that +// needs to be immutable during the course of command evaluation. +type ImmutableEvalContext interface { + // GetClosedTimestampOlderThanStorageSnapshot returns the closed timestamp + // that was active before the state of the storage engine was pinned. + GetClosedTimestampOlderThanStorageSnapshot() hlc.Timestamp } // MockEvalCtx is a dummy implementation of EvalContext for testing purposes. @@ -255,7 +269,10 @@ func (m *mockEvalCtxImpl) GetRangeInfo(ctx context.Context) roachpb.RangeInfo { func (m *mockEvalCtxImpl) GetCurrentReadSummary(ctx context.Context) rspb.ReadSummary { return m.CurrentReadSummary } -func (m *mockEvalCtxImpl) GetClosedTimestamp(ctx context.Context) hlc.Timestamp { +func (m *mockEvalCtxImpl) GetClosedTimestampOlderThanStorageSnapshot() hlc.Timestamp { + return m.ClosedTimestamp +} +func (m *mockEvalCtxImpl) GetCurrentClosedTimestamp(ctx context.Context) hlc.Timestamp { return m.ClosedTimestamp } func (m *mockEvalCtxImpl) GetExternalStorage( @@ -287,3 +304,4 @@ func (m *mockEvalCtxImpl) GetMaxBytes() int64 { func (m *mockEvalCtxImpl) GetEngineCapacity() (roachpb.StoreCapacity, error) { return roachpb.StoreCapacity{Available: 1, Capacity: 1}, nil } +func (m *mockEvalCtxImpl) Release() {} diff --git a/pkg/kv/kvserver/client_merge_test.go b/pkg/kv/kvserver/client_merge_test.go index c2e662ceaeca..d4a6b0d0e10a 100644 --- a/pkg/kv/kvserver/client_merge_test.go +++ b/pkg/kv/kvserver/client_merge_test.go @@ -2149,7 +2149,7 @@ func TestStoreRangeMergeLHSLeaseTransfersAfterFreezeTime(t *testing.T) { } return nil }) - lhsClosedTS := lhsLeaseholder.GetClosedTimestamp(ctx) + lhsClosedTS := lhsLeaseholder.GetCurrentClosedTimestamp(ctx) require.NotEmpty(t, lhsClosedTS) // Finally, allow the merge to complete. It should complete successfully. diff --git a/pkg/kv/kvserver/client_replica_circuit_breaker_test.go b/pkg/kv/kvserver/client_replica_circuit_breaker_test.go index 795d6d79be80..8181bd7044dd 100644 --- a/pkg/kv/kvserver/client_replica_circuit_breaker_test.go +++ b/pkg/kv/kvserver/client_replica_circuit_breaker_test.go @@ -998,7 +998,7 @@ func (cbt *circuitBreakerTest) FollowerRead(idx int) error { repl := cbt.repls[idx] get := roachpb.NewGet(repl.Desc().StartKey.AsRawKey(), false /* forUpdate */) ctx := context.Background() - ts := repl.GetClosedTimestamp(ctx) + ts := repl.GetCurrentClosedTimestamp(ctx) return cbt.SendCtxTS(ctx, idx, get, ts) } diff --git a/pkg/kv/kvserver/closed_timestamp_test.go b/pkg/kv/kvserver/closed_timestamp_test.go index e38ed1a89e2d..3c069a88dc1c 100644 --- a/pkg/kv/kvserver/closed_timestamp_test.go +++ b/pkg/kv/kvserver/closed_timestamp_test.go @@ -774,7 +774,7 @@ SET CLUSTER SETTING kv.closed_timestamp.follower_reads_enabled = true; require.NoError(t, err) r, err := store.GetReplica(rightDesc.RangeID) require.NoError(t, err) - maxClosed := r.GetClosedTimestamp(ctx) + maxClosed := r.GetCurrentClosedTimestamp(ctx) // Note that maxClosed would not necessarily be below the freeze start if // this was a LEAD_FOR_GLOBAL_READS range. assert.True(t, maxClosed.LessEq(freezeStartTimestamp), @@ -809,7 +809,7 @@ SET CLUSTER SETTING kv.closed_timestamp.follower_reads_enabled = true; mergedLeaseholder, err := leftLeaseholderStore.GetReplica(leftDesc.RangeID) require.NoError(t, err) writeTime := rhsLeaseStart.Prev() - require.True(t, mergedLeaseholder.GetClosedTimestamp(ctx).Less(writeTime)) + require.True(t, mergedLeaseholder.GetCurrentClosedTimestamp(ctx).Less(writeTime)) var baWrite roachpb.BatchRequest baWrite.Header.RangeID = leftDesc.RangeID baWrite.Header.Timestamp = writeTime diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go index d35bd5c91e93..70514440d40a 100644 --- a/pkg/kv/kvserver/replica.go +++ b/pkg/kv/kvserver/replica.go @@ -703,8 +703,6 @@ type Replica struct { } } -var _ batcheval.EvalContext = &Replica{} - // String returns the string representation of the replica using an // inconsistent copy of the range descriptor. Therefore, String does not // require a lock and its output may not be atomic with other ongoing work in @@ -1232,7 +1230,7 @@ func (r *Replica) State(ctx context.Context) kvserverpb.RangeInfo { // NB: this acquires an RLock(). Reentrant RLocks are deadlock prone, so do // this first before RLocking below. Performance of this extra lock // acquisition is not a concern. - ri.ActiveClosedTimestamp = r.GetClosedTimestamp(ctx) + ri.ActiveClosedTimestamp = r.GetCurrentClosedTimestamp(ctx) // NB: numRangefeedRegistrations doesn't require Replica.mu to be locked. // However, it does require coordination between multiple goroutines, so diff --git a/pkg/kv/kvserver/replica_closedts_internal_test.go b/pkg/kv/kvserver/replica_closedts_internal_test.go index 97df78100077..627bd4963d2e 100644 --- a/pkg/kv/kvserver/replica_closedts_internal_test.go +++ b/pkg/kv/kvserver/replica_closedts_internal_test.go @@ -509,7 +509,8 @@ func (r *mockReceiver) HTML() string { return "" } -// Test that r.GetClosedTimestamp() mixes its sources of information correctly. +// Test that r.GetCurrentClosedTimestamp() mixes its sources of information +// correctly. func TestReplicaClosedTimestamp(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) @@ -567,7 +568,7 @@ func TestReplicaClosedTimestamp(t *testing.T) { tc.repl.mu.state.RaftClosedTimestamp = test.raftClosed tc.repl.mu.state.LeaseAppliedIndex = uint64(test.applied) tc.repl.mu.Unlock() - require.Equal(t, test.expClosed, tc.repl.GetClosedTimestamp(ctx)) + require.Equal(t, test.expClosed, tc.repl.GetCurrentClosedTimestamp(ctx)) }) } } diff --git a/pkg/kv/kvserver/replica_eval_context.go b/pkg/kv/kvserver/replica_eval_context.go index 47744e471d5f..28a59ed0d6dd 100644 --- a/pkg/kv/kvserver/replica_eval_context.go +++ b/pkg/kv/kvserver/replica_eval_context.go @@ -12,10 +12,12 @@ package kvserver import ( "context" + "sync" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/batcheval" "github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset" "github.com/cockroachdb/cockroach/pkg/util" + "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" ) @@ -28,19 +30,78 @@ import ( // Do not introduce new uses of this. var todoSpanSet = &spanset.SpanSet{} +var evalContextPool = sync.Pool{ + New: func() interface{} { + return &evalContextImpl{} + }, +} + +// evalContextImpl implements the batcheval.EvalContext interface. +type evalContextImpl struct { + *Replica + // NB: We cannot use the emptiness of `closedTS` to determine whether the + // closed timestamp was elided during the creation of this eval context, so we + // track it separately. + closedTSElided bool + closedTS hlc.Timestamp +} + +func newEvalContextImpl( + ctx context.Context, r *Replica, requiresClosedTSOlderThanStorageSnap bool, +) (ec *evalContextImpl) { + var closedTS hlc.Timestamp + if requiresClosedTSOlderThanStorageSnap { + // We elide this call to get the replica's current closed timestamp unless + // the request requires it, in order to avoid redundant mutex contention. + closedTS = r.GetCurrentClosedTimestamp(ctx) + } + + ec = evalContextPool.Get().(*evalContextImpl) + *ec = evalContextImpl{ + Replica: r, + closedTSElided: !requiresClosedTSOlderThanStorageSnap, + closedTS: closedTS, + } + return ec +} + +// GetClosedTimestampOlderThanStorageSnapshot implements the EvalContext +// interface. +func (ec *evalContextImpl) GetClosedTimestampOlderThanStorageSnapshot() hlc.Timestamp { + if ec.closedTSElided { + panic("closed timestamp was elided during eval context creation; does the" + + " request set the requiresClosedTimestamp flag?") + } + return ec.closedTS +} + +// Release implements the EvalContext interface. +func (ec *evalContextImpl) Release() { + *ec = evalContextImpl{} + evalContextPool.Put(ec) +} + +var _ batcheval.EvalContext = &evalContextImpl{} + // NewReplicaEvalContext returns a batcheval.EvalContext to use for command // evaluation. The supplied SpanSet will be ignored except for race builds, in // which case state access is asserted against it. A SpanSet must always be // passed. -func NewReplicaEvalContext(r *Replica, ss *spanset.SpanSet) batcheval.EvalContext { +// The caller must call rec.Release() once done with the evaluation context in +// order to return its memory back to a sync.Pool. +func NewReplicaEvalContext( + ctx context.Context, r *Replica, ss *spanset.SpanSet, requiresClosedTSOlderThanStorageSnap bool, +) (rec batcheval.EvalContext) { if ss == nil { log.Fatalf(r.AnnotateCtx(context.Background()), "can't create a ReplicaEvalContext with assertions but no SpanSet") } + + rec = newEvalContextImpl(ctx, r, requiresClosedTSOlderThanStorageSnap) if util.RaceEnabled { return &SpanSetReplicaEvalContext{ - i: r, + i: rec, ss: *ss, } } - return r + return rec } diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go index 21f767adafbd..80a7fb42af26 100644 --- a/pkg/kv/kvserver/replica_eval_context_span.go +++ b/pkg/kv/kvserver/replica_eval_context_span.go @@ -222,9 +222,15 @@ func (rec *SpanSetReplicaEvalContext) GetCurrentReadSummary(ctx context.Context) return rec.i.GetCurrentReadSummary(ctx) } -// GetClosedTimestamp is part of the EvalContext interface. -func (rec *SpanSetReplicaEvalContext) GetClosedTimestamp(ctx context.Context) hlc.Timestamp { - return rec.i.GetClosedTimestamp(ctx) +// GetCurrentClosedTimestamp is part of the EvalContext interface. +func (rec *SpanSetReplicaEvalContext) GetCurrentClosedTimestamp(ctx context.Context) hlc.Timestamp { + return rec.i.GetCurrentClosedTimestamp(ctx) +} + +// GetClosedTimestampOlderThanStorageSnapshot is part of the EvalContext +// interface. +func (rec *SpanSetReplicaEvalContext) GetClosedTimestampOlderThanStorageSnapshot() hlc.Timestamp { + return rec.i.GetClosedTimestampOlderThanStorageSnapshot() } // GetExternalStorage returns an ExternalStorage object, based on @@ -267,3 +273,6 @@ func (rec *SpanSetReplicaEvalContext) GetMaxBytes() int64 { func (rec *SpanSetReplicaEvalContext) GetEngineCapacity() (roachpb.StoreCapacity, error) { return rec.i.GetEngineCapacity() } + +// Release implements the batcheval.EvalContext interface. +func (rec *SpanSetReplicaEvalContext) Release() { rec.i.Release() } diff --git a/pkg/kv/kvserver/replica_follower_read.go b/pkg/kv/kvserver/replica_follower_read.go index a189a6798280..3edd73aa51b5 100644 --- a/pkg/kv/kvserver/replica_follower_read.go +++ b/pkg/kv/kvserver/replica_follower_read.go @@ -81,7 +81,7 @@ func (r *Replica) canServeFollowerReadRLocked(ctx context.Context, ba *roachpb.B } requiredFrontier := ba.RequiredFrontier() - maxClosed := r.getClosedTimestampRLocked(ctx, requiredFrontier /* sufficient */) + maxClosed := r.getCurrentClosedTimestampLocked(ctx, requiredFrontier /* sufficient */) canServeFollowerRead := requiredFrontier.LessEq(maxClosed) tsDiff := requiredFrontier.GoTime().Sub(maxClosed.GoTime()) if !canServeFollowerRead { @@ -106,13 +106,13 @@ func (r *Replica) canServeFollowerReadRLocked(ctx context.Context, ba *roachpb.B return true } -// getClosedTimestampRLocked is like maxClosed, except that it requires r.mu to be -// rlocked. It also optionally takes a hint: if sufficient is not -// empty, getClosedTimestampRLocked might return a timestamp that's lower than the -// maximum closed timestamp that we know about, as long as the returned -// timestamp is still >= sufficient. This is a performance optimization because -// we can avoid consulting the ClosedTimestampReceiver. -func (r *Replica) getClosedTimestampRLocked( +// getCurrentClosedTimestampRLocked is like GetCurrentClosedTimestamp, except +// that it requires r.mu to be RLocked. It also optionally takes a hint: if +// sufficient is not empty, getClosedTimestampRLocked might return a timestamp +// that's lower than the maximum closed timestamp that we know about, as long as +// the returned timestamp is still >= sufficient. This is a performance +// optimization because we can avoid consulting the ClosedTimestampReceiver. +func (r *Replica) getCurrentClosedTimestampLocked( ctx context.Context, sufficient hlc.Timestamp, ) hlc.Timestamp { appliedLAI := ctpb.LAI(r.mu.state.LeaseAppliedIndex) @@ -126,11 +126,10 @@ func (r *Replica) getClosedTimestampRLocked( return maxClosed } -// GetClosedTimestamp returns the maximum closed timestamp for this range. -// -// GetClosedTimestamp is part of the EvalContext interface. -func (r *Replica) GetClosedTimestamp(ctx context.Context) hlc.Timestamp { +// GetCurrentClosedTimestamp returns the current maximum closed timestamp for +// this range. +func (r *Replica) GetCurrentClosedTimestamp(ctx context.Context) hlc.Timestamp { r.mu.RLock() defer r.mu.RUnlock() - return r.getClosedTimestampRLocked(ctx, hlc.Timestamp{} /* sufficient */) + return r.getCurrentClosedTimestampLocked(ctx, hlc.Timestamp{} /* sufficient */) } diff --git a/pkg/kv/kvserver/replica_gossip.go b/pkg/kv/kvserver/replica_gossip.go index f22a76ced01f..2ce320d8effc 100644 --- a/pkg/kv/kvserver/replica_gossip.go +++ b/pkg/kv/kvserver/replica_gossip.go @@ -174,7 +174,10 @@ func (r *Replica) MaybeGossipNodeLivenessRaftMuLocked( ba.Timestamp = r.store.Clock().Now() ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeaderFromSpan(span)}) // Call evaluateBatch instead of Send to avoid reacquiring latches. - rec := NewReplicaEvalContext(r, todoSpanSet) + rec := NewReplicaEvalContext( + ctx, r, todoSpanSet, false, /* requiresClosedTSOlderThanStorageSnap */ + ) + defer rec.Release() rw := r.Engine().NewReadOnly(storage.StandardDurability) defer rw.Close() @@ -217,7 +220,10 @@ func (r *Replica) loadSystemConfig(ctx context.Context) (*config.SystemConfigEnt ba.Timestamp = r.store.Clock().Now() ba.Add(&roachpb.ScanRequest{RequestHeader: roachpb.RequestHeaderFromSpan(keys.SystemConfigSpan)}) // Call evaluateBatch instead of Send to avoid reacquiring latches. - rec := NewReplicaEvalContext(r, todoSpanSet) + rec := NewReplicaEvalContext( + ctx, r, todoSpanSet, false, /* requiresClosedTSOlderThanStorageSnap */ + ) + defer rec.Release() rw := r.Engine().NewReadOnly(storage.StandardDurability) defer rw.Close() diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index 762fc3eec052..495185544a4a 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -435,7 +435,7 @@ func (r *Replica) registerWithRangefeedRaftMuLocked( // Check for an initial closed timestamp update immediately to help // initialize the rangefeed's resolved timestamp as soon as possible. - r.handleClosedTimestampUpdateRaftMuLocked(ctx, r.GetClosedTimestamp(ctx)) + r.handleClosedTimestampUpdateRaftMuLocked(ctx, r.GetCurrentClosedTimestamp(ctx)) return p } diff --git a/pkg/kv/kvserver/replica_read.go b/pkg/kv/kvserver/replica_read.go index 9e59fab0b8c9..5879925c5cef 100644 --- a/pkg/kv/kvserver/replica_read.go +++ b/pkg/kv/kvserver/replica_read.go @@ -49,7 +49,8 @@ func (r *Replica) executeReadOnlyBatch( ui := uncertainty.ComputeInterval(&ba.Header, st, r.Clock().MaxOffset()) // Evaluate read-only batch command. - rec := NewReplicaEvalContext(r, g.LatchSpans()) + rec := NewReplicaEvalContext(ctx, r, g.LatchSpans(), ba.RequiresClosedTSOlderThanStorageSnapshot()) + defer rec.Release() // TODO(irfansharif): It's unfortunate that in this read-only code path, // we're stuck with a ReadWriter because of the way evaluateBatch is @@ -60,6 +61,12 @@ func (r *Replica) executeReadOnlyBatch( // may start relying on this, so we assert here. panic("expected consistent iterators") } + // Pin engine state eagerly so that all iterators created over this Reader are + // based off the state of the engine as of this point and are mutually + // consistent. + if err := rw.PinEngineStateForIterators(); err != nil { + return nil, g, roachpb.NewError(err) + } if util.RaceEnabled { rw = spanset.NewReadWriterAt(rw, g.LatchSpans(), ba.Timestamp) } diff --git a/pkg/kv/kvserver/replica_test.go b/pkg/kv/kvserver/replica_test.go index d2daa30d2744..d26af6432d2c 100644 --- a/pkg/kv/kvserver/replica_test.go +++ b/pkg/kv/kvserver/replica_test.go @@ -71,7 +71,7 @@ import ( "github.com/kr/pretty" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - raft "go.etcd.io/etcd/raft/v3" + "go.etcd.io/etcd/raft/v3" "go.etcd.io/etcd/raft/v3/raftpb" "go.etcd.io/etcd/raft/v3/tracker" "golang.org/x/net/trace" @@ -945,7 +945,9 @@ func TestReplicaLease(t *testing.T) { } { if _, err := batcheval.RequestLease(ctx, tc.store.Engine(), batcheval.CommandArgs{ - EvalCtx: NewReplicaEvalContext(tc.repl, allSpans()), + EvalCtx: NewReplicaEvalContext( + ctx, tc.repl, allSpans(), false, /* requiresClosedTSOlderThanStorageSnap */ + ), Args: &roachpb.RequestLeaseRequest{ Lease: lease, }, @@ -4984,7 +4986,9 @@ func TestEndTxnDirectGC(t *testing.T) { var gr roachpb.GetResponse if _, err := batcheval.Get( ctx, tc.engine, batcheval.CommandArgs{ - EvalCtx: NewReplicaEvalContext(tc.repl, allSpans()), + EvalCtx: NewReplicaEvalContext( + ctx, tc.repl, allSpans(), false, /* requiresClosedTSOlderThanStorageSnap */ + ), Args: &roachpb.GetRequest{RequestHeader: roachpb.RequestHeader{ Key: keys.TransactionKey(txn.Key, txn.ID), }}, @@ -5367,7 +5371,8 @@ func TestAbortSpanError(t *testing.T) { t.Fatal(err) } - rec := &SpanSetReplicaEvalContext{tc.repl, *allSpans()} + ec := newEvalContextImpl(ctx, tc.repl, false /* requireClosedTS */) + rec := &SpanSetReplicaEvalContext{ec, *allSpans()} pErr := checkIfTxnAborted(ctx, rec, tc.engine, txn) if _, ok := pErr.GetDetail().(*roachpb.TransactionAbortedError); ok { expected := txn.Clone() @@ -5777,7 +5782,12 @@ func TestResolveIntentPushTxnReplyTxn(t *testing.T) { // return args.PusherTxn. h = roachpb.Header{Timestamp: tc.Clock().Now()} var reply roachpb.PushTxnResponse - if _, err := batcheval.PushTxn(ctx, b, batcheval.CommandArgs{EvalCtx: tc.repl, Stats: &ms, Header: h, Args: &pa}, &reply); err != nil { + ec := newEvalContextImpl( + ctx, + tc.repl, + false, /* requireClosedTS */ + ) + if _, err := batcheval.PushTxn(ctx, b, batcheval.CommandArgs{EvalCtx: ec, Stats: &ms, Header: h, Args: &pa}, &reply); err != nil { t.Fatal(err) } else if reply.Txn != nil { t.Fatalf("expected nil response txn, but got %s", reply.Txn) @@ -8462,10 +8472,11 @@ func TestGCWithoutThreshold(t *testing.T) { rw := spanset.NewBatch(batch, &spans) var resp roachpb.GCResponse - if _, err := batcheval.GC(ctx, rw, batcheval.CommandArgs{ - Args: &gc, - EvalCtx: NewReplicaEvalContext(tc.repl, &spans), + Args: &gc, + EvalCtx: NewReplicaEvalContext( + ctx, tc.repl, &spans, false, /* requiresClosedTSOlderThanStorageSnap */ + ), }, &resp); err != nil { t.Fatal(err) } diff --git a/pkg/kv/kvserver/replica_tscache.go b/pkg/kv/kvserver/replica_tscache.go index 29ee8a8665ab..e71a8b32d8e8 100644 --- a/pkg/kv/kvserver/replica_tscache.go +++ b/pkg/kv/kvserver/replica_tscache.go @@ -594,7 +594,7 @@ func (r *Replica) GetCurrentReadSummary(ctx context.Context) rspb.ReadSummary { // Forward the read summary by the range's closed timestamp, because any // replica could have served reads below this time. We also return the // closed timestamp separately, in case callers want it split out. - closedTS := r.GetClosedTimestamp(ctx) + closedTS := r.GetCurrentClosedTimestamp(ctx) sum.Merge(rspb.FromTimestamp(closedTS)) return sum } diff --git a/pkg/kv/kvserver/replica_write.go b/pkg/kv/kvserver/replica_write.go index 8b9ffc849c38..c12fe8375f53 100644 --- a/pkg/kv/kvserver/replica_write.go +++ b/pkg/kv/kvserver/replica_write.go @@ -424,7 +424,8 @@ func (r *Replica) evaluateWriteBatch( } ms := new(enginepb.MVCCStats) - rec := NewReplicaEvalContext(r, g.LatchSpans()) + rec := NewReplicaEvalContext(ctx, r, g.LatchSpans(), ba.RequiresClosedTSOlderThanStorageSnapshot()) + defer rec.Release() batch, br, res, pErr := r.evaluateWriteBatchWithServersideRefreshes( ctx, idKey, rec, ms, ba, ui, g, nil /* deadline */) return batch, *ms, br, res, pErr @@ -489,7 +490,8 @@ func (r *Replica) evaluate1PC( // Is this relying on the batch being write-only? ui := uncertainty.Interval{} - rec := NewReplicaEvalContext(r, g.LatchSpans()) + rec := NewReplicaEvalContext(ctx, r, g.LatchSpans(), ba.RequiresClosedTSOlderThanStorageSnapshot()) + defer rec.Release() var br *roachpb.BatchResponse var res result.Result var pErr *roachpb.Error diff --git a/pkg/kv/kvserver/replicate_queue.go b/pkg/kv/kvserver/replicate_queue.go index c1165751fba6..45a5809e7220 100644 --- a/pkg/kv/kvserver/replicate_queue.go +++ b/pkg/kv/kvserver/replicate_queue.go @@ -542,8 +542,6 @@ func (rq *replicateQueue) processOneChange( // range descriptor. desc, conf := repl.DescAndSpanConfig() - // Avoid taking action if the range has too many dead replicas to make quorum. - // Consider stores marked suspect as live in order to make this determination. voterReplicas := desc.Replicas().VoterDescriptors() nonVoterReplicas := desc.Replicas().NonVoterDescriptors() liveVoterReplicas, deadVoterReplicas := rq.allocator.storePool.liveAndDeadReplicas( diff --git a/pkg/kv/kvserver/store.go b/pkg/kv/kvserver/store.go index 82793f033400..44dcd2245d74 100644 --- a/pkg/kv/kvserver/store.go +++ b/pkg/kv/kvserver/store.go @@ -2294,7 +2294,7 @@ func (s *Store) startRangefeedUpdater(ctx context.Context) { if r == nil { continue } - r.handleClosedTimestampUpdate(ctx, r.GetClosedTimestamp(ctx)) + r.handleClosedTimestampUpdate(ctx, r.GetCurrentClosedTimestamp(ctx)) } case <-confCh: // Loop around to use the updated timer. @@ -3156,7 +3156,7 @@ func (s *Store) updateReplicationGauges(ctx context.Context) error { if w := metrics.LockTableMetrics.TopKLocksByWaitDuration[0].MaxWaitDurationNanos; w > maxLockWaitDurationNanos { maxLockWaitDurationNanos = w } - mc := rep.GetClosedTimestamp(ctx) + mc := rep.GetCurrentClosedTimestamp(ctx) if minMaxClosedTS.IsEmpty() || mc.Less(minMaxClosedTS) { minMaxClosedTS = mc } diff --git a/pkg/kv/kvserver/store_split.go b/pkg/kv/kvserver/store_split.go index 45327e102047..434fac27cd2c 100644 --- a/pkg/kv/kvserver/store_split.go +++ b/pkg/kv/kvserver/store_split.go @@ -136,15 +136,15 @@ func splitPreApply( // Persist the closed timestamp. // // In order to tolerate a nil initClosedTS input, let's forward to - // r.GetClosedTimestamp(). Generally, initClosedTS is not expected to be nil - // (and is expected to be in advance of r.GetClosedTimestamp() since it's - // coming hot off a Raft command), but let's not rely on the non-nil. Note - // that r.GetClosedTimestamp() does not yet incorporate initClosedTS because - // the split command has not been applied yet. + // r.GetCurrentClosedTimestamp(). Generally, initClosedTS is not expected to + // be nil (and is expected to be in advance of r.GetCurrentClosedTimestamp() + // since it's coming hot off a Raft command), but let's not rely on the + // non-nil. Note that r.GetCurrentClosedTimestamp() does not yet incorporate + // initClosedTS because the split command has not been applied yet. if initClosedTS == nil { initClosedTS = &hlc.Timestamp{} } - initClosedTS.Forward(r.GetClosedTimestamp(ctx)) + initClosedTS.Forward(r.GetCurrentClosedTimestamp(ctx)) if err := rsl.SetClosedTimestamp(ctx, readWriter, initClosedTS); err != nil { log.Fatalf(ctx, "%s", err) } diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 4d3c9cddfa42..6661fbd6b02e 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -77,24 +77,25 @@ func (rc ReadConsistencyType) SupportsBatch(ba BatchRequest) error { type flag int const ( - isAdmin flag = 1 << iota // admin cmds don't go through raft, but run on lease holder - isRead // read-only cmds don't go through raft, but may run on lease holder - isWrite // write cmds go through raft and must be proposed on lease holder - isTxn // txn commands may be part of a transaction - isLocking // locking cmds acquire locks for their transaction - isIntentWrite // intent write cmds leave intents when they succeed - isRange // range commands may span multiple keys - isReverse // reverse commands traverse ranges in descending direction - isAlone // requests which must be alone in a batch - isPrefix // requests which, in a batch, must not be split from the following request - isUnsplittable // range command that must not be split during sending - skipsLeaseCheck // commands which skip the check that the evaluating replica has a valid lease - appliesTSCache // commands which apply the timestamp cache and closed timestamp - updatesTSCache // commands which update the timestamp cache - updatesTSCacheOnErr // commands which make read data available on errors - needsRefresh // commands which require refreshes to avoid serializable retries - canBackpressure // commands which deserve backpressure when a Range grows too large - bypassesReplicaCircuitBreaker // commands which bypass the replica circuit breaker, i.e. opt out of fail-fast + isAdmin flag = 1 << iota // admin cmds don't go through raft, but run on lease holder + isRead // read-only cmds don't go through raft, but may run on lease holder + isWrite // write cmds go through raft and must be proposed on lease holder + isTxn // txn commands may be part of a transaction + isLocking // locking cmds acquire locks for their transaction + isIntentWrite // intent write cmds leave intents when they succeed + isRange // range commands may span multiple keys + isReverse // reverse commands traverse ranges in descending direction + isAlone // requests which must be alone in a batch + isPrefix // requests which, in a batch, must not be split from the following request + isUnsplittable // range command that must not be split during sending + skipsLeaseCheck // commands which skip the check that the evaluating replica has a valid lease + appliesTSCache // commands which apply the timestamp cache and closed timestamp + updatesTSCache // commands which update the timestamp cache + updatesTSCacheOnErr // commands which make read data available on errors + needsRefresh // commands which require refreshes to avoid serializable retries + canBackpressure // commands which deserve backpressure when a Range grows too large + bypassesReplicaCircuitBreaker // commands which bypass the replica circuit breaker, i.e. opt out of fail-fast + requiresClosedTSOlderThanStorageSnapshot // commands which read a replica's closed timestamp that is older than the state of the storage engine ) // flagDependencies specifies flag dependencies, asserted by TestFlagCombinations. @@ -1404,9 +1405,11 @@ func (r *RefreshRangeRequest) flags() flag { return isRead | isTxn | isRange | updatesTSCache } -func (*SubsumeRequest) flags() flag { return isRead | isAlone | updatesTSCache } -func (*RangeStatsRequest) flags() flag { return isRead } -func (*QueryResolvedTimestampRequest) flags() flag { return isRead | isRange } +func (*SubsumeRequest) flags() flag { return isRead | isAlone | updatesTSCache } +func (*RangeStatsRequest) flags() flag { return isRead } +func (*QueryResolvedTimestampRequest) flags() flag { + return isRead | isRange | requiresClosedTSOlderThanStorageSnapshot +} func (*ScanInterleavedIntentsRequest) flags() flag { return isRead | isRange } func (*BarrierRequest) flags() flag { return isWrite | isRange } diff --git a/pkg/roachpb/batch.go b/pkg/roachpb/batch.go index 20d65e9d4049..de58e1cef531 100644 --- a/pkg/roachpb/batch.go +++ b/pkg/roachpb/batch.go @@ -275,6 +275,15 @@ func (ba *BatchRequest) Require1PC() bool { return etArg.Require1PC } +// RequiresClosedTSOlderThanStorageSnapshot returns true if the batch contains a +// request that needs to read a replica's closed timestamp that is older than +// the state of the storage snapshot the request is evaluating over. +// +// NB: This is only used by QueryResolvedTimestampRequest at the moment. +func (ba *BatchRequest) RequiresClosedTSOlderThanStorageSnapshot() bool { + return ba.hasFlag(requiresClosedTSOlderThanStorageSnapshot) +} + // IsSingleAbortTxnRequest returns true iff the batch contains a single request, // and that request is an EndTxnRequest(commit=false). func (ba *BatchRequest) IsSingleAbortTxnRequest() bool { diff --git a/pkg/server/admin.go b/pkg/server/admin.go index 41a9d602aa85..f45132055fab 100644 --- a/pkg/server/admin.go +++ b/pkg/server/admin.go @@ -2389,7 +2389,7 @@ func (s *adminServer) decommissionStatusHelper( // numReplicaReport is the number of replicas reported for each node. var replicasToReport map[roachpb.NodeID][]*serverpb.DecommissionStatusResponse_Replica if numReplicaReport > 0 { - log.Ops.Warning(ctx, "possible decommission stall detected; reporting decommissioning replicas") + log.Ops.Warning(ctx, "possible decommission stall detected") replicasToReport = make(map[roachpb.NodeID][]*serverpb.DecommissionStatusResponse_Replica) } diff --git a/pkg/sql/colexec/BUILD.bazel b/pkg/sql/colexec/BUILD.bazel index 068362c038f5..551b4e627177 100644 --- a/pkg/sql/colexec/BUILD.bazel +++ b/pkg/sql/colexec/BUILD.bazel @@ -1,6 +1,7 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") load(":COLEXEC.bzl", "gen_sort_partitioner_rule") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexec", @@ -102,7 +103,6 @@ go_test( "count_test.go", "crossjoiner_test.go", "default_agg_test.go", - "dep_test.go", "distinct_test.go", "external_distinct_test.go", "external_hash_aggregator_test.go", @@ -168,7 +168,6 @@ go_test( "//pkg/sql/sem/tree", "//pkg/sql/types", "//pkg/testutils", - "//pkg/testutils/buildutil", "//pkg/testutils/colcontainerutils", "//pkg/testutils/distsqlutils", "//pkg/testutils/skip", @@ -220,3 +219,12 @@ gen_sort_partitioner_rule( name = "gen-sort-partitioner", target = "sort_partitioner.eg.go", ) + +disallowed_imports_test( + "colexec", + [ + "//pkg/sql/colflow", + "//pkg/sql/rowexec", + "//pkg/sql/rowflow", + ], +) diff --git a/pkg/sql/colexec/aggregators_test.go b/pkg/sql/colexec/aggregators_test.go index 1aba308e1d9e..b9289b874ae5 100644 --- a/pkg/sql/colexec/aggregators_test.go +++ b/pkg/sql/colexec/aggregators_test.go @@ -1172,25 +1172,17 @@ func BenchmarkAggregator(b *testing.B) { numRows = []int{32, 32 * coldata.BatchSize()} groupSizes = []int{1, coldata.BatchSize()} } - for _, aggFn := range []execinfrapb.AggregatorSpec_Func{ - // We choose any_not_null aggregate function because it is the simplest - // possible and, thus, its Compute function call will have the least - // impact when benchmarking the aggregator logic. - execinfrapb.AnyNotNull, - // min aggregate function has been used before transitioning to - // any_not_null in 22.1 cycle. It is kept so that we could use it for - // comparison of 22.1 against 21.2. - // TODO(yuzefovich): use only any_not_null in 22.2 (#75106). - execinfrapb.Min, - } { - for _, agg := range aggTypes { - for _, numInputRows := range numRows { - for _, groupSize := range groupSizes { - benchmarkAggregateFunction( - b, agg, aggFn, []*types.T{types.Int}, 1, /* numGroupCol */ - groupSize, 0 /* distinctProb */, numInputRows, - 0 /* chunkSize */, 0 /* limit */) - } + // We choose any_not_null aggregate function because it is the simplest + // possible and, thus, its Compute function call will have the least impact + // when benchmarking the aggregator logic. + aggFn := execinfrapb.AnyNotNull + for _, agg := range aggTypes { + for _, numInputRows := range numRows { + for _, groupSize := range groupSizes { + benchmarkAggregateFunction( + b, agg, aggFn, []*types.T{types.Int}, 1, /* numGroupCol */ + groupSize, 0 /* distinctProb */, numInputRows, + 0 /* chunkSize */, 0 /* limit */) } } } diff --git a/pkg/sql/colexec/colexecagg/BUILD.bazel b/pkg/sql/colexec/colexecagg/BUILD.bazel index bfa781c5da3c..10ee2571d2e4 100644 --- a/pkg/sql/colexec/colexecagg/BUILD.bazel +++ b/pkg/sql/colexec/colexecagg/BUILD.bazel @@ -1,5 +1,6 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("@io_bazel_rules_go//go:def.bzl", "go_library") load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexecagg", @@ -32,14 +33,6 @@ go_library( ], ) -go_test( - name = "colexecagg_test", - srcs = ["dep_test.go"], - embed = [":colexecagg"], - tags = ["no-remote"], - deps = ["//pkg/testutils/buildutil"], -) - # Map between target name and relevant template. targets = [ ("hash_any_not_null_agg.eg.go", "any_not_null_agg_tmpl.go"), @@ -77,3 +70,16 @@ eg_go_filegroup( # Define gen rules for individual eg.go files. gen_eg_go_rules(targets) + +disallowed_imports_test( + "colexecagg", + [ + "//pkg/sql/colexec", + "//pkg/sql/colexec/colexecbase", + "//pkg/sql/colexec/colexechash", + "//pkg/sql/colexec/colexecjoin", + "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecsel", + "//pkg/sql/colexec/colexecwindow", + ], +) diff --git a/pkg/sql/colexec/colexecagg/dep_test.go b/pkg/sql/colexec/colexecagg/dep_test.go deleted file mode 100644 index fec1005aa19e..000000000000 --- a/pkg/sql/colexec/colexecagg/dep_test.go +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexecagg - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow", - }, nil, - ) -} diff --git a/pkg/sql/colexec/colexecbase/BUILD.bazel b/pkg/sql/colexec/colexecbase/BUILD.bazel index e6fb834ad91c..1139b32b27ec 100644 --- a/pkg/sql/colexec/colexecbase/BUILD.bazel +++ b/pkg/sql/colexec/colexecbase/BUILD.bazel @@ -1,5 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexecbase", @@ -45,7 +46,6 @@ go_test( srcs = [ "cast_test.go", "const_test.go", - "dep_test.go", "inject_setup_test.go", "main_test.go", "ordinality_test.go", @@ -72,7 +72,6 @@ go_test( "//pkg/sql/randgen", "//pkg/sql/sem/tree", "//pkg/sql/types", - "//pkg/testutils/buildutil", "//pkg/testutils/skip", "//pkg/util/leaktest", "//pkg/util/log", @@ -101,3 +100,16 @@ eg_go_filegroup( # Define gen rules for individual eg.go files. gen_eg_go_rules(targets) + +disallowed_imports_test( + "colexecbase", + [ + "//pkg/sql/colexec", + "//pkg/sql/colexec/colexecagg", + "//pkg/sql/colexec/colexechash", + "//pkg/sql/colexec/colexecjoin", + "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecsel", + "//pkg/sql/colexec/colexecwindow", + ], +) diff --git a/pkg/sql/colexec/colexecbase/dep_test.go b/pkg/sql/colexec/colexecbase/dep_test.go deleted file mode 100644 index b753badd58c7..000000000000 --- a/pkg/sql/colexec/colexecbase/dep_test.go +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexecbase_test - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel", - }, nil, - ) -} diff --git a/pkg/sql/colexec/colexechash/BUILD.bazel b/pkg/sql/colexec/colexechash/BUILD.bazel index 7ce372d3b709..32702cb6ea4c 100644 --- a/pkg/sql/colexec/colexechash/BUILD.bazel +++ b/pkg/sql/colexec/colexechash/BUILD.bazel @@ -1,5 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexechash", @@ -32,7 +33,6 @@ go_library( go_test( name = "colexechash_test", srcs = [ - "dep_test.go", "hash_test.go", "hash_utils_test.go", "main_test.go", @@ -48,7 +48,6 @@ go_test( "//pkg/sql/execinfra", "//pkg/sql/sem/tree", "//pkg/sql/types", - "//pkg/testutils/buildutil", "//pkg/util/leaktest", "//pkg/util/log", "//pkg/util/mon", @@ -72,3 +71,16 @@ eg_go_filegroup( # Define gen rules for individual eg.go files. gen_eg_go_rules(targets) + +disallowed_imports_test( + "colexechash", + [ + "//pkg/sql/colexec", + "//pkg/sql/colexec/colexecagg", + "//pkg/sql/colexec/colexecbase", + "//pkg/sql/colexec/colexecjoin", + "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecsel", + "//pkg/sql/colexec/colexecwindow", + ], +) diff --git a/pkg/sql/colexec/colexechash/dep_test.go b/pkg/sql/colexec/colexechash/dep_test.go deleted file mode 100644 index b561de11f487..000000000000 --- a/pkg/sql/colexec/colexechash/dep_test.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexechash - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow", - }, nil, - ) -} diff --git a/pkg/sql/colexec/colexecjoin/BUILD.bazel b/pkg/sql/colexec/colexecjoin/BUILD.bazel index f54185729a69..1641e6073b1a 100644 --- a/pkg/sql/colexec/colexecjoin/BUILD.bazel +++ b/pkg/sql/colexec/colexecjoin/BUILD.bazel @@ -1,5 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexecjoin", @@ -43,7 +44,6 @@ go_library( go_test( name = "colexecjoin_test", srcs = [ - "dep_test.go", "main_test.go", "mergejoiner_test.go", ], @@ -63,7 +63,6 @@ go_test( "//pkg/sql/execinfrapb", "//pkg/sql/sem/tree", "//pkg/sql/types", - "//pkg/testutils/buildutil", "//pkg/testutils/colcontainerutils", "//pkg/testutils/skip", "//pkg/util/leaktest", @@ -99,3 +98,14 @@ eg_go_filegroup( # Define gen rules for individual eg.go files. gen_eg_go_rules(targets) + +disallowed_imports_test( + "colexecjoin", + [ + "//pkg/sql/colexec", + "//pkg/sql/colexec/colexecagg", + "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecsel", + "//pkg/sql/colexec/colexecwindow", + ], +) diff --git a/pkg/sql/colexec/colexecjoin/dep_test.go b/pkg/sql/colexec/colexecjoin/dep_test.go deleted file mode 100644 index 53824a2573c0..000000000000 --- a/pkg/sql/colexec/colexecjoin/dep_test.go +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexecjoin_test - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow", - }, nil, - ) -} diff --git a/pkg/sql/colexec/colexecproj/BUILD.bazel b/pkg/sql/colexec/colexecproj/BUILD.bazel index 693a14f95264..7a27207cc880 100644 --- a/pkg/sql/colexec/colexecproj/BUILD.bazel +++ b/pkg/sql/colexec/colexecproj/BUILD.bazel @@ -1,5 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexecproj", @@ -39,7 +40,6 @@ go_test( name = "colexecproj_test", srcs = [ "default_cmp_op_test.go", - "dep_test.go", "inject_setup_test.go", "main_test.go", "projection_ops_test.go", @@ -64,7 +64,6 @@ go_test( "//pkg/sql/sem/tree/treebin", "//pkg/sql/sem/tree/treecmp", "//pkg/sql/types", - "//pkg/testutils/buildutil", "//pkg/testutils/skip", "//pkg/util/leaktest", "//pkg/util/log", @@ -92,3 +91,15 @@ eg_go_filegroup( # Define gen rules for individual eg.go files. gen_eg_go_rules(targets) + +disallowed_imports_test( + "colexecproj", + [ + "//pkg/sql/colexec", + "//pkg/sql/colexec/colexecagg", + "//pkg/sql/colexec/colexechash", + "//pkg/sql/colexec/colexecjoin", + "//pkg/sql/colexec/colexecsel", + "//pkg/sql/colexec/colexecwindow", + ], +) diff --git a/pkg/sql/colexec/colexecproj/dep_test.go b/pkg/sql/colexec/colexecproj/dep_test.go deleted file mode 100644 index 9db603d168ed..000000000000 --- a/pkg/sql/colexec/colexecproj/dep_test.go +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexecproj - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow", - }, nil, - ) -} diff --git a/pkg/sql/colexec/colexecsel/BUILD.bazel b/pkg/sql/colexec/colexecsel/BUILD.bazel index a0d8a35171fa..e5fdc3d904b9 100644 --- a/pkg/sql/colexec/colexecsel/BUILD.bazel +++ b/pkg/sql/colexec/colexecsel/BUILD.bazel @@ -1,5 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexecsel", @@ -32,7 +33,6 @@ go_library( go_test( name = "colexecsel_test", srcs = [ - "dep_test.go", "like_ops_test.go", "main_test.go", "selection_ops_test.go", @@ -52,7 +52,6 @@ go_test( "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treecmp", "//pkg/sql/types", - "//pkg/testutils/buildutil", "//pkg/testutils/skip", "//pkg/util/leaktest", "//pkg/util/log", @@ -77,3 +76,16 @@ eg_go_filegroup( # Define gen rules for individual eg.go files. gen_eg_go_rules(targets) + +disallowed_imports_test( + "colexecsel", + [ + "//pkg/sql/colexec", + "//pkg/sql/colexec/colexecagg", + "//pkg/sql/colexec/colexecbase", + "//pkg/sql/colexec/colexechash", + "//pkg/sql/colexec/colexecjoin", + "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecwindow", + ], +) diff --git a/pkg/sql/colexec/colexecsel/dep_test.go b/pkg/sql/colexec/colexecsel/dep_test.go deleted file mode 100644 index 704747f55b23..000000000000 --- a/pkg/sql/colexec/colexecsel/dep_test.go +++ /dev/null @@ -1,32 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexecsel - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecbase", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow", - }, nil, - ) -} diff --git a/pkg/sql/colexec/colexecspan/BUILD.bazel b/pkg/sql/colexec/colexecspan/BUILD.bazel index 7c206a39e00f..c54a22734f4c 100644 --- a/pkg/sql/colexec/colexecspan/BUILD.bazel +++ b/pkg/sql/colexec/colexecspan/BUILD.bazel @@ -1,5 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexecspan", @@ -35,7 +36,6 @@ go_library( go_test( name = "colexecspan_test", srcs = [ - "dep_test.go", "main_test.go", "span_assembler_test.go", ], @@ -63,7 +63,6 @@ go_test( "//pkg/sql/sem/tree", "//pkg/sql/span", "//pkg/sql/types", - "//pkg/testutils/buildutil", "//pkg/testutils/skip", "//pkg/util", "//pkg/util/leaktest", @@ -85,3 +84,17 @@ eg_go_filegroup( # Define gen rules for individual eg.go files. gen_eg_go_rules(targets) + +disallowed_imports_test( + "colexecspan", + [ + "//pkg/sql/colexec", + "//pkg/sql/colexec/colexecagg", + "//pkg/sql/colexec/colexecbase", + "//pkg/sql/colexec/colexechash", + "//pkg/sql/colexec/colexecjoin", + "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecsel", + "//pkg/sql/colexec/colexecwindow", + ], +) diff --git a/pkg/sql/colexec/colexecspan/dep_test.go b/pkg/sql/colexec/colexecspan/dep_test.go deleted file mode 100644 index 4d979668bbf1..000000000000 --- a/pkg/sql/colexec/colexecspan/dep_test.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexecspan - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecspan", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel", - }, nil, - ) -} diff --git a/pkg/sql/colexec/colexecwindow/BUILD.bazel b/pkg/sql/colexec/colexecwindow/BUILD.bazel index 080330c21b00..8ce456a0ac65 100644 --- a/pkg/sql/colexec/colexecwindow/BUILD.bazel +++ b/pkg/sql/colexec/colexecwindow/BUILD.bazel @@ -1,5 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//pkg/sql/colexecop:EXECGEN.bzl", "eg_go_filegroup", "gen_eg_go_rules") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexecwindow", @@ -46,7 +47,6 @@ go_library( go_test( name = "colexecwindow_test", srcs = [ - "dep_test.go", "inject_setup_test.go", "main_test.go", "min_max_queue_test.go", @@ -75,7 +75,6 @@ go_test( "//pkg/sql/sem/tree", "//pkg/sql/sem/tree/treewindow", "//pkg/sql/types", - "//pkg/testutils/buildutil", "//pkg/testutils/colcontainerutils", "//pkg/testutils/skip", "//pkg/util/encoding", @@ -116,3 +115,14 @@ eg_go_filegroup( # Define gen rules for individual eg.go files. gen_eg_go_rules(targets) + +disallowed_imports_test( + "colexecwindow", + [ + "//pkg/sql/colexec", + "//pkg/sql/colexec/colexechash", + "//pkg/sql/colexec/colexecjoin", + "//pkg/sql/colexec/colexecproj", + "//pkg/sql/colexec/colexecsel", + ], +) diff --git a/pkg/sql/colexec/colexecwindow/dep_test.go b/pkg/sql/colexec/colexecwindow/dep_test.go deleted file mode 100644 index 17e40ce5e6d4..000000000000 --- a/pkg/sql/colexec/colexecwindow/dep_test.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2021 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexecwindow - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecwindow", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecagg", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexechash", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecjoin", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecproj", - "github.com/cockroachdb/cockroach/pkg/sql/colexec/colexecsel", - }, nil, - ) -} diff --git a/pkg/sql/colexec/dep_test.go b/pkg/sql/colexec/dep_test.go deleted file mode 100644 index b0632bbdef23..000000000000 --- a/pkg/sql/colexec/dep_test.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexec - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" - "github.com/cockroachdb/cockroach/pkg/util/log" -) - -func TestNoLinkForbidden(t *testing.T) { - defer leaktest.AfterTest(t)() - defer log.Scope(t).Close(t) - - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colflow", - "github.com/cockroachdb/cockroach/pkg/sql/rowexec", - "github.com/cockroachdb/cockroach/pkg/sql/rowflow", - }, nil, - ) -} diff --git a/pkg/sql/colexec/distinct_test.go b/pkg/sql/colexec/distinct_test.go index c8b11219ab6c..147cdf91ac7a 100644 --- a/pkg/sql/colexec/distinct_test.go +++ b/pkg/sql/colexec/distinct_test.go @@ -637,11 +637,8 @@ func BenchmarkDistinct(b *testing.B) { }, } unorderedShuffled := "UnorderedShuffled" - // TODO(yuzefovich): remove Unordered in 22.2 without renaming - // unorderedShuffled (#75106). - distinctNames := []string{"Unordered", "PartiallyOrdered", "Ordered", unorderedShuffled} - distinctConstructors = append(distinctConstructors, distinctConstructors[0]) - orderedColsFraction := []float64{0, 0.5, 1.0, 0} + distinctNames := []string{unorderedShuffled, "PartiallyOrdered", "Ordered"} + orderedColsFraction := []float64{0, 0.5, 1.0} for distinctIdx, distinctConstructor := range distinctConstructors { runDistinctBenchmarks( ctx, diff --git a/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel b/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel index d86f1f19dbc6..3d36c3eee257 100644 --- a/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel +++ b/pkg/sql/colexec/execgen/cmd/execgen/BUILD.bazel @@ -1,4 +1,5 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library", "go_test") +load("@io_bazel_rules_go//go:def.bzl", "go_binary", "go_library") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "execgen_lib", @@ -85,9 +86,12 @@ go_binary( visibility = ["//visibility:public"], ) -go_test( - name = "execgen_test", - srcs = ["dep_test.go"], - embed = [":execgen_lib"], - deps = ["//pkg/testutils/buildutil"], +disallowed_imports_test( + "execgen", + [ + "//pkg/roachpb", + "//pkg/sql/catalog", + "//pkg/sql/execinfrapb", + "//pkg/sql/sem/tree", + ], ) diff --git a/pkg/sql/colexec/execgen/cmd/execgen/dep_test.go b/pkg/sql/colexec/execgen/cmd/execgen/dep_test.go deleted file mode 100644 index fcd377d8d483..000000000000 --- a/pkg/sql/colexec/execgen/cmd/execgen/dep_test.go +++ /dev/null @@ -1,29 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package main - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexec/execgen/cmd/execgen", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/roachpb", - "github.com/cockroachdb/cockroach/pkg/sql/catalog", - "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb", - "github.com/cockroachdb/cockroach/pkg/sql/tree", - }, nil, - ) -} diff --git a/pkg/sql/colexec/external_distinct_test.go b/pkg/sql/colexec/external_distinct_test.go index 0b34d35da753..2c772ef7c030 100644 --- a/pkg/sql/colexec/external_distinct_test.go +++ b/pkg/sql/colexec/external_distinct_test.go @@ -281,48 +281,41 @@ func BenchmarkExternalDistinct(b *testing.B) { var monitorRegistry colexecargs.MonitorRegistry defer monitorRegistry.Close(ctx) - // TODO(yuzefovich): remove shuffleInput == false case in 22.2 without - // changing the name of the benchmark (#75106). - for _, shuffleInput := range []bool{false, true} { - for _, spillForced := range []bool{false, true} { - for _, maintainOrdering := range []bool{false, true} { - if !spillForced && maintainOrdering { - // The in-memory unordered distinct maintains the input ordering - // by design, so it's not an interesting case to test it with - // both options for 'maintainOrdering' parameter, and we skip - // one. - continue - } - flowCtx.Cfg.TestingKnobs.ForceDiskSpill = spillForced - name := fmt.Sprintf("spilled=%t/ordering=%t", spillForced, maintainOrdering) - if shuffleInput { - name = name + "/shuffled" - } - runDistinctBenchmarks( - ctx, - b, - func(allocator *colmem.Allocator, input colexecop.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecop.Operator, error) { - var outputOrdering execinfrapb.Ordering - if maintainOrdering { - outputOrdering = convertDistinctColsToOrdering(distinctCols) - } - op, _, err := createExternalDistinct( - ctx, flowCtx, []colexecop.Operator{input}, typs, - distinctCols, false /* nullsAreDistinct */, "", /* errorOnDup */ - outputOrdering, queueCfg, &colexecop.TestingSemaphore{}, - nil /* spillingCallbackFn */, 0, /* numForcedRepartitions */ - &monitorRegistry, - ) - return op, err - }, - func(nCols int) int { - return 0 - }, - name, - true, /* isExternal */ - shuffleInput, - ) + for _, spillForced := range []bool{false, true} { + for _, maintainOrdering := range []bool{false, true} { + if !spillForced && maintainOrdering { + // The in-memory unordered distinct maintains the input ordering + // by design, so it's not an interesting case to test it with + // both options for 'maintainOrdering' parameter, and we skip + // one. + continue } + flowCtx.Cfg.TestingKnobs.ForceDiskSpill = spillForced + name := fmt.Sprintf("spilled=%t/ordering=%t/shuffled", spillForced, maintainOrdering) + runDistinctBenchmarks( + ctx, + b, + func(allocator *colmem.Allocator, input colexecop.Operator, distinctCols []uint32, numOrderedCols int, typs []*types.T) (colexecop.Operator, error) { + var outputOrdering execinfrapb.Ordering + if maintainOrdering { + outputOrdering = convertDistinctColsToOrdering(distinctCols) + } + op, _, err := createExternalDistinct( + ctx, flowCtx, []colexecop.Operator{input}, typs, + distinctCols, false /* nullsAreDistinct */, "", /* errorOnDup */ + outputOrdering, queueCfg, &colexecop.TestingSemaphore{}, + nil /* spillingCallbackFn */, 0, /* numForcedRepartitions */ + &monitorRegistry, + ) + return op, err + }, + func(nCols int) int { + return 0 + }, + name, + true, /* isExternal */ + true, /* shuffleInput */ + ) } } } diff --git a/pkg/sql/colexec/external_hash_aggregator_test.go b/pkg/sql/colexec/external_hash_aggregator_test.go index f6967aff469b..e984af91dabe 100644 --- a/pkg/sql/colexec/external_hash_aggregator_test.go +++ b/pkg/sql/colexec/external_hash_aggregator_test.go @@ -184,42 +184,34 @@ func BenchmarkExternalHashAggregator(b *testing.B) { numRows = []int{64 * coldata.BatchSize()} groupSizes = []int{1, coldata.BatchSize()} } - for _, aggFn := range []execinfrapb.AggregatorSpec_Func{ - // We choose any_not_null aggregate function because it is the simplest - // possible and, thus, its Compute function call will have the least - // impact when benchmarking the aggregator logic. - execinfrapb.AnyNotNull, - // min aggregate function has been used before transitioning to - // any_not_null in 22.1 cycle. It is kept so that we could use it for - // comparison of 22.1 against 21.2. - // TODO(yuzefovich): use only any_not_null in 22.2 (#75106). - execinfrapb.Min, - } { - for _, spillForced := range []bool{false, true} { - flowCtx.Cfg.TestingKnobs.ForceDiskSpill = spillForced - for _, numInputRows := range numRows { - for _, groupSize := range groupSizes { - benchmarkAggregateFunction( - b, aggType{ - new: func(args *colexecagg.NewAggregatorArgs) colexecop.ResettableOperator { - op, _, err := createExternalHashAggregator( - ctx, flowCtx, args, queueCfg, &colexecop.TestingSemaphore{}, - 0 /* numForcedRepartitions */, &monitorRegistry, - ) - require.NoError(b, err) - // The hash-based partitioner is not a - // ResettableOperator, so in order to not change the - // signatures of the aggregator constructors, we - // wrap it with a noop operator. It is ok for the - // purposes of this benchmark. - return colexecop.NewNoop(op) - }, - name: fmt.Sprintf("spilled=%t", spillForced), - order: unordered, + // We choose any_not_null aggregate function because it is the simplest + // possible and, thus, its Compute function call will have the least impact + // when benchmarking the aggregator logic. + aggFn := execinfrapb.AnyNotNull + for _, spillForced := range []bool{false, true} { + flowCtx.Cfg.TestingKnobs.ForceDiskSpill = spillForced + for _, numInputRows := range numRows { + for _, groupSize := range groupSizes { + benchmarkAggregateFunction( + b, aggType{ + new: func(args *colexecagg.NewAggregatorArgs) colexecop.ResettableOperator { + op, _, err := createExternalHashAggregator( + ctx, flowCtx, args, queueCfg, &colexecop.TestingSemaphore{}, + 0 /* numForcedRepartitions */, &monitorRegistry, + ) + require.NoError(b, err) + // The hash-based partitioner is not a + // ResettableOperator, so in order to not change the + // signatures of the aggregator constructors, we + // wrap it with a noop operator. It is ok for the + // purposes of this benchmark. + return colexecop.NewNoop(op) }, - aggFn, []*types.T{types.Int}, 1 /* numGroupCol */, groupSize, - 0 /* distinctProb */, numInputRows, 0 /* chunkSize */, 0 /* limit */) - } + name: fmt.Sprintf("spilled=%t", spillForced), + order: unordered, + }, + aggFn, []*types.T{types.Int}, 1 /* numGroupCol */, groupSize, + 0 /* distinctProb */, numInputRows, 0 /* chunkSize */, 0 /* limit */) } } } diff --git a/pkg/sql/colexec/hash_aggregator_test.go b/pkg/sql/colexec/hash_aggregator_test.go index e94561f9d8b8..facab850cd4c 100644 --- a/pkg/sql/colexec/hash_aggregator_test.go +++ b/pkg/sql/colexec/hash_aggregator_test.go @@ -467,48 +467,40 @@ func BenchmarkHashAggregatorInputTuplesTracking(b *testing.B) { groupSizes = []int{1, coldata.BatchSize()} } var memAccounts []*mon.BoundAccount - for _, aggFn := range []execinfrapb.AggregatorSpec_Func{ - // We choose any_not_null aggregate function because it is the simplest - // possible and, thus, its Compute function call will have the least - // impact when benchmarking the aggregator logic. - execinfrapb.AnyNotNull, - // min aggregate function has been used before transitioning to - // any_not_null in 22.1 cycle. It is kept so that we could use it for - // comparison of 22.1 against 21.2. - // TODO(yuzefovich): use only any_not_null in 22.2 (#75106). - execinfrapb.Min, - } { - for _, numInputRows := range numRows { - for _, groupSize := range groupSizes { - for _, agg := range []aggType{ - { - new: func(args *colexecagg.NewAggregatorArgs) colexecop.ResettableOperator { - return NewHashAggregator(args, nil /* newSpillingQueueArgs */, testAllocator, math.MaxInt64) - }, - name: "tracking=false", - order: unordered, + // We choose any_not_null aggregate function because it is the simplest + // possible and, thus, its Compute function call will have the least + // impact when benchmarking the aggregator logic. + aggFn := execinfrapb.AnyNotNull + for _, numInputRows := range numRows { + for _, groupSize := range groupSizes { + for _, agg := range []aggType{ + { + new: func(args *colexecagg.NewAggregatorArgs) colexecop.ResettableOperator { + return NewHashAggregator(args, nil /* newSpillingQueueArgs */, testAllocator, math.MaxInt64) }, - { - new: func(args *colexecagg.NewAggregatorArgs) colexecop.ResettableOperator { - spillingQueueMemAcc := testMemMonitor.MakeBoundAccount() - memAccounts = append(memAccounts, &spillingQueueMemAcc) - return NewHashAggregator(args, &colexecutils.NewSpillingQueueArgs{ - UnlimitedAllocator: colmem.NewAllocator(ctx, &spillingQueueMemAcc, testColumnFactory), - Types: args.InputTypes, - MemoryLimit: execinfra.DefaultMemoryLimit, - DiskQueueCfg: queueCfg, - FDSemaphore: &colexecop.TestingSemaphore{}, - DiskAcc: testDiskAcc, - }, testAllocator, math.MaxInt64) - }, - name: "tracking=true", - order: unordered, + name: "tracking=false", + order: unordered, + }, + { + new: func(args *colexecagg.NewAggregatorArgs) colexecop.ResettableOperator { + spillingQueueMemAcc := testMemMonitor.MakeBoundAccount() + memAccounts = append(memAccounts, &spillingQueueMemAcc) + return NewHashAggregator(args, &colexecutils.NewSpillingQueueArgs{ + UnlimitedAllocator: colmem.NewAllocator(ctx, &spillingQueueMemAcc, testColumnFactory), + Types: args.InputTypes, + MemoryLimit: execinfra.DefaultMemoryLimit, + DiskQueueCfg: queueCfg, + FDSemaphore: &colexecop.TestingSemaphore{}, + DiskAcc: testDiskAcc, + }, testAllocator, math.MaxInt64) }, - } { - benchmarkAggregateFunction( - b, agg, aggFn, []*types.T{types.Int}, 1 /* numGroupCol */, groupSize, - 0 /* distinctProb */, numInputRows, 0 /* chunkSize */, 0 /* limit */) - } + name: "tracking=true", + order: unordered, + }, + } { + benchmarkAggregateFunction( + b, agg, aggFn, []*types.T{types.Int}, 1 /* numGroupCol */, groupSize, + 0 /* distinctProb */, numInputRows, 0 /* chunkSize */, 0 /* limit */) } } } @@ -556,46 +548,38 @@ func BenchmarkHashAggregatorPartialOrder(b *testing.B) { DiskAcc: testDiskAcc, }, testAllocator, math.MaxInt64) } - for _, aggFn := range []execinfrapb.AggregatorSpec_Func{ - // We choose any_not_null aggregate function because it is the simplest - // possible and, thus, its Compute function call will have the least - // impact when benchmarking the aggregator logic. - execinfrapb.AnyNotNull, - // min aggregate function has been used before transitioning to - // any_not_null in 22.1 cycle. It is kept so that we could use it for - // comparison of 22.1 against 21.2. - // TODO(yuzefovich): use only any_not_null in 22.2 (#75106). - execinfrapb.Min, - } { - for _, numInputRows := range numRows { - for _, limit := range limits { - if limit > numInputRows { - continue - } - for _, groupSize := range groupSizes { - for _, chunkSize := range chunkSizes { - if groupSize > chunkSize || chunkSize > numInputRows { + // We choose any_not_null aggregate function because it is the simplest + // possible and, thus, its Compute function call will have the least impact + // when benchmarking the aggregator logic. + aggFn := execinfrapb.AnyNotNull + for _, numInputRows := range numRows { + for _, limit := range limits { + if limit > numInputRows { + continue + } + for _, groupSize := range groupSizes { + for _, chunkSize := range chunkSizes { + if groupSize > chunkSize || chunkSize > numInputRows { + continue + } + for _, agg := range []aggType{ + { + new: f, + name: fmt.Sprintf("hash-unordered/limit=%d/chunkSize=%d", limit, chunkSize), + order: unordered, + }, + { + new: f, + name: fmt.Sprintf("hash-partial-order/limit=%d/chunkSize=%d", limit, chunkSize), + order: partial, + }, + } { + if agg.order == unordered && chunkSize != chunkSizes[0] { + // Chunk size isn't a factor for the unordered hash aggregator, + // so we can skip all but one case. continue } - for _, agg := range []aggType{ - { - new: f, - name: fmt.Sprintf("hash-unordered/limit=%d/chunkSize=%d", limit, chunkSize), - order: unordered, - }, - { - new: f, - name: fmt.Sprintf("hash-partial-order/limit=%d/chunkSize=%d", limit, chunkSize), - order: partial, - }, - } { - if agg.order == unordered && chunkSize != chunkSizes[0] { - // Chunk size isn't a factor for the unordered hash aggregator, - // so we can skip all but one case. - continue - } - benchmarkAggregateFunction(b, agg, aggFn, []*types.T{types.Int}, 2, groupSize, 0, numInputRows, chunkSize, limit) - } + benchmarkAggregateFunction(b, agg, aggFn, []*types.T{types.Int}, 2, groupSize, 0, numInputRows, chunkSize, limit) } } } diff --git a/pkg/sql/colexecop/BUILD.bazel b/pkg/sql/colexecop/BUILD.bazel index 784641baa6db..28c7143272ac 100644 --- a/pkg/sql/colexecop/BUILD.bazel +++ b/pkg/sql/colexecop/BUILD.bazel @@ -1,4 +1,5 @@ -load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("@io_bazel_rules_go//go:def.bzl", "go_library") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colexecop", @@ -21,14 +22,13 @@ go_library( ], ) -go_test( - name = "colexecop_test", - size = "small", - srcs = ["dep_test.go"], - embed = [":colexecop"], - tags = ["no-remote"], - deps = [ - "//pkg/testutils/buildutil", - "//pkg/util/leaktest", +disallowed_imports_test( + "colexecop", + [ + "//pkg/sql/colcontainer", + "//pkg/sql/colexec", + "//pkg/sql/colflow", + "//pkg/sql/rowexec", + "//pkg/sql/rowflow", ], ) diff --git a/pkg/sql/colexecop/dep_test.go b/pkg/sql/colexecop/dep_test.go deleted file mode 100644 index c134353e58d7..000000000000 --- a/pkg/sql/colexecop/dep_test.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2020 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colexecop - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" -) - -func TestNoLinkForbidden(t *testing.T) { - defer leaktest.AfterTest(t)() - - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colexecop", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colcontainer", - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colflow", - "github.com/cockroachdb/cockroach/pkg/sql/rowexec", - "github.com/cockroachdb/cockroach/pkg/sql/rowflow", - }, nil, - ) -} diff --git a/pkg/sql/colflow/BUILD.bazel b/pkg/sql/colflow/BUILD.bazel index eaf4e94a2bdc..725b4a756ab4 100644 --- a/pkg/sql/colflow/BUILD.bazel +++ b/pkg/sql/colflow/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "colflow", @@ -61,7 +62,6 @@ go_test( size = "medium", srcs = [ "colbatch_scan_test.go", - "dep_test.go", "draining_test.go", "main_test.go", "routers_test.go", @@ -110,7 +110,6 @@ go_test( "//pkg/sql/types", "//pkg/storage", "//pkg/testutils", - "//pkg/testutils/buildutil", "//pkg/testutils/colcontainerutils", "//pkg/testutils/distsqlutils", "//pkg/testutils/serverutils", @@ -133,3 +132,11 @@ go_test( "@com_github_stretchr_testify//require", ], ) + +disallowed_imports_test( + "colflow", + [ + "//pkg/sql/distsql", + "//pkg/sql/rowflow", + ], +) diff --git a/pkg/sql/colflow/dep_test.go b/pkg/sql/colflow/dep_test.go deleted file mode 100644 index af9d6f87bcc1..000000000000 --- a/pkg/sql/colflow/dep_test.go +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package colflow - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" -) - -func TestNoLinkForbidden(t *testing.T) { - defer leaktest.AfterTest(t)() - - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/colflow", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/distsql", - "github.com/cockroachdb/cockroach/pkg/sql/rowflow", - }, nil, - ) -} diff --git a/pkg/sql/execinfra/BUILD.bazel b/pkg/sql/execinfra/BUILD.bazel index b81635763f63..f10ee45c53d0 100644 --- a/pkg/sql/execinfra/BUILD.bazel +++ b/pkg/sql/execinfra/BUILD.bazel @@ -1,5 +1,6 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") load("//build:STRINGER.bzl", "stringer") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "execinfra", @@ -84,7 +85,6 @@ go_test( size = "small", srcs = [ "base_test.go", - "dep_test.go", "main_test.go", ], embed = [":execinfra"], @@ -98,7 +98,6 @@ go_test( "//pkg/sql/rowenc", "//pkg/sql/sem/tree", "//pkg/sql/types", - "//pkg/testutils/buildutil", "//pkg/testutils/serverutils", "//pkg/testutils/testcluster", "//pkg/util/leaktest", @@ -117,3 +116,14 @@ stringer( src = "base.go", typ = "ConsumerStatus", ) + +disallowed_imports_test( + "execinfra", + [ + "//pkg/sql/colexec", + "//pkg/sql/colflow", + "//pkg/sql/flowinfra", + "//pkg/sql/rowexec", + "//pkg/sql/rowflow", + ], +) diff --git a/pkg/sql/execinfra/dep_test.go b/pkg/sql/execinfra/dep_test.go deleted file mode 100644 index 09a7dd5dc034..000000000000 --- a/pkg/sql/execinfra/dep_test.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package execinfra - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" -) - -func TestNoLinkForbidden(t *testing.T) { - defer leaktest.AfterTest(t)() - - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/execinfra", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colflow", - "github.com/cockroachdb/cockroach/pkg/sql/flowinfra", - "github.com/cockroachdb/cockroach/pkg/sql/rowexec", - "github.com/cockroachdb/cockroach/pkg/sql/rowflow", - }, nil, - ) -} diff --git a/pkg/sql/flowinfra/BUILD.bazel b/pkg/sql/flowinfra/BUILD.bazel index 53c5461c4f1e..18ea09fca3af 100644 --- a/pkg/sql/flowinfra/BUILD.bazel +++ b/pkg/sql/flowinfra/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "flowinfra", @@ -52,7 +53,6 @@ go_test( size = "medium", srcs = [ "cluster_test.go", - "dep_test.go", "flow_registry_test.go", "flow_scheduler_test.go", "flow_test.go", @@ -94,7 +94,6 @@ go_test( "//pkg/sql/tests", "//pkg/sql/types", "//pkg/testutils", - "//pkg/testutils/buildutil", "//pkg/testutils/distsqlutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", @@ -112,3 +111,8 @@ go_test( "@com_github_stretchr_testify//require", ], ) + +disallowed_imports_test( + "flowinfra", + ["//pkg/sql/distsql"], +) diff --git a/pkg/sql/flowinfra/dep_test.go b/pkg/sql/flowinfra/dep_test.go deleted file mode 100644 index 2e83ef810027..000000000000 --- a/pkg/sql/flowinfra/dep_test.go +++ /dev/null @@ -1,29 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package flowinfra - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" -) - -func TestNoLinkForbidden(t *testing.T) { - defer leaktest.AfterTest(t)() - - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/flowinfra", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/distsql", - }, nil, - ) -} diff --git a/pkg/sql/inverted/BUILD.bazel b/pkg/sql/inverted/BUILD.bazel index 82793278e4df..e414cb9152be 100644 --- a/pkg/sql/inverted/BUILD.bazel +++ b/pkg/sql/inverted/BUILD.bazel @@ -1,6 +1,7 @@ load("@rules_proto//proto:defs.bzl", "proto_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "inverted", @@ -18,15 +19,11 @@ go_library( go_test( name = "inverted_test", size = "small", - srcs = [ - "dep_test.go", - "expression_test.go", - ], + srcs = ["expression_test.go"], data = glob(["testdata/**"]), embed = [":inverted"], deps = [ "//pkg/testutils", - "//pkg/testutils/buildutil", "//pkg/util/encoding", "//pkg/util/leaktest", "//pkg/util/treeprinter", @@ -52,3 +49,8 @@ go_proto_library( visibility = ["//visibility:public"], deps = ["@com_github_gogo_protobuf//gogoproto"], ) + +disallowed_imports_test( + "inverted", + ["//pkg/roachpb"], +) diff --git a/pkg/sql/inverted/dep_test.go b/pkg/sql/inverted/dep_test.go deleted file mode 100644 index 3123af8bf5e3..000000000000 --- a/pkg/sql/inverted/dep_test.go +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package inverted - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/inverted", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/roachpb", - }, nil, - ) -} diff --git a/pkg/sql/lex/BUILD.bazel b/pkg/sql/lex/BUILD.bazel index 25b7b0463f4e..28472048ddd9 100644 --- a/pkg/sql/lex/BUILD.bazel +++ b/pkg/sql/lex/BUILD.bazel @@ -1,6 +1,7 @@ load("@rules_proto//proto:defs.bzl", "proto_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "lex", @@ -18,12 +19,8 @@ go_library( go_test( name = "lex_test", - srcs = [ - "dep_test.go", - "encode_test.go", - ], - embed = [":lex"], - deps = ["//pkg/testutils/buildutil"], + srcs = ["encode_test.go"], + deps = [":lex"], ) proto_library( @@ -42,3 +39,8 @@ go_proto_library( visibility = ["//visibility:public"], deps = ["@com_github_gogo_protobuf//gogoproto"], ) + +disallowed_imports_test( + "lex", + ["//pkg/sql/sessiondatapb"], +) diff --git a/pkg/sql/lex/dep_test.go b/pkg/sql/lex/dep_test.go deleted file mode 100644 index b6f5d9b4ed7a..000000000000 --- a/pkg/sql/lex/dep_test.go +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package lex - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/lex", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb", - }, nil, - ) -} diff --git a/pkg/sql/logictest/testdata/logic_test/builtin_function b/pkg/sql/logictest/testdata/logic_test/builtin_function index 97659d2de255..eb70dba38e27 100644 --- a/pkg/sql/logictest/testdata/logic_test/builtin_function +++ b/pkg/sql/logictest/testdata/logic_test/builtin_function @@ -3138,10 +3138,8 @@ subtest session_revival_token user host-cluster-root -# TODO(rafi): use ALTER TENANT ALL when available. statement ok -INSERT INTO system.tenant_settings (tenant_id, name, value, value_type) VALUES - (0, 'server.user_login.session_revival_token.enabled', 'true', 'b'); +ALTER TENANT ALL SET CLUSTER SETTING server.user_login.session_revival_token.enabled = true user root @@ -3206,10 +3204,8 @@ Ed25519 testuser true true true true user host-cluster-root -# TODO(rafi): use ALTER TENANT ALL when available. statement ok -UPSERT INTO system.tenant_settings (tenant_id, name, value, value_type) VALUES - (0, 'server.user_login.session_revival_token.enabled', 'false', 'b') +ALTER TENANT ALL SET CLUSTER SETTING server.user_login.session_revival_token.enabled = false user root diff --git a/pkg/sql/logictest/testdata/logic_test/limit b/pkg/sql/logictest/testdata/logic_test/limit index 03b408966e42..8e9adc9228b3 100644 --- a/pkg/sql/logictest/testdata/logic_test/limit +++ b/pkg/sql/logictest/testdata/logic_test/limit @@ -351,3 +351,10 @@ SELECT * FROM t ORDER BY v, w LIMIT 3; 6 -36 216 4 -16 94 2 -4 8 + +query IT +SELECT oid::INT, typname FROM pg_type ORDER BY oid LIMIT 3 +---- +16 bool +17 bytea +18 char diff --git a/pkg/sql/logictest/testdata/logic_test/pg_builtins b/pkg/sql/logictest/testdata/logic_test/pg_builtins index 8de758f0a9e0..ae6bc8ee7e72 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_builtins +++ b/pkg/sql/logictest/testdata/logic_test/pg_builtins @@ -610,3 +610,133 @@ a NULL b c · d e f=g + + +statement ok +CREATE TYPE test_type AS ENUM ('open', 'closed', 'inactive'); + +statement ok +CREATE ROLE test_role LOGIN; + +statement ok +CREATE TABLE t1 (a int); + +query T +SELECT to_regclass('pg_roles') +---- +pg_roles + +query T +SELECT to_regclass('4294967230') +---- +NULL + +query T +SELECT to_regclass('pg_policy') +---- +pg_policy + +query T +SELECT to_regclass('t1') +---- +t1 + +query T +SELECT to_regnamespace('crdb_internal') +---- +crdb_internal + +query T +SELECT to_regnamespace('public') +---- +public + +query T +SELECT to_regnamespace('1330834471') +---- +NULL + +query T +SELECT to_regnamespace(' 1330834471') +---- +NULL + +query T +SELECT to_regproc('_st_contains') +---- +_st_contains + +query T +SELECT to_regproc('version') +---- +version + +query T +SELECT to_regproc('bit_in') +---- +bit_in + +query T +SELECT to_regprocedure('bit_in') +---- +bit_in + +query T +SELECT to_regprocedure('version') +---- +version + +query T +SELECT to_regprocedure('961893967') +---- +NULL + +query T +SELECT to_regrole('admin') +---- +admin + +query T +SELECT to_regrole('test_role') +---- +test_role + +query T +SELECT to_regrole('foo') +---- +NULL + +query T +SELECT to_regrole('1546506610') +---- +NULL + +query T +SELECT to_regtype('interval') +---- +interval + +query T +SELECT to_regtype('integer') +---- +bigint + +query T +SELECT to_regtype('int_4') +---- +NULL + +query T +SELECT to_regtype('string') +---- +text + +query T +SELECT to_regtype('1186') +---- +NULL + +query T +SELECT to_regtype('test_type') +---- +test_type diff --git a/pkg/sql/opt/exec/execbuilder/testdata/limit b/pkg/sql/opt/exec/execbuilder/testdata/limit index e43523f9b8d5..ad65490b9e2b 100644 --- a/pkg/sql/opt/exec/execbuilder/testdata/limit +++ b/pkg/sql/opt/exec/execbuilder/testdata/limit @@ -527,3 +527,42 @@ vectorized: true estimated row count: 100 - 1,001 (100% of the table; stats collected ago) table: a@a_i_j_idx spans: FULL SCAN + +# A limit cannot be pushed into the scan of a virtual table with ORDER BY. +query T +EXPLAIN SELECT oid, typname FROM pg_type ORDER BY oid LIMIT 10 +---- +distribution: local +vectorized: true +· +• limit +│ count: 10 +│ +└── • virtual table + table: pg_type@pg_type_oid_idx + +# A limit can be pushed into the scan of a virtual table without ORDER BY. +query T +EXPLAIN SELECT oid, typname FROM pg_type LIMIT 10 +---- +distribution: local +vectorized: true +· +• virtual table + table: pg_type@primary + limit: 10 + +# A limit cannot be pushed into the constrained scan of a virtual table with +# ORDER BY. +query T +EXPLAIN SELECT oid, typname FROM pg_type WHERE OID BETWEEN 1 AND 1000 ORDER BY oid LIMIT 10 +---- +distribution: local +vectorized: true +· +• limit +│ count: 10 +│ +└── • virtual table + table: pg_type@pg_type_oid_idx + spans: [/1 - /1000] diff --git a/pkg/sql/opt/memo/expr.go b/pkg/sql/opt/memo/expr.go index 6905a8a48289..5bae3f57ddb8 100644 --- a/pkg/sql/opt/memo/expr.go +++ b/pkg/sql/opt/memo/expr.go @@ -668,6 +668,12 @@ func (s *ScanPrivate) IsFullIndexScan(md *opt.Metadata) bool { s.HardLimit == 0 } +// IsVirtualTable returns true if the table being scanned is a virtual table. +func (s *ScanPrivate) IsVirtualTable(md *opt.Metadata) bool { + tab := md.Table(s.Table) + return tab.IsVirtualTable() +} + // IsLocking returns true if the ScanPrivate is configured to use a row-level // locking mode. This can be the case either because the Scan is in the scope of // a SELECT .. FOR [KEY] UPDATE/SHARE clause or because the Scan was configured diff --git a/pkg/sql/opt/xform/limit_funcs.go b/pkg/sql/opt/xform/limit_funcs.go index d7865839e6f6..782650b01a55 100644 --- a/pkg/sql/opt/xform/limit_funcs.go +++ b/pkg/sql/opt/xform/limit_funcs.go @@ -60,7 +60,14 @@ func (c *CustomFuncs) CanLimitFilteredScan( // unconstrained scans on non-partial indexes. return false } - + // Virtual indexes are not sorted, but are presented to the optimizer as + // sorted, and have a sort automatically applied to them on the output of the + // scan. Since this implicit sort happens after the scan, we can't place a + // hard limit on the scan if the query semantics require a sort order on the + // entire relation. + if scanPrivate.IsVirtualTable(md) && !required.Any() { + return false + } ok, _ := ordering.ScanPrivateCanProvide(c.e.mem.Metadata(), scanPrivate, &required) return ok } @@ -87,6 +94,14 @@ func (c *CustomFuncs) CanLimitFilteredScan( func (c *CustomFuncs) GenerateLimitedScans( grp memo.RelExpr, scanPrivate *memo.ScanPrivate, limit tree.Datum, required props.OrderingChoice, ) { + // Virtual indexes are not sorted, but are presented to the optimizer as + // sorted, and have a sort automatically applied to them on the output of the + // scan. Since this implicit sort happens after the scan, we can't place a + // hard limit on the scan if the query semantics require a sort order on the + // entire relation. + if scanPrivate.IsVirtualTable(c.e.mem.Metadata()) && !required.Any() { + return + } limitVal := int64(*limit.(*tree.DInt)) var pkCols opt.ColSet diff --git a/pkg/sql/rowexec/BUILD.bazel b/pkg/sql/rowexec/BUILD.bazel index 9b046f0916a0..3606292564bd 100644 --- a/pkg/sql/rowexec/BUILD.bazel +++ b/pkg/sql/rowexec/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "rowexec", @@ -107,7 +108,6 @@ go_test( srcs = [ "aggregator_test.go", "backfiller_test.go", - "dep_test.go", "distinct_test.go", "filterer_test.go", "hashjoiner_test.go", @@ -176,7 +176,6 @@ go_test( "//pkg/startupmigrations", "//pkg/storage", "//pkg/testutils", - "//pkg/testutils/buildutil", "//pkg/testutils/distsqlutils", "//pkg/testutils/serverutils", "//pkg/testutils/skip", @@ -202,3 +201,12 @@ go_test( "@com_github_stretchr_testify//require", ], ) + +disallowed_imports_test( + "rowexec", + [ + "//pkg/sql/colexec", + "//pkg/sql/colflow", + "//pkg/sql/rowflow", + ], +) diff --git a/pkg/sql/rowexec/dep_test.go b/pkg/sql/rowexec/dep_test.go deleted file mode 100644 index fa0256f0331b..000000000000 --- a/pkg/sql/rowexec/dep_test.go +++ /dev/null @@ -1,31 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package rowexec - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" -) - -func TestNoLinkForbidden(t *testing.T) { - defer leaktest.AfterTest(t)() - - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/rowexec", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colexec", - "github.com/cockroachdb/cockroach/pkg/sql/colflow", - "github.com/cockroachdb/cockroach/pkg/sql/rowflow", - }, nil, - ) -} diff --git a/pkg/sql/rowflow/BUILD.bazel b/pkg/sql/rowflow/BUILD.bazel index aff5c52aba7d..d6501f182511 100644 --- a/pkg/sql/rowflow/BUILD.bazel +++ b/pkg/sql/rowflow/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "rowflow", @@ -34,7 +35,6 @@ go_test( name = "rowflow_test", size = "small", srcs = [ - "dep_test.go", "input_sync_test.go", "routers_test.go", ], @@ -54,7 +54,6 @@ go_test( "//pkg/sql/types", "//pkg/storage", "//pkg/testutils", - "//pkg/testutils/buildutil", "//pkg/testutils/distsqlutils", "//pkg/util/encoding", "//pkg/util/leaktest", @@ -66,3 +65,11 @@ go_test( "@com_github_stretchr_testify//require", ], ) + +disallowed_imports_test( + "rowflow", + [ + "//pkg/sql/colflow", + "//pkg/sql/distsql", + ], +) diff --git a/pkg/sql/rowflow/dep_test.go b/pkg/sql/rowflow/dep_test.go deleted file mode 100644 index d9a4b72a65fc..000000000000 --- a/pkg/sql/rowflow/dep_test.go +++ /dev/null @@ -1,30 +0,0 @@ -// Copyright 2019 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package rowflow - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" - "github.com/cockroachdb/cockroach/pkg/util/leaktest" -) - -func TestNoLinkForbidden(t *testing.T) { - defer leaktest.AfterTest(t)() - - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/rowflow", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/sql/colflow", - "github.com/cockroachdb/cockroach/pkg/sql/distsql", - }, nil, - ) -} diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go index 523859266b83..e413452e9755 100644 --- a/pkg/sql/sem/builtins/pg_builtins.go +++ b/pkg/sql/sem/builtins/pg_builtins.go @@ -12,6 +12,7 @@ package builtins import ( "fmt" + "strconv" "strings" "time" @@ -146,18 +147,23 @@ func initPGBuiltins() { builtins[name] = builtin } - // Make crdb_internal.create_regfoo builtins. - for _, typ := range []*types.T{ - types.RegClass, - types.RegNamespace, - types.RegProc, - types.RegProcedure, - types.RegRole, - types.RegType, + // Make crdb_internal.create_regfoo and to_regfoo builtins. + for _, b := range []struct { + toRegOverloadHelpText string + typ *types.T + }{ + {"Translates a textual relation name to its OID", types.RegClass}, + {"Translates a textual schema name to its OID", types.RegNamespace}, + {"Translates a textual function or procedure name to its OID", types.RegProc}, + {"Translates a textual function or procedure name(with argument types) to its OID", types.RegProcedure}, + {"Translates a textual role name to its OID", types.RegRole}, + {"Translates a textual type name to its OID", types.RegType}, } { - typName := typ.SQLStandardName() - builtins["crdb_internal.create_"+typName] = makeCreateRegDef(typ) + typName := b.typ.SQLStandardName() + builtins["crdb_internal.create_"+typName] = makeCreateRegDef(b.typ) + builtins["to_"+typName] = makeToRegOverload(b.typ, b.toRegOverloadHelpText) } + } var errUnimplemented = pgerror.New(pgcode.FeatureNotSupported, "unimplemented") @@ -520,6 +526,33 @@ func makeCreateRegDef(typ *types.T) builtinDefinition { ) } +func makeToRegOverload(typ *types.T, helpText string) builtinDefinition { + return makeBuiltin(tree.FunctionProperties{Category: categorySystemInfo}, + tree.Overload{ + Types: tree.ArgTypes{ + {"text", types.String}, + }, + ReturnType: tree.FixedReturnType(types.RegType), + Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) { + typName := tree.MustBeDString(args[0]) + int, _ := strconv.Atoi(strings.TrimSpace(string(typName))) + if int > 0 { + return tree.DNull, nil + } + typOid, err := tree.ParseDOid(ctx, string(typName), typ) + if err != nil { + //nolint:returnerrcheck + return tree.DNull, nil + } + + return typOid, nil + }, + Info: helpText, + Volatility: tree.VolatilityStable, + }, + ) +} + var pgBuiltins = map[string]builtinDefinition{ // See https://www.postgresql.org/docs/9.6/static/functions-info.html. "pg_backend_pid": makeBuiltin(defProps(), diff --git a/pkg/sql/types/BUILD.bazel b/pkg/sql/types/BUILD.bazel index 09b896c1103e..967c01ec8cfc 100644 --- a/pkg/sql/types/BUILD.bazel +++ b/pkg/sql/types/BUILD.bazel @@ -1,6 +1,7 @@ load("@rules_proto//proto:defs.bzl", "proto_library") load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library") load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "types", @@ -33,7 +34,6 @@ go_test( name = "types_test", size = "small", srcs = [ - "dep_test.go", "types_test.go", "types_text_marshal_test.go", ], @@ -43,7 +43,6 @@ go_test( "//pkg/sql/catalog/descpb", "//pkg/sql/catalog/typedesc", "//pkg/sql/oidext", - "//pkg/testutils/buildutil", "//pkg/util/protoutil", "@com_github_lib_pq//oid", "@com_github_stretchr_testify//assert", @@ -73,3 +72,11 @@ go_proto_library( "@com_github_gogo_protobuf//gogoproto", ], ) + +disallowed_imports_test( + "types", + [ + "//pkg/clusterversion", + "//pkg/roachpb", + ], +) diff --git a/pkg/sql/types/dep_test.go b/pkg/sql/types/dep_test.go deleted file mode 100644 index 8a4b253de2b5..000000000000 --- a/pkg/sql/types/dep_test.go +++ /dev/null @@ -1,27 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package types - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/sql/types", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/clusterversion", - "github.com/cockroachdb/cockroach/pkg/roachpb", - }, nil, - ) -} diff --git a/pkg/storage/pebble.go b/pkg/storage/pebble.go index ccc54793d8ce..6bad3dfeeb7b 100644 --- a/pkg/storage/pebble.go +++ b/pkg/storage/pebble.go @@ -1767,9 +1767,11 @@ type pebbleReadOnly struct { normalIter pebbleIterator prefixEngineIter pebbleIterator normalEngineIter pebbleIterator - iter cloneableIter - durability DurabilityRequirement - closed bool + + iter cloneableIter + iterUnused bool + durability DurabilityRequirement + closed bool } var _ ReadWriter = &pebbleReadOnly{} @@ -1811,6 +1813,13 @@ func (p *pebbleReadOnly) Close() { panic("closing an already-closed pebbleReadOnly") } p.closed = true + if p.iterUnused { + err := p.iter.Close() + if err != nil { + panic(err) + } + } + // Setting iter to nil is sufficient since it will be closed by one of the // subsequent destroy calls. p.iter = nil @@ -1929,11 +1938,12 @@ func (p *pebbleReadOnly) NewMVCCIterator(iterKind MVCCIterKind, opts IterOptions if iter.iter != nil { iter.setBounds(opts.LowerBound, opts.UpperBound) } else { - iter.init(p.parent.db, p.iter, opts, p.durability) + iter.init(p.parent.db, p.iter, p.iterUnused, opts, p.durability) if p.iter == nil { // For future cloning. p.iter = iter.iter } + p.iterUnused = false iter.reusable = true } @@ -1964,11 +1974,12 @@ func (p *pebbleReadOnly) NewEngineIterator(opts IterOptions) EngineIterator { if iter.iter != nil { iter.setBounds(opts.LowerBound, opts.UpperBound) } else { - iter.init(p.parent.db, p.iter, opts, p.durability) + iter.init(p.parent.db, p.iter, p.iterUnused, opts, p.durability) if p.iter == nil { // For future cloning. p.iter = iter.iter } + p.iterUnused = false iter.reusable = true } @@ -2001,6 +2012,10 @@ func (p *pebbleReadOnly) PinEngineStateForIterators() error { o = &pebble.IterOptions{OnlyReadGuaranteedDurable: true} } p.iter = p.parent.db.NewIter(o) + // Since the iterator is being created just to pin the state of the engine + // for future iterators, we'll avoid cloning it the next time we want an + // iterator and instead just re-use what we created here. + p.iterUnused = true } return nil } diff --git a/pkg/storage/pebble_batch.go b/pkg/storage/pebble_batch.go index 371f430dd590..ffb853472074 100644 --- a/pkg/storage/pebble_batch.go +++ b/pkg/storage/pebble_batch.go @@ -47,9 +47,11 @@ type pebbleBatch struct { normalIter pebbleIterator prefixEngineIter pebbleIterator normalEngineIter pebbleIterator - iter cloneableIter - writeOnly bool - closed bool + + iter cloneableIter + writeOnly bool + iterUnused bool + closed bool wrappedIntentWriter intentDemuxWriter // scratch space for wrappedIntentWriter. @@ -104,6 +106,12 @@ func (p *pebbleBatch) Close() { } p.closed = true + if p.iterUnused { + if err := p.iter.Close(); err != nil { + panic(err) + } + } + // Setting iter to nil is sufficient since it will be closed by one of the // subsequent destroy calls. p.iter = nil @@ -230,14 +238,15 @@ func (p *pebbleBatch) NewMVCCIterator(iterKind MVCCIterKind, opts IterOptions) M iter.setBounds(opts.LowerBound, opts.UpperBound) } else { if p.batch.Indexed() { - iter.init(p.batch, p.iter, opts, StandardDurability) + iter.init(p.batch, p.iter, p.iterUnused, opts, StandardDurability) } else { - iter.init(p.db, p.iter, opts, StandardDurability) + iter.init(p.db, p.iter, p.iterUnused, opts, StandardDurability) } if p.iter == nil { // For future cloning. p.iter = iter.iter } + p.iterUnused = false } iter.inuse = true @@ -272,14 +281,15 @@ func (p *pebbleBatch) NewEngineIterator(opts IterOptions) EngineIterator { iter.setBounds(opts.LowerBound, opts.UpperBound) } else { if p.batch.Indexed() { - iter.init(p.batch, p.iter, opts, StandardDurability) + iter.init(p.batch, p.iter, p.iterUnused, opts, StandardDurability) } else { - iter.init(p.db, p.iter, opts, StandardDurability) + iter.init(p.db, p.iter, p.iterUnused, opts, StandardDurability) } if p.iter == nil { // For future cloning. p.iter = iter.iter } + p.iterUnused = false } iter.inuse = true @@ -299,6 +309,10 @@ func (p *pebbleBatch) PinEngineStateForIterators() error { } else { p.iter = p.db.NewIter(nil) } + // Since the iterator is being created just to pin the state of the engine + // for future iterators, we'll avoid cloning it the next time we want an + // iterator and instead just re-use what we created here. + p.iterUnused = true } return nil } diff --git a/pkg/storage/pebble_iterator.go b/pkg/storage/pebble_iterator.go index 1797cd4cb1de..8fc315668657 100644 --- a/pkg/storage/pebble_iterator.go +++ b/pkg/storage/pebble_iterator.go @@ -76,6 +76,7 @@ var pebbleIterPool = sync.Pool{ type cloneableIter interface { Clone() (*pebble.Iterator, error) + Close() error } type testingSetBoundsListener interface { @@ -91,7 +92,7 @@ func newPebbleIterator( ) *pebbleIterator { iter := pebbleIterPool.Get().(*pebbleIterator) iter.reusable = false // defensive - iter.init(handle, iterToClone, opts, durability) + iter.init(handle, iterToClone, false /* iterUnused */, opts, durability) return iter } @@ -106,6 +107,7 @@ func newPebbleIterator( func (p *pebbleIterator) init( handle pebble.Reader, iterToClone cloneableIter, + iterUnused bool, opts IterOptions, durability DurabilityRequirement, ) { @@ -183,8 +185,15 @@ func (p *pebbleIterator) init( if doClone { var err error - if p.iter, err = iterToClone.Clone(); err != nil { - panic(err) + if iterUnused { + // NB: If the iterator was never used (at the time of writing, this means + // that the iterator was created by `PinEngineStateForIterators()`), we + // don't need to clone it. + p.iter = iterToClone.(*pebble.Iterator) + } else { + if p.iter, err = iterToClone.Clone(); err != nil { + panic(err) + } } p.iter.SetBounds(p.options.LowerBound, p.options.UpperBound) } else { diff --git a/pkg/ts/catalog/chart_catalog.go b/pkg/ts/catalog/chart_catalog.go index 08893c4befa8..3f6710fe4e8d 100644 --- a/pkg/ts/catalog/chart_catalog.go +++ b/pkg/ts/catalog/chart_catalog.go @@ -2859,6 +2859,12 @@ var charts = []sectionDescription{ { Organization: [][]string{{Jobs, "Execution"}}, Charts: []chartDescription{ + { + Title: "Active", + Metrics: []string{ + "jobs.running_non_idle", + }, + }, { Title: "Currently Running", Metrics: []string{ diff --git a/pkg/util/json/BUILD.bazel b/pkg/util/json/BUILD.bazel index abd9fa6c111d..7b4cad15aad8 100644 --- a/pkg/util/json/BUILD.bazel +++ b/pkg/util/json/BUILD.bazel @@ -1,4 +1,5 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") go_library( name = "json", @@ -34,7 +35,6 @@ go_test( name = "json_test", size = "small", srcs = [ - "dep_test.go", "encode_test.go", "json_test.go", ], @@ -44,7 +44,6 @@ go_test( "//pkg/sql/inverted", "//pkg/sql/pgwire/pgerror", "//pkg/testutils", - "//pkg/testutils/buildutil", "//pkg/util/encoding", "//pkg/util/randutil", "//pkg/util/timeutil", @@ -53,3 +52,8 @@ go_test( "@com_github_stretchr_testify//require", ], ) + +disallowed_imports_test( + "json", + ["//pkg/roachpb"], +) diff --git a/pkg/util/json/dep_test.go b/pkg/util/json/dep_test.go deleted file mode 100644 index 038c1c2773d6..000000000000 --- a/pkg/util/json/dep_test.go +++ /dev/null @@ -1,26 +0,0 @@ -// Copyright 2022 The Cockroach Authors. -// -// Use of this software is governed by the Business Source License -// included in the file licenses/BSL.txt. -// -// As of the Change Date specified in that file, in accordance with -// the Business Source License, use of this software will be governed -// by the Apache License, Version 2.0, included in the file -// licenses/APL.txt. - -package json - -import ( - "testing" - - "github.com/cockroachdb/cockroach/pkg/testutils/buildutil" -) - -func TestNoLinkForbidden(t *testing.T) { - buildutil.VerifyNoImports(t, - "github.com/cockroachdb/cockroach/pkg/util/json", true, - []string{ - "github.com/cockroachdb/cockroach/pkg/roachpb", - }, nil, - ) -}