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,
- )
-}