diff --git a/Makefile b/Makefile
index f6400373a85d..2ba3e2e25565 100644
--- a/Makefile
+++ b/Makefile
@@ -802,7 +802,7 @@ SQLPARSER_TARGETS = \
pkg/sql/parser/help_messages.go \
pkg/sql/lex/tokens.go \
pkg/sql/lex/keywords.go \
- pkg/sql/lex/reserved_keywords.go
+ pkg/sql/lexbase/reserved_keywords.go
PROTOBUF_TARGETS := bin/.go_protobuf_sources bin/.gw_protobuf_sources
@@ -1462,7 +1462,7 @@ pkg/sql/parser/gen/sql-gen.y: pkg/sql/parser/sql.y pkg/sql/parser/replace_help_r
mv -f $@.tmp $@
rm pkg/sql/parser/gen/types_regex.tmp
-pkg/sql/lex/reserved_keywords.go: pkg/sql/parser/sql.y pkg/sql/parser/reserved_keywords.awk | bin/.bootstrap
+pkg/sql/lexbase/reserved_keywords.go: pkg/sql/parser/sql.y pkg/sql/parser/reserved_keywords.awk | bin/.bootstrap
awk -f pkg/sql/parser/reserved_keywords.awk < $< > $@.tmp || rm $@.tmp
mv -f $@.tmp $@
gofmt -s -w $@
diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md
index b17a14af6f15..877c531f7f29 100644
--- a/docs/generated/http/full.md
+++ b/docs/generated/http/full.md
@@ -752,7 +752,7 @@ The underlying response type is something we're looking to get rid of.
| Field | Type | Label | Description |
| ----- | ---- | ----- | ----------- |
-| username | [string](#cockroach.server.serverpb.ListSessionsRequest-string) | | Username of the user making this request. |
+| username | [string](#cockroach.server.serverpb.ListSessionsRequest-string) | | Username of the user making this request. The caller is responsible to normalize the username (= case fold and perform unicode NFC normalization). |
@@ -864,7 +864,7 @@ The underlying response type is something we're looking to get rid of.
| Field | Type | Label | Description |
| ----- | ---- | ----- | ----------- |
-| username | [string](#cockroach.server.serverpb.ListSessionsRequest-string) | | Username of the user making this request. |
+| username | [string](#cockroach.server.serverpb.ListSessionsRequest-string) | | Username of the user making this request. The caller is responsible to normalize the username (= case fold and perform unicode NFC normalization). |
@@ -978,7 +978,7 @@ The underlying response type is something we're looking to get rid of.
| ----- | ---- | ----- | ----------- |
| node_id | [string](#cockroach.server.serverpb.CancelQueryRequest-string) | | ID of gateway node for the query to be canceled. TODO(itsbilal): use [(gogoproto.customname) = "NodeID"] below. Need to figure out how to teach grpc-gateway about custom names. node_id is a string so that "local" can be used to specify that no forwarding is necessary. |
| query_id | [string](#cockroach.server.serverpb.CancelQueryRequest-string) | | ID of query to be canceled (converted to string). |
-| username | [string](#cockroach.server.serverpb.CancelQueryRequest-string) | | Username of the user making this cancellation request. This may be omitted if the user is the same as the one issuing the CancelQueryRequest. |
+| username | [string](#cockroach.server.serverpb.CancelQueryRequest-string) | | Username of the user making this cancellation request. This may be omitted if the user is the same as the one issuing the CancelQueryRequest. The caller is responsible for case-folding and NFC normalization. |
@@ -1017,7 +1017,7 @@ The underlying response type is something we're looking to get rid of.
| ----- | ---- | ----- | ----------- |
| node_id | [string](#cockroach.server.serverpb.CancelSessionRequest-string) | | TODO(abhimadan): use [(gogoproto.customname) = "NodeID"] below. Need to figure out how to teach grpc-gateway about custom names. node_id is a string so that "local" can be used to specify that no forwarding is necessary. |
| session_id | [bytes](#cockroach.server.serverpb.CancelSessionRequest-bytes) | | |
-| username | [string](#cockroach.server.serverpb.CancelSessionRequest-string) | | Username of the user making this cancellation request. This may be omitted if the user is the same as the one issuing the CancelSessionRequest. |
+| username | [string](#cockroach.server.serverpb.CancelSessionRequest-string) | | Username of the user making this cancellation request. This may be omitted if the user is the same as the one issuing the CancelSessionRequest. The caller is responsiblef or case-folding and NFC normalization. |
diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf
index ba7e9097ff41..04f60ff6afde 100644
--- a/docs/generated/sql/bnf/stmt_block.bnf
+++ b/docs/generated/sql/bnf/stmt_block.bnf
@@ -314,7 +314,7 @@ role_spec_list ::=
( role_spec ) ( ( ',' role_spec ) )*
role_spec ::=
- non_reserved_word_or_sconst
+ username_or_sconst
| 'CURRENT_USER'
| 'SESSION_USER'
@@ -1174,7 +1174,7 @@ type_name_list ::=
type_list ::=
( typename ) ( ( ',' typename ) )*
-non_reserved_word_or_sconst ::=
+username_or_sconst ::=
non_reserved_word
| 'SCONST'
@@ -1392,6 +1392,10 @@ drop_type_stmt ::=
explain_option_name ::=
non_reserved_word
+non_reserved_word_or_sconst ::=
+ non_reserved_word
+ | 'SCONST'
+
kv_option_list ::=
( kv_option ) ( ( ',' kv_option ) )*
diff --git a/pkg/acceptance/cluster/certs.go b/pkg/acceptance/cluster/certs.go
index fab83bda9948..a9891c96e53b 100644
--- a/pkg/acceptance/cluster/certs.go
+++ b/pkg/acceptance/cluster/certs.go
@@ -39,12 +39,12 @@ func GenerateCerts(ctx context.Context) func() {
// Root user.
maybePanic(security.CreateClientPair(
certsDir, filepath.Join(certsDir, security.EmbeddedCAKey),
- 1024, 48*time.Hour, false, security.RootUser, true /* generate pk8 key */))
+ 1024, 48*time.Hour, false, security.RootUserName(), true /* generate pk8 key */))
// Test user.
maybePanic(security.CreateClientPair(
certsDir, filepath.Join(certsDir, security.EmbeddedCAKey),
- 1024, 48*time.Hour, false, "testuser", true /* generate pk8 key */))
+ 1024, 48*time.Hour, false, security.TestUserName(), true /* generate pk8 key */))
// Certs for starting a cockroach server. Key size is from cli/cert.go:defaultKeySize.
maybePanic(security.CreateNodePair(
diff --git a/pkg/acceptance/localcluster/cluster.go b/pkg/acceptance/localcluster/cluster.go
index 2cc8f1c0cf6c..45684fcf1ecb 100644
--- a/pkg/acceptance/localcluster/cluster.go
+++ b/pkg/acceptance/localcluster/cluster.go
@@ -268,7 +268,7 @@ func (c *Cluster) RPCPort(nodeIdx int) string {
func (c *Cluster) makeNode(ctx context.Context, nodeIdx int, cfg NodeConfig) (*Node, <-chan error) {
baseCtx := &base.Config{
- User: security.NodeUser,
+ User: security.NodeUserName(),
Insecure: true,
}
rpcCtx := rpc.NewContext(rpc.ContextOptions{
diff --git a/pkg/base/config.go b/pkg/base/config.go
index caa5423be009..bdf540be5050 100644
--- a/pkg/base/config.go
+++ b/pkg/base/config.go
@@ -169,7 +169,7 @@ type Config struct {
// User running this process. It could be the user under which
// the server is running or the user passed in client calls.
- User string
+ User security.SQLUsername
// Addr is the address the server is listening on.
Addr string
@@ -255,7 +255,7 @@ func (*Config) HistogramWindowInterval() time.Duration {
// This is also used in tests to reset global objects.
func (cfg *Config) InitDefaults() {
cfg.Insecure = defaultInsecure
- cfg.User = defaultUser
+ cfg.User = security.MakeSQLUsernameFromPreNormalizedString(defaultUser)
cfg.Addr = defaultAddr
cfg.AdvertiseAddr = cfg.Addr
cfg.HTTPAddr = defaultHTTPAddr
diff --git a/pkg/ccl/backupccl/backup_destination.go b/pkg/ccl/backupccl/backup_destination.go
index 4e3c48046512..5e0dea77fd12 100644
--- a/pkg/ccl/backupccl/backup_destination.go
+++ b/pkg/ccl/backupccl/backup_destination.go
@@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
@@ -31,7 +32,7 @@ import (
// chain.
func fetchPreviousBackups(
ctx context.Context,
- user string,
+ user security.SQLUsername,
makeCloudStorage cloud.ExternalStorageFromURIFactory,
prevBackupURIs []string,
encryptionParams backupEncryptionParams,
@@ -70,7 +71,7 @@ func fetchPreviousBackups(
// can determine the latter from the former.
func resolveDest(
ctx context.Context,
- user string,
+ user security.SQLUsername,
nested, appendToLatest bool,
defaultURI string,
urisByLocalityKV map[string]string,
@@ -151,7 +152,7 @@ func resolveDest(
// getBackupManifests fetches the backup manifest from a list of backup URIs.
func getBackupManifests(
ctx context.Context,
- user string,
+ user security.SQLUsername,
makeCloudStorage cloud.ExternalStorageFromURIFactory,
backupURIs []string,
encryption *jobspb.BackupEncryptionOptions,
@@ -195,7 +196,7 @@ func getBackupManifests(
// base backups.
func getEncryptionFromBase(
ctx context.Context,
- user string,
+ user security.SQLUsername,
makeCloudStorage cloud.ExternalStorageFromURIFactory,
baseBackupURI string,
encryptionParams backupEncryptionParams,
@@ -236,7 +237,7 @@ func getEncryptionFromBase(
// should use for a backup that is pointing to a collection.
func resolveBackupCollection(
ctx context.Context,
- user string,
+ user security.SQLUsername,
defaultURI string,
appendToLatest bool,
makeCloudStorage cloud.ExternalStorageFromURIFactory,
diff --git a/pkg/ccl/backupccl/backup_destination_test.go b/pkg/ccl/backupccl/backup_destination_test.go
index 2c5e23ccd677..8687e47d4597 100644
--- a/pkg/ccl/backupccl/backup_destination_test.go
+++ b/pkg/ccl/backupccl/backup_destination_test.go
@@ -36,7 +36,7 @@ func newTestStorageFactory(t *testing.T) (cloud.ExternalStorageFromURIFactory, f
settings := cluster.MakeTestingClusterSettings()
settings.ExternalIODir = dir
clientFactory := blobs.TestBlobServiceClient(settings.ExternalIODir)
- externalStorageFromURI := func(ctx context.Context, uri, user string) (cloud.ExternalStorage,
+ externalStorageFromURI := func(ctx context.Context, uri string, user security.SQLUsername) (cloud.ExternalStorage,
error) {
conf, err := cloudimpl.ExternalStorageConfFromURI(uri, user)
require.NoError(t, err)
@@ -60,7 +60,7 @@ func TestBackupRestoreResolveDestination(t *testing.T) {
// writeManifest writes an empty backup manifest file to the given URI.
writeManifest := func(t *testing.T, uri string) {
- storage, err := externalStorageFromURI(ctx, uri, security.RootUser)
+ storage, err := externalStorageFromURI(ctx, uri, security.RootUserName())
defer storage.Close()
require.NoError(t, err)
require.NoError(t, storage.WriteFile(ctx, backupManifestName, emptyReader))
@@ -69,7 +69,7 @@ func TestBackupRestoreResolveDestination(t *testing.T) {
// writeLatest writes latestBackupSuffix to the LATEST file in the given
// collection.
writeLatest := func(t *testing.T, collectionURI, latestBackupSuffix string) {
- storage, err := externalStorageFromURI(ctx, collectionURI, security.RootUser)
+ storage, err := externalStorageFromURI(ctx, collectionURI, security.RootUserName())
defer storage.Close()
require.NoError(t, err)
require.NoError(t, storage.WriteFile(ctx, latestFileName, bytes.NewReader([]byte(latestBackupSuffix))))
@@ -136,7 +136,7 @@ func TestBackupRestoreResolveDestination(t *testing.T) {
require.NoError(t, err)
collectionURI, defaultURI, chosenSuffix, urisByLocalityKV, prevBackupURIs, err := resolveDest(
- ctx, security.RootUser,
+ ctx, security.RootUserName(),
false /* nested */, false, /* appendToLatest */
defaultDest, localitiesDest,
externalStorageFromURI, endTime,
@@ -205,7 +205,7 @@ func TestBackupRestoreResolveDestination(t *testing.T) {
dest, localitiesDest, err := getURIsByLocalityKV(to, "")
require.NoError(t, err)
collectionURI, defaultURI, chosenSuffix, urisByLocalityKV, prevBackupURIs, err := resolveDest(
- ctx, security.RootUser,
+ ctx, security.RootUserName(),
false /* nested */, false, /* appendToLatest */
dest, localitiesDest,
externalStorageFromURI, endTime,
@@ -308,7 +308,7 @@ func TestBackupRestoreResolveDestination(t *testing.T) {
defaultCollection, localityCollections, err := getURIsByLocalityKV(collectionTo, "")
require.NoError(t, err)
collectionURI, defaultURI, chosenSuffix, urisByLocalityKV, prevBackupURIs, err := resolveDest(
- ctx, security.RootUser,
+ ctx, security.RootUserName(),
true /* nested */, appendToLatest,
defaultCollection, localityCollections,
externalStorageFromURI, endTime,
diff --git a/pkg/ccl/backupccl/backup_job.go b/pkg/ccl/backupccl/backup_job.go
index 04c09341a320..726f0b72eb8f 100644
--- a/pkg/ccl/backupccl/backup_job.go
+++ b/pkg/ccl/backupccl/backup_job.go
@@ -609,7 +609,7 @@ func (b *backupResumer) maybeNotifyScheduledJobCompletion(
ctx,
"lookup-schedule-info",
txn,
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
fmt.Sprintf(
"SELECT created_by_id FROM %s WHERE id=$1 AND created_by_type=$2",
env.SystemJobsTableName()),
@@ -678,7 +678,7 @@ func (b *backupResumer) OnFailOrCancel(ctx context.Context, phs interface{}) err
}
func (b *backupResumer) deleteCheckpoint(
- ctx context.Context, cfg *sql.ExecutorConfig, user string,
+ ctx context.Context, cfg *sql.ExecutorConfig, user security.SQLUsername,
) {
// Attempt to delete BACKUP-CHECKPOINT.
if err := func() error {
diff --git a/pkg/ccl/backupccl/backup_processor.go b/pkg/ccl/backupccl/backup_processor.go
index 334f4cdbd27f..ff84afa76948 100644
--- a/pkg/ccl/backupccl/backup_processor.go
+++ b/pkg/ccl/backupccl/backup_processor.go
@@ -141,13 +141,13 @@ func runBackupProcessor(
// For all backups, partitioned or not, the main BACKUP manifest is stored at
// details.URI.
- defaultConf, err := cloudimpl.ExternalStorageConfFromURI(spec.DefaultURI, spec.User)
+ defaultConf, err := cloudimpl.ExternalStorageConfFromURI(spec.DefaultURI, spec.User())
if err != nil {
return err
}
storageByLocalityKV := make(map[string]*roachpb.ExternalStorage)
for kv, uri := range spec.URIsByLocalityKV {
- conf, err := cloudimpl.ExternalStorageConfFromURI(uri, spec.User)
+ conf, err := cloudimpl.ExternalStorageConfFromURI(uri, spec.User())
if err != nil {
return err
}
diff --git a/pkg/ccl/backupccl/backup_processor_planning.go b/pkg/ccl/backupccl/backup_processor_planning.go
index 68988089d104..26f6d3fcc8de 100644
--- a/pkg/ccl/backupccl/backup_processor_planning.go
+++ b/pkg/ccl/backupccl/backup_processor_planning.go
@@ -14,6 +14,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
roachpb "github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
@@ -136,7 +137,7 @@ func makeBackupDataProcessorSpecs(
mvccFilter roachpb.MVCCFilter,
encryption *jobspb.BackupEncryptionOptions,
startTime, endTime hlc.Timestamp,
- user string,
+ user security.SQLUsername,
execCfg *sql.ExecutorConfig,
) (map[roachpb.NodeID]*execinfrapb.BackupDataSpec, error) {
var spanPartitions []sql.SpanPartition
@@ -191,7 +192,7 @@ func makeBackupDataProcessorSpecs(
PKIDs: pkIDs,
BackupStartTime: startTime,
BackupEndTime: endTime,
- User: user,
+ UserProto: user.EncodeProto(),
}
nodeToSpec[partition.Node] = spec
}
@@ -212,7 +213,7 @@ func makeBackupDataProcessorSpecs(
PKIDs: pkIDs,
BackupStartTime: startTime,
BackupEndTime: endTime,
- User: user,
+ UserProto: user.EncodeProto(),
}
nodeToSpec[partition.Node] = spec
}
diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go
index 278c849f0517..37058a219650 100644
--- a/pkg/ccl/backupccl/backup_test.go
+++ b/pkg/ccl/backupccl/backup_test.go
@@ -598,7 +598,8 @@ func TestBackupRestoreAppend(t *testing.T) {
store, err := cloudimpl.ExternalStorageFromURI(ctx, "userfile:///0",
base.ExternalIODirConfig{},
tc.Servers[0].ClusterSettings(),
- blobs.TestEmptyBlobClientFactory, "root",
+ blobs.TestEmptyBlobClientFactory,
+ security.RootUserName(),
tc.Servers[0].InternalExecutor().(*sql.InternalExecutor), tc.Servers[0].DB())
require.NoError(t, err)
defer store.Close()
@@ -1094,7 +1095,7 @@ func TestBackupRestoreSystemJobs(t *testing.T) {
sqlDB.Exec(t, `BACKUP TABLE bank TO $1 INCREMENTAL FROM $2`, incDir, fullDir)
if err := jobutils.VerifySystemJob(t, sqlDB, 1, jobspb.TypeBackup, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf(
`BACKUP TABLE bank TO '%s' INCREMENTAL FROM '%s'`,
sanitizedIncDir+"redacted", sanitizedFullDir+"redacted",
@@ -1109,7 +1110,7 @@ func TestBackupRestoreSystemJobs(t *testing.T) {
sqlDB.Exec(t, `RESTORE TABLE bank FROM $1, $2 WITH OPTIONS (into_db='restoredb')`, fullDir, incDir)
if err := jobutils.VerifySystemJob(t, sqlDB, 0, jobspb.TypeRestore, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf(
`RESTORE TABLE bank FROM '%s', '%s' WITH into_db='restoredb'`,
sanitizedFullDir+"redacted", sanitizedIncDir+"redacted",
@@ -1195,7 +1196,7 @@ func TestEncryptedBackupRestoreSystemJobs(t *testing.T) {
// Verify the BACKUP job description is sanitized.
if err := jobutils.VerifySystemJob(t, sqlDB, 0, jobspb.TypeBackup, jobs.StatusSucceeded,
jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf(
`BACKUP DATABASE data TO '%s' WITH %s`,
backupLoc1, sanitizedEncryptionOption),
@@ -1213,7 +1214,7 @@ into_db='restoredb', %s)`, encryptionOption), backupLoc1)
// Verify the RESTORE job description is sanitized.
if err := jobutils.VerifySystemJob(t, sqlDB, 0, jobspb.TypeRestore, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf(
`RESTORE TABLE data.bank FROM '%s' WITH %s, into_db='restoredb'`,
backupLoc1, sanitizedEncryptionOption,
@@ -1419,7 +1420,7 @@ func createAndWaitForJob(
t.Helper()
now := timeutil.ToUnixMicros(timeutil.Now())
payload, err := protoutil.Marshal(&jobspb.Payload{
- Username: security.RootUser,
+ UsernameProto: security.RootUserName().EncodeProto(),
DescriptorIDs: descriptorIDs,
StartedMicros: now,
Details: jobspb.WrapPayloadDetails(details),
diff --git a/pkg/ccl/backupccl/create_scheduled_backup.go b/pkg/ccl/backupccl/create_scheduled_backup.go
index 482812d026b5..37d5da0f64ef 100644
--- a/pkg/ccl/backupccl/create_scheduled_backup.go
+++ b/pkg/ccl/backupccl/create_scheduled_backup.go
@@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/scheduledjobs"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
@@ -408,7 +409,7 @@ func checkForExistingBackupsInCollection(
func makeBackupSchedule(
env scheduledjobs.JobSchedulerEnv,
- owner string,
+ owner security.SQLUsername,
label string,
recurrence *scheduleRecurrence,
details jobspb.ScheduleDetails,
diff --git a/pkg/ccl/backupccl/create_scheduled_backup_test.go b/pkg/ccl/backupccl/create_scheduled_backup_test.go
index e7f085f7ec1c..ec83b6beeafb 100644
--- a/pkg/ccl/backupccl/create_scheduled_backup_test.go
+++ b/pkg/ccl/backupccl/create_scheduled_backup_test.go
@@ -140,7 +140,7 @@ func (h *testHelper) createBackupSchedule(
// Query system.scheduled_job table and load those schedules.
datums, cols, err := h.cfg.InternalExecutor.QueryWithCols(
context.Background(), "sched-load", nil,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SELECT * FROM system.scheduled_jobs WHERE schedule_id = $1",
id,
)
diff --git a/pkg/ccl/backupccl/manifest_handling.go b/pkg/ccl/backupccl/manifest_handling.go
index 78e7e3fd5459..d79ee25ee2b1 100644
--- a/pkg/ccl/backupccl/manifest_handling.go
+++ b/pkg/ccl/backupccl/manifest_handling.go
@@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/storageccl"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
@@ -102,7 +103,7 @@ func (r BackupFileDescriptors) Less(i, j int) bool {
func ReadBackupManifestFromURI(
ctx context.Context,
uri string,
- user string,
+ user security.SQLUsername,
makeExternalStorageFromURI cloud.ExternalStorageFromURIFactory,
encryption *jobspb.BackupEncryptionOptions,
) (BackupManifest, error) {
@@ -503,7 +504,7 @@ func writeTableStatistics(
func loadBackupManifests(
ctx context.Context,
uris []string,
- user string,
+ user security.SQLUsername,
makeExternalStorageFromURI cloud.ExternalStorageFromURIFactory,
encryption *jobspb.BackupEncryptionOptions,
) ([]BackupManifest, error) {
@@ -629,7 +630,7 @@ func resolveBackupManifests(
from [][]string,
endTime hlc.Timestamp,
encryption *jobspb.BackupEncryptionOptions,
- user string,
+ user security.SQLUsername,
) (
defaultURIs []string,
mainBackupManifests []BackupManifest,
@@ -1030,7 +1031,7 @@ func checkForPreviousBackup(
// enforce that this file be deleted.
func verifyWriteableDestination(
ctx context.Context,
- user string,
+ user security.SQLUsername,
makeCloudStorage cloud.ExternalStorageFromURIFactory,
baseURI string,
) error {
diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go
index 54b030f7d251..a738c65e1c7f 100644
--- a/pkg/ccl/backupccl/restore_job.go
+++ b/pkg/ccl/backupccl/restore_job.go
@@ -122,7 +122,7 @@ func makeImportSpans(
backups []BackupManifest,
backupLocalityInfo []jobspb.RestoreDetails_BackupLocalityInfo,
lowWaterMark roachpb.Key,
- user string,
+ user security.SQLUsername,
onMissing func(span covering.Range, start, end hlc.Timestamp) error,
) ([]execinfrapb.RestoreSpanEntry, hlc.Timestamp, error) {
// Put the covering for the already-completed spans into the
@@ -282,7 +282,7 @@ rangeLoop:
func WriteDescriptors(
ctx context.Context,
txn *kv.Txn,
- user string,
+ user security.SQLUsername,
descsCol *descs.Collection,
databases []catalog.DatabaseDescriptor,
schemas []catalog.SchemaDescriptor,
@@ -872,7 +872,7 @@ func createImportingDescriptors(
if details.DescriptorCoverage == tree.AllDescriptors {
tempSystemDBID := getTempSystemDBID(details)
databases = append(databases, dbdesc.NewInitial(tempSystemDBID, restoreTempSystemDB,
- security.AdminRole))
+ security.AdminRoleName()))
}
// We get the spans of the restoring tables _as they appear in the backup_,
@@ -1372,7 +1372,7 @@ func (r *restoreResumer) publishDescriptors(
// Convert any mutations that were in progress on the table descriptor
// when the backup was taken, and convert them to schema change jobs.
newJobs, err := createSchemaChangeJobsFromMutations(ctx,
- r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().Username, mutTable)
+ r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().UsernameProto.Decode(), mutTable)
if err != nil {
return newDescriptorChangeJobs, err
}
@@ -1391,7 +1391,7 @@ func (r *restoreResumer) publishDescriptors(
allMutDescs = append(allMutDescs, typ)
newTypes = append(newTypes, typ.TypeDesc())
if typ.HasPendingSchemaChanges() {
- typJob, err := createTypeChangeJobFromDesc(ctx, r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().Username, typ)
+ typJob, err := createTypeChangeJobFromDesc(ctx, r.execCfg.JobRegistry, r.execCfg.Codec, txn, r.job.Payload().UsernameProto.Decode(), typ)
if err != nil {
return newDescriptorChangeJobs, err
}
@@ -1585,7 +1585,7 @@ func (r *restoreResumer) dropDescriptors(
}
gcJobRecord := jobs.Record{
Description: fmt.Sprintf("GC for %s", r.job.Payload().Description),
- Username: r.job.Payload().Username,
+ Username: r.job.Payload().UsernameProto.Decode(),
DescriptorIDs: tablesToGC,
Details: gcDetails,
Progress: jobspb.SchemaChangeGCProgress{},
@@ -1769,7 +1769,7 @@ func getRestoringPrivileges(
ctx context.Context,
txn *kv.Txn,
desc catalog.Descriptor,
- user string,
+ user security.SQLUsername,
wroteDBs map[descpb.ID]catalog.DatabaseDescriptor,
descCoverage tree.DescriptorCoverage,
) (*descpb.PrivilegeDescriptor, error) {
diff --git a/pkg/ccl/backupccl/restore_schema_change_creation.go b/pkg/ccl/backupccl/restore_schema_change_creation.go
index b35e63e54ab9..ca80d8a2b4cc 100644
--- a/pkg/ccl/backupccl/restore_schema_change_creation.go
+++ b/pkg/ccl/backupccl/restore_schema_change_creation.go
@@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
@@ -121,7 +122,7 @@ func createTypeChangeJobFromDesc(
jr *jobs.Registry,
codec keys.SQLCodec,
txn *kv.Txn,
- username string,
+ username security.SQLUsername,
typ catalog.TypeDescriptor,
) (*jobs.StartableJob, error) {
record := jobs.Record{
@@ -151,7 +152,7 @@ func createSchemaChangeJobsFromMutations(
jr *jobs.Registry,
codec keys.SQLCodec,
txn *kv.Txn,
- username string,
+ username security.SQLUsername,
tableDesc *tabledesc.Mutable,
) ([]*jobs.StartableJob, error) {
mutationJobs := make([]descpb.TableDescriptor_MutationJob, 0, len(tableDesc.Mutations))
diff --git a/pkg/ccl/backupccl/show.go b/pkg/ccl/backupccl/show.go
index 6e66058ee177..d8aaca756340 100644
--- a/pkg/ccl/backupccl/show.go
+++ b/pkg/ccl/backupccl/show.go
@@ -401,7 +401,6 @@ func showPrivileges(descriptor *descpb.Descriptor) string {
return ""
}
for _, userPriv := range privDesc.Show(objectType) {
- user := userPriv.User
privs := userPriv.Privileges
if len(privs) == 0 {
continue
@@ -417,7 +416,7 @@ func showPrivileges(descriptor *descpb.Descriptor) string {
privStringBuilder.WriteString(" ON ")
privStringBuilder.WriteString(descpb.GetDescriptorName(descriptor))
privStringBuilder.WriteString(" TO ")
- privStringBuilder.WriteString(user)
+ privStringBuilder.WriteString(userPriv.User.SQLIdentifier())
privStringBuilder.WriteString("; ")
}
diff --git a/pkg/ccl/backupccl/targets_test.go b/pkg/ccl/backupccl/targets_test.go
index 715bed3ff1d8..5340531b0de9 100644
--- a/pkg/ccl/backupccl/targets_test.go
+++ b/pkg/ccl/backupccl/targets_test.go
@@ -54,7 +54,7 @@ func TestDescriptorsMatchingTargets(t *testing.T) {
return desc
}
mkDB := func(id descpb.ID, name string) *dbdesc.Immutable {
- return &dbdesc.NewInitial(id, name, security.AdminRole).Immutable
+ return &dbdesc.NewInitial(id, name, security.AdminRoleName()).Immutable
}
mkTyp := func(desc typDesc) *typedesc.Immutable {
// Set a default parent schema for the type descriptors.
diff --git a/pkg/ccl/changefeedccl/changefeed_dist.go b/pkg/ccl/changefeedccl/changefeed_dist.go
index 402763534ea4..96e7e1c5e749 100644
--- a/pkg/ccl/changefeedccl/changefeed_dist.go
+++ b/pkg/ccl/changefeedccl/changefeed_dist.go
@@ -141,9 +141,9 @@ func distChangefeedFlow(
corePlacement[i].NodeID = sp.Node
corePlacement[i].Core.ChangeAggregator = &execinfrapb.ChangeAggregatorSpec{
- Watches: watches,
- Feed: details,
- User: phs.User(),
+ Watches: watches,
+ Feed: details,
+ UserProto: phs.User().EncodeProto(),
}
}
// NB: This SpanFrontier processor depends on the set of tracked spans being
@@ -154,7 +154,7 @@ func distChangefeedFlow(
TrackedSpans: trackedSpans,
Feed: details,
JobID: jobID,
- User: phs.User(),
+ UserProto: phs.User().EncodeProto(),
}
p := sql.MakePhysicalPlan(gatewayNodeID)
diff --git a/pkg/ccl/changefeedccl/changefeed_processors.go b/pkg/ccl/changefeedccl/changefeed_processors.go
index 07a1a715bcaf..9f093c78518f 100644
--- a/pkg/ccl/changefeedccl/changefeed_processors.go
+++ b/pkg/ccl/changefeedccl/changefeed_processors.go
@@ -161,7 +161,7 @@ func (ca *changeAggregator) Start(ctx context.Context) context.Context {
if ca.sink, err = getSink(
ctx, ca.spec.Feed.SinkURI, nodeID, ca.spec.Feed.Opts, ca.spec.Feed.Targets,
- ca.flowCtx.Cfg.Settings, timestampOracle, ca.flowCtx.Cfg.ExternalStorageFromURI, ca.spec.User,
+ ca.flowCtx.Cfg.Settings, timestampOracle, ca.flowCtx.Cfg.ExternalStorageFromURI, ca.spec.User(),
); err != nil {
err = MarkRetryableError(err)
// Early abort in the case that there is an error creating the sink.
@@ -545,7 +545,7 @@ func (cf *changeFrontier) Start(ctx context.Context) context.Context {
var nilOracle timestampLowerBoundOracle
if cf.sink, err = getSink(
ctx, cf.spec.Feed.SinkURI, nodeID, cf.spec.Feed.Opts, cf.spec.Feed.Targets,
- cf.flowCtx.Cfg.Settings, nilOracle, cf.flowCtx.Cfg.ExternalStorageFromURI, cf.spec.User,
+ cf.flowCtx.Cfg.Settings, nilOracle, cf.flowCtx.Cfg.ExternalStorageFromURI, cf.spec.User(),
); err != nil {
err = MarkRetryableError(err)
cf.MoveToDraining(err)
diff --git a/pkg/ccl/changefeedccl/sink.go b/pkg/ccl/changefeedccl/sink.go
index c36fbbce9312..e00fd1eae76c 100644
--- a/pkg/ccl/changefeedccl/sink.go
+++ b/pkg/ccl/changefeedccl/sink.go
@@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
@@ -73,7 +74,7 @@ func getSink(
settings *cluster.Settings,
timestampOracle timestampLowerBoundOracle,
makeExternalStorageFromURI cloud.ExternalStorageFromURIFactory,
- user string,
+ user security.SQLUsername,
) (Sink, error) {
u, err := url.Parse(sinkURI)
if err != nil {
diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage.go b/pkg/ccl/changefeedccl/sink_cloudstorage.go
index 39b0d0e1f14c..8ece882d6e7b 100644
--- a/pkg/ccl/changefeedccl/sink_cloudstorage.go
+++ b/pkg/ccl/changefeedccl/sink_cloudstorage.go
@@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -306,7 +307,7 @@ func makeCloudStorageSink(
opts map[string]string,
timestampOracle timestampLowerBoundOracle,
makeExternalStorageFromURI cloud.ExternalStorageFromURIFactory,
- user string,
+ user security.SQLUsername,
) (Sink, error) {
// Date partitioning is pretty standard, so no override for now, but we could
// plumb one down if someone needs it.
diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go
index 913a48d1e9b1..189c685634f6 100644
--- a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go
+++ b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go
@@ -102,13 +102,13 @@ func TestCloudStorageSink(t *testing.T) {
require.NoError(t, err)
clientFactory := blobs.TestBlobServiceClient(settings.ExternalIODir)
- externalStorageFromURI := func(ctx context.Context, uri, user string) (cloud.ExternalStorage,
+ externalStorageFromURI := func(ctx context.Context, uri string, user security.SQLUsername) (cloud.ExternalStorage,
error) {
return cloudimpl.ExternalStorageFromURI(ctx, uri, base.ExternalIODirConfig{}, settings,
clientFactory, user, nil, nil)
}
- user := security.RootUser
+ user := security.RootUserName()
t.Run(`golden`, func(t *testing.T) {
t1 := tabledesc.NewImmutable(descpb.TableDescriptor{Name: `t1`})
diff --git a/pkg/ccl/cliccl/load.go b/pkg/ccl/cliccl/load.go
index 324200ef0f55..7b679634a9c6 100644
--- a/pkg/ccl/cliccl/load.go
+++ b/pkg/ccl/cliccl/load.go
@@ -62,7 +62,7 @@ func runLoadShow(cmd *cobra.Command, args []string) error {
}
externalStorageFromURI := func(ctx context.Context, uri string,
- user string) (cloud.ExternalStorage, error) {
+ user security.SQLUsername) (cloud.ExternalStorage, error) {
return cloudimpl.ExternalStorageFromURI(ctx, uri, base.ExternalIODirConfig{},
cluster.NoSettings, blobs.TestEmptyBlobClientFactory, user, nil, nil)
}
@@ -70,7 +70,7 @@ func runLoadShow(cmd *cobra.Command, args []string) error {
// upgraded from the old FK representation, or even older formats). If more
// fields are added to the output, the table descriptors may need to be
// upgraded.
- desc, err := backupccl.ReadBackupManifestFromURI(ctx, basepath, security.RootUser,
+ desc, err := backupccl.ReadBackupManifestFromURI(ctx, basepath, security.RootUserName(),
externalStorageFromURI, nil)
if err != nil {
return err
diff --git a/pkg/ccl/gssapiccl/gssapi.go b/pkg/ccl/gssapiccl/gssapi.go
index bd2aece6f229..8790b78b9b61 100644
--- a/pkg/ccl/gssapiccl/gssapi.go
+++ b/pkg/ccl/gssapiccl/gssapi.go
@@ -51,7 +51,7 @@ func authGSS(
execCfg *sql.ExecutorConfig,
entry *hba.Entry,
) (security.UserAuthHook, error) {
- return func(requestedUser string, clientConnection bool) (func(), error) {
+ return func(requestedUser security.SQLUsername, clientConnection bool) (func(), error) {
var (
majStat, minStat, lminS, gflags C.OM_uint32
gbuf C.gss_buffer_desc
@@ -153,7 +153,8 @@ func authGSS(
return connClose, errors.New("GSSAPI did not return realm but realm matching was requested")
}
- if !strings.EqualFold(gssUser, requestedUser) {
+ gssUsername, _ := security.MakeSQLUsernameFromUserInput(gssUser, security.UsernameValidation)
+ if gssUsername != requestedUser {
return connClose, errors.Errorf("requested user is %s, but GSSAPI auth is for %s", requestedUser, gssUser)
}
diff --git a/pkg/ccl/importccl/exportcsv.go b/pkg/ccl/importccl/exportcsv.go
index d7f46a497c3d..e1fdfa63a1cd 100644
--- a/pkg/ccl/importccl/exportcsv.go
+++ b/pkg/ccl/importccl/exportcsv.go
@@ -234,7 +234,7 @@ func (sp *csvWriter) Run(ctx context.Context) {
return errors.Wrap(err, "failed to flush csv writer")
}
- conf, err := cloudimpl.ExternalStorageConfFromURI(sp.spec.Destination, sp.spec.User)
+ conf, err := cloudimpl.ExternalStorageConfFromURI(sp.spec.Destination, sp.spec.User())
if err != nil {
return err
}
diff --git a/pkg/ccl/importccl/import_processor.go b/pkg/ccl/importccl/import_processor.go
index d8150c05ff0e..b876c45cc396 100644
--- a/pkg/ccl/importccl/import_processor.go
+++ b/pkg/ccl/importccl/import_processor.go
@@ -151,7 +151,7 @@ func makeInputConverter(
case roachpb.IOFileFormat_CSV:
isWorkload := true
for _, file := range spec.Uri {
- if conf, err := cloudimpl.ExternalStorageConfFromURI(file, spec.User); err != nil || conf.Provider != roachpb.ExternalStorageProvider_Workload {
+ if conf, err := cloudimpl.ExternalStorageConfFromURI(file, spec.User()); err != nil || conf.Provider != roachpb.ExternalStorageProvider_Workload {
isWorkload = false
break
}
diff --git a/pkg/ccl/importccl/import_processor_test.go b/pkg/ccl/importccl/import_processor_test.go
index dc43473fe898..6de1e8a5a3ce 100644
--- a/pkg/ccl/importccl/import_processor_test.go
+++ b/pkg/ccl/importccl/import_processor_test.go
@@ -125,7 +125,7 @@ func TestConverterFlushesBatches(t *testing.T) {
group.Go(func() error {
defer close(kvCh)
return conv.readFiles(ctx, testCase.inputs, nil, converterSpec.Format,
- externalStorageFactory, security.RootUser)
+ externalStorageFactory, security.RootUserName())
})
lastBatch := 0
diff --git a/pkg/ccl/importccl/import_stmt.go b/pkg/ccl/importccl/import_stmt.go
index 41cea4601c7c..41175d01b47d 100644
--- a/pkg/ccl/importccl/import_stmt.go
+++ b/pkg/ccl/importccl/import_stmt.go
@@ -1651,7 +1651,7 @@ func (r *importResumer) dropTables(
}
gcJobRecord := jobs.Record{
Description: fmt.Sprintf("GC for %s", r.job.Payload().Description),
- Username: r.job.Payload().Username,
+ Username: r.job.Payload().UsernameProto.Decode(),
DescriptorIDs: tablesToGC,
Details: gcDetails,
Progress: jobspb.SchemaChangeGCProgress{},
diff --git a/pkg/ccl/importccl/import_stmt_test.go b/pkg/ccl/importccl/import_stmt_test.go
index 3a02c0fd3dd9..06e01baf0edd 100644
--- a/pkg/ccl/importccl/import_stmt_test.go
+++ b/pkg/ccl/importccl/import_stmt_test.go
@@ -1651,7 +1651,7 @@ func TestImportCSVStmt(t *testing.T) {
jobPrefix := fmt.Sprintf(`IMPORT TABLE %s.public.t (a INT8 PRIMARY KEY, b STRING, INDEX (b), INDEX (a, b))`, intodb)
if err := jobutils.VerifySystemJob(t, sqlDB, testNum, jobspb.TypeImport, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf(jobPrefix+` CSV DATA (%s)`+tc.jobOpts, strings.ReplaceAll(strings.Join(tc.files, ", "), "?AWS_SESSION_TOKEN=secrets", "?AWS_SESSION_TOKEN=redacted")),
}); err != nil {
t.Fatal(err)
@@ -1914,7 +1914,7 @@ b STRING) CSV DATA (%s)`, testFiles.files[0])); err != nil {
t.Run("userfile-simple", func(t *testing.T) {
userfileURI := "userfile://defaultdb.public.root/test.csv"
userfileStorage, err := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).DistSQLSrv.
- ExternalStorageFromURI(ctx, userfileURI, security.RootUser)
+ ExternalStorageFromURI(ctx, userfileURI, security.RootUserName())
require.NoError(t, err)
data := []byte("1,2")
@@ -1930,7 +1930,7 @@ b STRING) CSV DATA (%s)`, testFiles.files[0])); err != nil {
t.Run("userfile-relative-file-path", func(t *testing.T) {
userfileURI := "userfile:///import-test/employees.csv"
userfileStorage, err := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).DistSQLSrv.
- ExternalStorageFromURI(ctx, userfileURI, security.RootUser)
+ ExternalStorageFromURI(ctx, userfileURI, security.RootUserName())
require.NoError(t, err)
data := []byte("1,2")
@@ -2007,7 +2007,7 @@ func TestImportObjectLevelRBAC(t *testing.T) {
// Write to userfile storage now that testuser has CREATE privileges.
ie := tc.Server(0).InternalExecutor().(*sql.InternalExecutor)
fileTableSystem1, err := cloudimpl.ExternalStorageFromURI(ctx, dest, base.ExternalIODirConfig{},
- cluster.NoSettings, blobs.TestEmptyBlobClientFactory, "testuser", ie, tc.Server(0).DB())
+ cluster.NoSettings, blobs.TestEmptyBlobClientFactory, security.TestUserName(), ie, tc.Server(0).DB())
require.NoError(t, err)
require.NoError(t, fileTableSystem1.WriteFile(ctx, filename, bytes.NewReader([]byte("1,aaa"))))
}
@@ -2502,7 +2502,7 @@ func TestImportIntoCSV(t *testing.T) {
jobPrefix := fmt.Sprintf(`IMPORT INTO defaultdb.public.t(a, b)`)
if err := jobutils.VerifySystemJob(t, sqlDB, testNum, jobspb.TypeImport, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf(jobPrefix+` CSV DATA (%s)`+tc.jobOpts, strings.ReplaceAll(strings.Join(tc.files, ", "), "?AWS_SESSION_TOKEN=secrets", "?AWS_SESSION_TOKEN=redacted")),
}); err != nil {
t.Fatal(err)
@@ -3034,7 +3034,7 @@ func TestImportIntoCSV(t *testing.T) {
t.Run("import-into-userfile-simple", func(t *testing.T) {
userfileURI := "userfile://defaultdb.public.root/test.csv"
userfileStorage, err := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).DistSQLSrv.
- ExternalStorageFromURI(ctx, userfileURI, security.RootUser)
+ ExternalStorageFromURI(ctx, userfileURI, security.RootUserName())
require.NoError(t, err)
data := []byte("1,2")
@@ -3101,7 +3101,7 @@ func benchUserUpload(b *testing.B, uploadBaseURI string) {
} else if uri.Scheme == "userfile" {
// Write the test data to userfile storage.
userfileStorage, err := tc.Server(0).ExecutorConfig().(sql.ExecutorConfig).DistSQLSrv.
- ExternalStorageFromURI(ctx, uploadBaseURI+testFileBase, security.RootUser)
+ ExternalStorageFromURI(ctx, uploadBaseURI+testFileBase, security.RootUserName())
require.NoError(b, err)
content, err := ioutil.ReadAll(r)
require.NoError(b, err)
@@ -4150,7 +4150,7 @@ func TestImportControlJobRBAC(t *testing.T) {
t.Run(tc.name, func(t *testing.T) {
// Start import job as root.
rootJobRecord := defaultRecord
- rootJobRecord.Username = "root"
+ rootJobRecord.Username = security.RootUserName()
rootJob := startLeasedJob(t, rootJobRecord)
// Test root can control root job.
@@ -4159,7 +4159,7 @@ func TestImportControlJobRBAC(t *testing.T) {
// Start import job as non-admin user.
nonAdminJobRecord := defaultRecord
- nonAdminJobRecord.Username = "testuser"
+ nonAdminJobRecord.Username = security.TestUserName()
userJob := startLeasedJob(t, nonAdminJobRecord)
// Test testuser can control testuser job.
diff --git a/pkg/ccl/importccl/import_table_creation.go b/pkg/ccl/importccl/import_table_creation.go
index 112e7b7cee2d..6c1b41f6348f 100644
--- a/pkg/ccl/importccl/import_table_creation.go
+++ b/pkg/ccl/importccl/import_table_creation.go
@@ -44,7 +44,7 @@ func readCreateTableFromStore(
ctx context.Context,
filename string,
externalStorageFromURI cloud.ExternalStorageFromURIFactory,
- user string,
+ user security.SQLUsername,
) (*tree.CreateTable, error) {
store, err := externalStorageFromURI(ctx, filename, user)
if err != nil {
@@ -159,7 +159,7 @@ func MakeSimpleTableDescriptor(
parentSchemaID,
tableID,
hlc.Timestamp{WallTime: walltime},
- descpb.NewDefaultPrivilegeDescriptor(security.AdminRole),
+ descpb.NewDefaultPrivilegeDescriptor(security.AdminRoleName()),
affected,
semaCtx,
&evalCtx,
diff --git a/pkg/ccl/importccl/read_import_avro.go b/pkg/ccl/importccl/read_import_avro.go
index de677e142753..ab7b38fe879a 100644
--- a/pkg/ccl/importccl/read_import_avro.go
+++ b/pkg/ccl/importccl/read_import_avro.go
@@ -17,8 +17,9 @@ import (
"unicode/utf8"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
- "github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
@@ -174,7 +175,7 @@ func (a *avroConsumer) convertNative(x interface{}, conv *row.DatumRowConverter)
}
for f, v := range record {
- field := lex.NormalizeName(f)
+ field := lexbase.NormalizeName(f)
idx, ok := a.fieldNameToIdx[field]
if !ok {
if a.strict {
@@ -480,7 +481,7 @@ func (a *avroInputReader) readFiles(
resumePos map[int32]int64,
format roachpb.IOFileFormat,
makeExternalStorage cloud.ExternalStorageFactory,
- user string,
+ user security.SQLUsername,
) error {
return readInputFiles(ctx, dataFiles, resumePos, format, a.readFile, makeExternalStorage, user)
}
diff --git a/pkg/ccl/importccl/read_import_base.go b/pkg/ccl/importccl/read_import_base.go
index 86f2fe602a74..6cdc2781fc55 100644
--- a/pkg/ccl/importccl/read_import_base.go
+++ b/pkg/ccl/importccl/read_import_base.go
@@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
@@ -99,7 +100,7 @@ func runImport(
}
return conv.readFiles(ctx, inputs, spec.ResumePos, spec.Format, flowCtx.Cfg.ExternalStorage,
- spec.User)
+ spec.User())
})
// Ingest the KVs that the producer group emitted to the chan and the row result
@@ -149,7 +150,7 @@ func readInputFiles(
format roachpb.IOFileFormat,
fileFunc readFileFunc,
makeExternalStorage cloud.ExternalStorageFactory,
- user string,
+ user security.SQLUsername,
) error {
done := ctx.Done()
@@ -342,7 +343,7 @@ func (f fileReader) ReadFraction() float32 {
type inputConverter interface {
start(group ctxgroup.Group)
readFiles(ctx context.Context, dataFiles map[int32]string, resumePos map[int32]int64,
- format roachpb.IOFileFormat, makeExternalStorage cloud.ExternalStorageFactory, user string) error
+ format roachpb.IOFileFormat, makeExternalStorage cloud.ExternalStorageFactory, user security.SQLUsername) error
}
func isMultiTableFormat(format roachpb.IOFileFormat_FileFormat) bool {
diff --git a/pkg/ccl/importccl/read_import_csv.go b/pkg/ccl/importccl/read_import_csv.go
index d3517ad57f67..36fa7c51788b 100644
--- a/pkg/ccl/importccl/read_import_csv.go
+++ b/pkg/ccl/importccl/read_import_csv.go
@@ -14,6 +14,7 @@ import (
"strings"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
@@ -62,7 +63,7 @@ func (c *csvInputReader) readFiles(
resumePos map[int32]int64,
format roachpb.IOFileFormat,
makeExternalStorage cloud.ExternalStorageFactory,
- user string,
+ user security.SQLUsername,
) error {
return readInputFiles(ctx, dataFiles, resumePos, format, c.readFile, makeExternalStorage, user)
}
diff --git a/pkg/ccl/importccl/read_import_mysql.go b/pkg/ccl/importccl/read_import_mysql.go
index 32d3d361974a..77c7ec018172 100644
--- a/pkg/ccl/importccl/read_import_mysql.go
+++ b/pkg/ccl/importccl/read_import_mysql.go
@@ -24,7 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
- "github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
@@ -91,7 +91,7 @@ func (m *mysqldumpReader) readFiles(
resumePos map[int32]int64,
format roachpb.IOFileFormat,
makeExternalStorage cloud.ExternalStorageFactory,
- user string,
+ user security.SQLUsername,
) error {
return readInputFiles(ctx, dataFiles, resumePos, format, m.readFile, makeExternalStorage, user)
}
@@ -126,7 +126,7 @@ func (m *mysqldumpReader) readFile(
switch i := stmt.(type) {
case *mysql.Insert:
name := safeString(i.Table.Name)
- conv, ok := m.tables[lex.NormalizeName(name)]
+ conv, ok := m.tables[lexbase.NormalizeName(name)]
if !ok {
// not importing this table.
continue
@@ -291,7 +291,7 @@ func readMysqlCreateTable(
fks fkHandler,
seqVals map[descpb.ID]int64,
) ([]*tabledesc.Mutable, error) {
- match = lex.NormalizeName(match)
+ match = lexbase.NormalizeName(match)
r := bufio.NewReaderSize(input, 1024*64)
tokens := mysql.NewTokenizer(r)
tokens.SkipSpecialComments = true
@@ -348,7 +348,7 @@ func readMysqlCreateTable(
type mysqlIdent interface{ CompliantName() string }
func safeString(in mysqlIdent) string {
- return lex.NormalizeName(in.CompliantName())
+ return lexbase.NormalizeName(in.CompliantName())
}
func safeName(in mysqlIdent) tree.Name {
@@ -400,7 +400,7 @@ func mysqlTableToCockroach(
var seqDesc *tabledesc.Mutable
// If we have an auto-increment seq, create it and increment the id.
- owner := security.AdminRole
+ owner := security.AdminRoleName()
if seqName != "" {
var opts tree.SequenceOptions
if startingValue != 0 {
@@ -411,7 +411,7 @@ func mysqlTableToCockroach(
if p != nil {
params := p.RunParams(ctx)
if params.SessionData() != nil {
- owner = params.SessionData().User
+ owner = params.SessionData().User()
}
priv := descpb.NewDefaultPrivilegeDescriptor(owner)
seqDesc, err = sql.NewSequenceTableDesc(
@@ -515,7 +515,7 @@ func mysqlTableToCockroach(
)
toCols := i.ReferencedColumns
d := &tree.ForeignKeyConstraintTableDef{
- Name: tree.Name(lex.NormalizeName(raw.Name)),
+ Name: tree.Name(lexbase.NormalizeName(raw.Name)),
FromCols: toNameList(fromCols),
ToCols: toNameList(toCols),
}
diff --git a/pkg/ccl/importccl/read_import_mysqlout.go b/pkg/ccl/importccl/read_import_mysqlout.go
index 626fd8da62ad..8d4c7ee596a3 100644
--- a/pkg/ccl/importccl/read_import_mysqlout.go
+++ b/pkg/ccl/importccl/read_import_mysqlout.go
@@ -16,6 +16,7 @@ import (
"unicode"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
@@ -63,7 +64,7 @@ func (d *mysqloutfileReader) readFiles(
resumePos map[int32]int64,
format roachpb.IOFileFormat,
makeExternalStorage cloud.ExternalStorageFactory,
- user string,
+ user security.SQLUsername,
) error {
return readInputFiles(ctx, dataFiles, resumePos, format, d.readFile, makeExternalStorage, user)
}
diff --git a/pkg/ccl/importccl/read_import_pgcopy.go b/pkg/ccl/importccl/read_import_pgcopy.go
index bb878eb80024..9906fbc77f1e 100644
--- a/pkg/ccl/importccl/read_import_pgcopy.go
+++ b/pkg/ccl/importccl/read_import_pgcopy.go
@@ -18,6 +18,7 @@ import (
"unicode"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
@@ -69,7 +70,7 @@ func (d *pgCopyReader) readFiles(
resumePos map[int32]int64,
format roachpb.IOFileFormat,
makeExternalStorage cloud.ExternalStorageFactory,
- user string,
+ user security.SQLUsername,
) error {
return readInputFiles(ctx, dataFiles, resumePos, format, d.readFile, makeExternalStorage, user)
}
diff --git a/pkg/ccl/importccl/read_import_pgdump.go b/pkg/ccl/importccl/read_import_pgdump.go
index 37cb926e1c26..56165d455bef 100644
--- a/pkg/ccl/importccl/read_import_pgdump.go
+++ b/pkg/ccl/importccl/read_import_pgdump.go
@@ -241,9 +241,9 @@ func readPostgresCreateTable(
stmt, err := ps.Next()
if err == io.EOF {
ret := make([]*tabledesc.Mutable, 0, len(createTbl))
- owner := security.AdminRole
+ owner := security.AdminRoleName()
if params.SessionData() != nil {
- owner = params.SessionData().User
+ owner = params.SessionData().User()
}
for name, seq := range createSeq {
id := descpb.ID(int(defaultCSVTableID) + len(ret))
@@ -614,7 +614,7 @@ func (m *pgDumpReader) readFiles(
resumePos map[int32]int64,
format roachpb.IOFileFormat,
makeExternalStorage cloud.ExternalStorageFactory,
- user string,
+ user security.SQLUsername,
) error {
return readInputFiles(ctx, dataFiles, resumePos, format, m.readFile, makeExternalStorage, user)
}
diff --git a/pkg/ccl/importccl/read_import_workload.go b/pkg/ccl/importccl/read_import_workload.go
index e0b8e5d0d552..e4ea324d1a4e 100644
--- a/pkg/ccl/importccl/read_import_workload.go
+++ b/pkg/ccl/importccl/read_import_workload.go
@@ -19,6 +19,7 @@ import (
"github.com/cockroachdb/apd/v2"
"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/row"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
@@ -118,7 +119,7 @@ func (w *workloadReader) readFiles(
_ map[int32]int64,
_ roachpb.IOFileFormat,
_ cloud.ExternalStorageFactory,
- _ string,
+ _ security.SQLUsername,
) error {
wcs := make([]*WorkloadKVConverter, 0, len(dataFiles))
diff --git a/pkg/ccl/importccl/testutils_test.go b/pkg/ccl/importccl/testutils_test.go
index 3c1b7c1d126a..42a3a11bc8b0 100644
--- a/pkg/ccl/importccl/testutils_test.go
+++ b/pkg/ccl/importccl/testutils_test.go
@@ -53,7 +53,7 @@ func descForTable(
name := parsed[0].AST.(*tree.CreateSequence).Name.String()
ts := hlc.Timestamp{WallTime: nanos}
- priv := descpb.NewDefaultPrivilegeDescriptor(security.AdminRole)
+ priv := descpb.NewDefaultPrivilegeDescriptor(security.AdminRoleName())
desc, err := sql.NewSequenceTableDesc(
ctx,
name,
diff --git a/pkg/ccl/oidcccl/authentication_oidc_test.go b/pkg/ccl/oidcccl/authentication_oidc_test.go
index bf619f56a091..08beb7c01afc 100644
--- a/pkg/ccl/oidcccl/authentication_oidc_test.go
+++ b/pkg/ccl/oidcccl/authentication_oidc_test.go
@@ -62,7 +62,7 @@ func TestOIDCBadRequestIfDisabled(t *testing.T) {
})
}
- plainHTTPCfg := testutils.NewTestBaseContext(server.TestUser)
+ plainHTTPCfg := testutils.NewTestBaseContext(security.TestUserName())
testCertsContext := newRPCContext(plainHTTPCfg)
client, err := testCertsContext.GetHTTPClient()
@@ -179,7 +179,7 @@ func TestOIDCEnabled(t *testing.T) {
sqlDB.Exec(t, `SET CLUSTER SETTING server.oidc_authentication.redirect_url = "https://cockroachlabs.com/oidc/v1/callback"`)
sqlDB.Exec(t, `SET CLUSTER SETTING server.oidc_authentication.enabled = "true"`)
- plainHTTPCfg := testutils.NewTestBaseContext(server.TestUser)
+ plainHTTPCfg := testutils.NewTestBaseContext(security.TestUserName())
testCertsContext := newRPCContext(plainHTTPCfg)
client, err := testCertsContext.GetHTTPClient()
diff --git a/pkg/ccl/serverccl/role_authentication_test.go b/pkg/ccl/serverccl/role_authentication_test.go
index b4efca2d81fa..9ad4a7e4486f 100644
--- a/pkg/ccl/serverccl/role_authentication_test.go
+++ b/pkg/ccl/serverccl/role_authentication_test.go
@@ -113,7 +113,8 @@ func TestVerifyPassword(t *testing.T) {
{"cthon98", "12345", true, ""},
} {
t.Run("", func(t *testing.T) {
- exists, canLogin, pwRetrieveFn, validUntilFn, err := sql.GetUserHashedPassword(context.Background(), &ie, tc.username)
+ username := security.MakeSQLUsernameFromPreNormalizedString(tc.username)
+ exists, canLogin, pwRetrieveFn, validUntilFn, err := sql.GetUserHashedPassword(context.Background(), &ie, username)
if err != nil {
t.Errorf(
diff --git a/pkg/ccl/storageccl/import_test.go b/pkg/ccl/storageccl/import_test.go
index 172fb79a42ea..89f3b5ea1d6a 100644
--- a/pkg/ccl/storageccl/import_test.go
+++ b/pkg/ccl/storageccl/import_test.go
@@ -234,7 +234,7 @@ func runTestImport(t *testing.T, init func(*cluster.Settings)) {
defer s.Stopper().Stop(ctx)
init(s.ClusterSettings())
- storage, err := cloudimpl.ExternalStorageConfFromURI("nodelocal://0/foo", security.RootUser)
+ storage, err := cloudimpl.ExternalStorageConfFromURI("nodelocal://0/foo", security.RootUserName())
if err != nil {
t.Fatalf("%+v", err)
}
diff --git a/pkg/cli/cert.go b/pkg/cli/cert.go
index 71e8e30f5bb9..fd2477a19936 100644
--- a/pkg/cli/cert.go
+++ b/pkg/cli/cert.go
@@ -17,7 +17,6 @@ import (
"time"
"github.com/cockroachdb/cockroach/pkg/security"
- "github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/errors"
"github.com/spf13/cobra"
)
@@ -180,10 +179,8 @@ Creation fails if the CA expiration time is before the desired certificate expir
// TODO(marc): there is currently no way to specify which CA cert to use if more
// than one if present.
func runCreateClientCert(cmd *cobra.Command, args []string) error {
- var err error
- var username string
- // We intentionally allow the `node` user to have a cert.
- if username, err = sql.NormalizeAndValidateUsernameNoBlocklist(args[0]); err != nil {
+ username, err := security.MakeSQLUsernameFromUserInput(args[0], security.UsernameCreation)
+ if err != nil {
return errors.Wrap(err, "failed to generate client certificate and key")
}
diff --git a/pkg/cli/cli_test.go b/pkg/cli/cli_test.go
index 90c997c3696d..84180af01cd1 100644
--- a/pkg/cli/cli_test.go
+++ b/pkg/cli/cli_test.go
@@ -159,7 +159,7 @@ func newCLITest(params cliTestParams) cliTest {
log.Infof(context.Background(), "SQL listener at %s", c.ServingSQLAddr())
}
- baseCfg.User = security.NodeUser
+ baseCfg.User = security.NodeUserName()
// Ensure that CLI error messages and anything meant for the
// original stderr is redirected to stdout, where it can be
@@ -1391,8 +1391,7 @@ func Example_cert() {
// cert create-client Ομηρος
// cert create-client 0foo
// cert create-client ,foo
- // ERROR: failed to generate client certificate and key: username ",foo" invalid
- // SQLSTATE: 42602
+ // ERROR: failed to generate client certificate and key: username is invalid
// HINT: Usernames are case insensitive, must start with a letter, digit or underscore, may contain letters, digits, dashes, periods, or underscores, and must not exceed 63 characters.
}
diff --git a/pkg/cli/client_url.go b/pkg/cli/client_url.go
index cd63a1e1c99c..23068115a1a0 100644
--- a/pkg/cli/client_url.go
+++ b/pkg/cli/client_url.go
@@ -295,9 +295,9 @@ func (u urlParser) setInternal(v string, warn bool) error {
return nil
}
- userName := security.RootUser
+ userName := security.RootUserName()
if cliCtx.sqlConnUser != "" {
- userName = cliCtx.sqlConnUser
+ userName, _ = security.MakeSQLUsernameFromUserInput(cliCtx.sqlConnUser, security.UsernameValidation)
}
if err := tryCertsDir("sslrootcert", security.CACertFilename()); err != nil {
return err
@@ -353,9 +353,9 @@ func (cliCtx *cliContext) makeClientConnURL() (url.URL, error) {
if netHost != "" {
// Only add TLS parameters when using a network connection.
- userName := cliCtx.sqlConnUser
- if userName == "" {
- userName = security.RootUser
+ userName, _ := security.MakeSQLUsernameFromUserInput(cliCtx.sqlConnUser, security.UsernameValidation)
+ if userName.Undefined() {
+ userName = security.RootUserName()
}
sCtx := rpc.MakeSecurityContext(cliCtx.Config, security.CommandTLSSettings{}, roachpb.SystemTenantID)
if err := sCtx.LoadSecurityOptions(
diff --git a/pkg/cli/demo_cluster.go b/pkg/cli/demo_cluster.go
index aecb211fa647..bc95f3bb8f2b 100644
--- a/pkg/cli/demo_cluster.go
+++ b/pkg/cli/demo_cluster.go
@@ -53,7 +53,7 @@ type transientCluster struct {
servers []*server.TestServer
adminPassword string
- adminUser string
+ adminUser security.SQLUsername
}
func (c *transientCluster) checkConfigAndSetupLogging(
@@ -275,10 +275,10 @@ func (c *transientCluster) start(
if err != nil {
return err
}
- c.adminUser = demoUsername
+ c.adminUser = security.MakeSQLUsernameFromPreNormalizedString(demoUsername)
c.adminPassword = demoPassword
if demoCtx.insecure {
- c.adminUser = security.RootUser
+ c.adminUser = security.RootUserName()
c.adminPassword = "unused"
}
@@ -546,7 +546,7 @@ func generateCerts(certsDir string) (err error) {
defaultKeySize,
defaultCertLifetime,
false, /* overwrite */
- security.RootUser,
+ security.RootUserName(),
false, /* generatePKCS8Key */
)
}
@@ -574,7 +574,7 @@ func (c *transientCluster) getNetworkURLForServer(
sqlURL.User = url.User(security.RootUser)
options.Add("sslmode", "disable")
} else {
- sqlURL.User = url.UserPassword(c.adminUser, c.adminPassword)
+ sqlURL.User = url.UserPassword(c.adminUser.Normalized(), c.adminPassword)
options.Add("sslmode", "require")
}
sqlURL.RawQuery = options.Encode()
@@ -758,7 +758,7 @@ func (c *transientCluster) sockForServer(nodeID roachpb.NodeID) unixSocketDetail
type unixSocketDetails struct {
socketDir string
portNumber int
- username string
+ username security.SQLUsername
password string
}
@@ -784,7 +784,7 @@ func (s unixSocketDetails) String() string {
// mode the password is not checked on the server.
sqlURL := url.URL{
Scheme: "postgres",
- User: url.UserPassword(s.username, s.password),
+ User: url.UserPassword(s.username.Normalized(), s.password),
RawQuery: options.Encode(),
}
return sqlURL.String()
diff --git a/pkg/cli/flags.go b/pkg/cli/flags.go
index 962092136389..6114ef7eeaf9 100644
--- a/pkg/cli/flags.go
+++ b/pkg/cli/flags.go
@@ -928,7 +928,7 @@ func extraServerFlagInit(cmd *cobra.Command) error {
if err := security.SetCertPrincipalMap(startCtx.serverCertPrincipalMap); err != nil {
return err
}
- serverCfg.User = security.NodeUser
+ serverCfg.User = security.NodeUserName()
serverCfg.Insecure = startCtx.serverInsecure
serverCfg.SSLCertsDir = startCtx.serverSSLCertsDir
diff --git a/pkg/cli/userfile.go b/pkg/cli/userfile.go
index bc75dff4dc43..edecbb3eab6f 100644
--- a/pkg/cli/userfile.go
+++ b/pkg/cli/userfile.go
@@ -20,8 +20,10 @@ import (
"path"
"strings"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql"
+ "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/errors"
@@ -153,15 +155,17 @@ func openUserFile(source string) (io.ReadCloser, error) {
}
// Construct the userfile ExternalStorage URI from CLI args.
-func constructUserfileDestinationURI(source, destination, user string) string {
+func constructUserfileDestinationURI(source, destination string, user security.SQLUsername) string {
// User has not specified a destination URI/path. We use the default URI
// scheme and host, and the basename from the source arg as the path.
if destination == "" {
sourceFilename := path.Base(source)
userFileURL := url.URL{
Scheme: defaultUserfileScheme,
- Host: defaultQualifiedNamePrefix + user,
- Path: sourceFilename,
+ // TODO(knz): This looks suspicious, see
+ // https://github.com/cockroachdb/cockroach/issues/55389
+ Host: defaultQualifiedNamePrefix + user.Normalized(),
+ Path: sourceFilename,
}
return userFileURL.String()
}
@@ -177,7 +181,9 @@ func constructUserfileDestinationURI(source, destination, user string) string {
if userfileURI, err = url.ParseRequestURI(destination); err == nil {
if userfileURI.Scheme == defaultUserfileScheme {
if userfileURI.Host == "" {
- userfileURI.Host = defaultQualifiedNamePrefix + user
+ // TODO(knz): This looks suspicious, see
+ // https://github.com/cockroachdb/cockroach/issues/55389
+ userfileURI.Host = defaultQualifiedNamePrefix + user.Normalized()
}
return userfileURI.String()
}
@@ -187,8 +193,10 @@ func constructUserfileDestinationURI(source, destination, user string) string {
// userfile URI schema and host, and the destination as the path.
userFileURL := url.URL{
Scheme: defaultUserfileScheme,
- Host: defaultQualifiedNamePrefix + user,
- Path: destination,
+ // TODO(knz): This looks suspicious, see
+ // https://github.com/cockroachdb/cockroach/issues/55389
+ Host: defaultQualifiedNamePrefix + user.Normalized(),
+ Path: destination,
}
return userFileURL.String()
}
@@ -241,8 +249,9 @@ func listUserFile(ctx context.Context, conn *sqlConn, glob string) error {
return err
}
- userFileTableConf, err := cloudimpl.ExternalStorageConfFromURI(unescapedUserfileListURI,
- connURL.User.Username())
+ reqUsername, _ := security.MakeSQLUsernameFromUserInput(connURL.User.Username(), security.UsernameValidation)
+
+ userFileTableConf, err := cloudimpl.ExternalStorageConfFromURI(unescapedUserfileListURI, reqUsername)
if err != nil {
return err
}
@@ -281,8 +290,9 @@ func deleteUserFile(ctx context.Context, conn *sqlConn, glob string) error {
return err
}
- userFileTableConf, err := cloudimpl.ExternalStorageConfFromURI(unescapedUserfileListURI,
- connURL.User.Username())
+ reqUsername, _ := security.MakeSQLUsernameFromUserInput(connURL.User.Username(), security.UsernameValidation)
+
+ userFileTableConf, err := cloudimpl.ExternalStorageConfFromURI(unescapedUserfileListURI, reqUsername)
if err != nil {
return err
}
@@ -326,7 +336,10 @@ func deleteUserFile(ctx context.Context, conn *sqlConn, glob string) error {
return errors.WithDetail(err, fmt.Sprintf("deletion failed at %s", file))
}
- resolvedHost := defaultQualifiedNamePrefix + connURL.User.Username()
+ composedTableName := tree.Name(cloudimpl.DefaultQualifiedNamePrefix + connURL.User.Username())
+ resolvedHost := cloudimpl.DefaultQualifiedNamespace +
+ // Escape special identifiers as needed.
+ composedTableName.String()
if userfileParsedURL.Host != "" {
resolvedHost = userfileParsedURL.Host
}
@@ -395,10 +408,20 @@ func uploadUserFile(
return err
}
+ // Validate the username for creation. We need to do this because
+ // there is no guarantee that the username in the connection string
+ // is the same one on the remote machine, and it may contain special
+ // characters.
+ // See also: https://github.com/cockroachdb/cockroach/issues/55389
+ username, err := security.MakeSQLUsernameFromUserInput(connURL.User.Username(), security.UsernameCreation)
+ if err != nil {
+ return err
+ }
+
// Construct the userfile URI as the destination for the CopyIn stmt.
// Currently we hardcode the db.schema prefix, in the future we might allow
// users to specify this.
- userfileURI := constructUserfileDestinationURI(source, destination, connURL.User.Username())
+ userfileURI := constructUserfileDestinationURI(source, destination, username)
// Accounts for filenames with arbitrary unicode characters. url.URL escapes
// these characters by default when setting the Path above.
diff --git a/pkg/cli/userfiletable_test.go b/pkg/cli/userfiletable_test.go
index 401c49db05bd..1cc04bae79ba 100644
--- a/pkg/cli/userfiletable_test.go
+++ b/pkg/cli/userfiletable_test.go
@@ -83,7 +83,8 @@ func checkUserFileContent(
ctx context.Context,
t *testing.T,
execcCfg interface{},
- user, userfileURI string,
+ user security.SQLUsername,
+ userfileURI string,
expectedContent []byte,
) {
store, err := execcCfg.(sql.ExecutorConfig).DistSQLSrv.ExternalStorageFromURI(ctx,
@@ -141,8 +142,8 @@ func TestUserFileUpload(t *testing.T) {
_, err = c.RunWithCapture(fmt.Sprintf("userfile upload %s %s", filePath, destination))
require.NoError(t, err)
- checkUserFileContent(ctx, t, c.ExecutorConfig(), security.RootUser,
- constructUserfileDestinationURI("", destination, security.RootUser),
+ checkUserFileContent(ctx, t, c.ExecutorConfig(), security.RootUserName(),
+ constructUserfileDestinationURI("", destination, security.RootUserName()),
tc.fileContent)
})
@@ -152,7 +153,7 @@ func TestUserFileUpload(t *testing.T) {
destination))
require.NoError(t, err)
- checkUserFileContent(ctx, t, c.ExecutorConfig(), security.RootUser,
+ checkUserFileContent(ctx, t, c.ExecutorConfig(), security.RootUserName(),
destination, tc.fileContent)
})
@@ -164,7 +165,7 @@ func TestUserFileUpload(t *testing.T) {
destination))
require.NoError(t, err)
- checkUserFileContent(ctx, t, c.ExecutorConfig(), security.RootUser,
+ checkUserFileContent(ctx, t, c.ExecutorConfig(), security.RootUserName(),
destination, tc.fileContent)
})
}
diff --git a/pkg/jobs/adopt.go b/pkg/jobs/adopt.go
index e1b06547b60b..d454db036b83 100644
--- a/pkg/jobs/adopt.go
+++ b/pkg/jobs/adopt.go
@@ -49,7 +49,7 @@ WHERE claim_session_id IS NULL ORDER BY created DESC LIMIT $3 RETURNING id`,
func (r *Registry) processClaimedJobs(ctx context.Context, s sqlliveness.Session) error {
rows, err := r.ex.QueryEx(
ctx, "select-running/get-claimed-jobs", nil,
- sessiondata.InternalExecutorOverride{User: security.NodeUser}, `
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, `
SELECT id FROM system.jobs
WHERE (status = $1 OR status = $2) AND (claim_session_id = $3 AND claim_instance_id = $4)`,
StatusRunning, StatusReverting, s.ID().UnsafeBytes(), r.ID(),
@@ -124,7 +124,7 @@ func (r *Registry) resumeJob(ctx context.Context, jobID int64, s sqlliveness.Ses
log.Infof(ctx, "job %d: resuming execution", jobID)
row, err := r.ex.QueryRowEx(
ctx, "get-job-row", nil,
- sessiondata.InternalExecutorOverride{User: security.NodeUser}, `
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, `
SELECT status, payload, progress, crdb_internal.sql_liveness_is_alive(claim_session_id)
FROM system.jobs WHERE id = $1 AND claim_session_id = $2`,
jobID, s.ID().UnsafeBytes(),
@@ -229,7 +229,7 @@ func (r *Registry) runJob(
if job.mu.payload.FinalResumeError != nil {
finalResumeError = errors.DecodeError(ctx, *job.mu.payload.FinalResumeError)
}
- username := job.mu.payload.Username
+ username := job.mu.payload.UsernameProto.Decode()
typ := job.mu.payload.Type()
job.mu.Unlock()
@@ -262,7 +262,7 @@ func (r *Registry) runJob(
func (r *Registry) servePauseAndCancelRequests(ctx context.Context, s sqlliveness.Session) error {
return r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
rows, err := r.ex.QueryEx(
- ctx, "cancel/pause-requested", txn, sessiondata.InternalExecutorOverride{User: security.NodeUser}, `
+ ctx, "cancel/pause-requested", txn, sessiondata.InternalExecutorOverride{User: security.NodeUserName()}, `
UPDATE system.jobs
SET status =
CASE
diff --git a/pkg/jobs/deprecated.go b/pkg/jobs/deprecated.go
index 615f861cfa0f..c1a24d2d22d4 100644
--- a/pkg/jobs/deprecated.go
+++ b/pkg/jobs/deprecated.go
@@ -352,7 +352,7 @@ func (r *Registry) deprecatedResume(
}
// Bookkeeping.
payload := job.Payload()
- phs, cleanup := r.planFn("resume-"+job.taskName(), payload.Username)
+ phs, cleanup := r.planFn("resume-"+job.taskName(), payload.UsernameProto.Decode())
defer cleanup()
spanName := fmt.Sprintf(`%s-%d`, payload.Type(), *job.ID())
var span opentracing.Span
diff --git a/pkg/jobs/executor_impl.go b/pkg/jobs/executor_impl.go
index 9f8297cc2aad..1ce535e3bef8 100644
--- a/pkg/jobs/executor_impl.go
+++ b/pkg/jobs/executor_impl.go
@@ -57,7 +57,7 @@ func (e *inlineScheduledJobExecutor) ExecuteJob(
// Also, performing this under the same transaction as the scan loop is not ideal
// since a single failure would result in rollback for all of the changes.
_, err := cfg.InternalExecutor.ExecEx(ctx, "inline-exec", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
sqlArgs.Statement,
)
diff --git a/pkg/jobs/job_scheduler.go b/pkg/jobs/job_scheduler.go
index 0053ffdeb238..098526e2835b 100644
--- a/pkg/jobs/job_scheduler.go
+++ b/pkg/jobs/job_scheduler.go
@@ -217,7 +217,7 @@ func newLoopStats(
readyToRunStmt, numRunningJobsStmt)
datums, err := ex.QueryRowEx(ctx, "scheduler-stats", txn,
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
statsStmt)
if err != nil {
return nil, err
@@ -283,7 +283,7 @@ func (s *jobScheduler) executeSchedules(
rows, cols, err := s.InternalExecutor.QueryWithCols(
ctx, "find-scheduled-jobs",
txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
findSchedulesStmt)
if err != nil {
diff --git a/pkg/jobs/job_scheduler_test.go b/pkg/jobs/job_scheduler_test.go
index 94ab5ce1ebe3..05826d78b191 100644
--- a/pkg/jobs/job_scheduler_test.go
+++ b/pkg/jobs/job_scheduler_test.go
@@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/scheduledjobs"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/testutils"
@@ -576,7 +577,7 @@ func TestJobSchedulerDaemonUsesSystemTables(t *testing.T) {
// Create a one off job which writes some values into 'foo' table.
schedule := NewScheduledJob(scheduledjobs.ProdJobSchedulerEnv)
schedule.SetScheduleLabel("test schedule")
- schedule.SetOwner("test")
+ schedule.SetOwner(security.TestUserName())
schedule.SetNextRun(timeutil.Now())
any, err := types.MarshalAny(
&jobspb.SqlStatementExecutionArg{Statement: "INSERT INTO defaultdb.foo VALUES (1), (2), (3)"})
@@ -610,7 +611,7 @@ func TestTransientTxnErrors(t *testing.T) {
for i := 0; i < 10; i++ {
schedule := NewScheduledJob(h.env)
schedule.SetScheduleLabel(fmt.Sprintf("test schedule: %d", i))
- schedule.SetOwner("test")
+ schedule.SetOwner(security.TestUserName())
require.NoError(t, schedule.SetSchedule("*/1 * * * *"))
any, err := types.MarshalAny(&jobspb.SqlStatementExecutionArg{
Statement: fmt.Sprintf("UPSERT INTO defaultdb.foo (a, b) VALUES (%d, now())", i),
diff --git a/pkg/jobs/jobs.go b/pkg/jobs/jobs.go
index bdea54aa1a56..9360e946531b 100644
--- a/pkg/jobs/jobs.go
+++ b/pkg/jobs/jobs.go
@@ -65,7 +65,7 @@ type CreatedByInfo struct {
type Record struct {
Description string
Statement string
- Username string
+ Username security.SQLUsername
DescriptorIDs descpb.IDs
Details jobspb.Details
Progress jobspb.ProgressDetails
@@ -499,7 +499,7 @@ func (j *Job) pauseRequested(ctx context.Context, fn onPauseRequestFunc) error {
return fmt.Errorf("job with status %s cannot be requested to be paused", md.Status)
}
if fn != nil {
- phs, cleanup := j.registry.planFn("pause request", j.Payload().Username)
+ phs, cleanup := j.registry.planFn("pause request", j.Payload().UsernameProto.Decode())
defer cleanup()
if err := fn(ctx, phs, txn, md.Progress); err != nil {
return err
@@ -722,7 +722,7 @@ func (j *Job) load(ctx context.Context) error {
stmt = newStmt
}
row, err := j.registry.ex.QueryRowEx(
- ctx, "load-job-query", txn, sessiondata.InternalExecutorOverride{User: security.RootUser},
+ ctx, "load-job-query", txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()},
stmt, *j.ID())
if err != nil {
return err
diff --git a/pkg/jobs/jobs_test.go b/pkg/jobs/jobs_test.go
index 40f03b12b72f..c4358ad74571 100644
--- a/pkg/jobs/jobs_test.go
+++ b/pkg/jobs/jobs_test.go
@@ -95,7 +95,7 @@ func (expected *expectation) verify(id *int64, expectedStatus jobs.Status) error
Description: payload.Description,
Details: details,
DescriptorIDs: payload.DescriptorIDs,
- Username: payload.Username,
+ Username: payload.UsernameProto.Decode(),
Progress: progressDetail,
}); !reflect.DeepEqual(e, a) {
diff := strings.Join(pretty.Diff(e, a), "\n")
@@ -885,9 +885,10 @@ func TestJobLifecycle(t *testing.T) {
t.Run("valid job lifecycles succeed", func(t *testing.T) {
// Woody is a successful job.
+ woodyPride, _ := security.MakeSQLUsernameFromUserInput("Woody Pride", security.UsernameValidation)
woodyJob, woodyExp := createJob(jobs.Record{
Description: "There's a snake in my boot!",
- Username: "Woody Pride",
+ Username: woodyPride,
DescriptorIDs: []descpb.ID{1, 2, 3},
Details: jobspb.RestoreDetails{},
Progress: jobspb.RestoreProgress{},
@@ -943,9 +944,10 @@ func TestJobLifecycle(t *testing.T) {
}
// Buzz fails after it starts running.
+ buzzL, _ := security.MakeSQLUsernameFromUserInput("Buzz Lightyear", security.UsernameValidation)
buzzRecord := jobs.Record{
Description: "To infinity and beyond!",
- Username: "Buzz Lightyear",
+ Username: buzzL,
DescriptorIDs: []descpb.ID{3, 2, 1},
Details: jobspb.BackupDetails{},
Progress: jobspb.BackupProgress{},
@@ -994,9 +996,10 @@ func TestJobLifecycle(t *testing.T) {
}
// Sid fails before it starts running.
+ sidP, _ := security.MakeSQLUsernameFromUserInput("Sid Phillips", security.UsernameValidation)
sidJob, sidExp := createJob(jobs.Record{
Description: "The toys! The toys are alive!",
- Username: "Sid Phillips",
+ Username: sidP,
DescriptorIDs: []descpb.ID{6, 6, 6},
Details: jobspb.RestoreDetails{},
Progress: jobspb.RestoreProgress{},
@@ -1422,7 +1425,7 @@ func TestShowJobs(t *testing.T) {
typ string
status string
description string
- username string
+ username security.SQLUsername
err string
created time.Time
started time.Time
@@ -1440,7 +1443,7 @@ func TestShowJobs(t *testing.T) {
typ: "SCHEMA CHANGE",
status: "superfailed",
description: "failjob",
- username: "failure",
+ username: security.MakeSQLUsernameFromPreNormalizedString("failure"),
err: "boom",
// lib/pq returns time.Time objects with goofy locations, which breaks
// reflect.DeepEqual without this time.FixedZone song and dance.
@@ -1458,7 +1461,7 @@ func TestShowJobs(t *testing.T) {
typ: "CHANGEFEED",
status: "running",
description: "persistent feed",
- username: "persistent",
+ username: security.MakeSQLUsernameFromPreNormalizedString("persistent"),
err: "",
// lib/pq returns time.Time objects with goofy locations, which breaks
// reflect.DeepEqual without this time.FixedZone song and dance.
@@ -1482,7 +1485,7 @@ func TestShowJobs(t *testing.T) {
Description: in.description,
StartedMicros: in.started.UnixNano() / time.Microsecond.Nanoseconds(),
FinishedMicros: in.finished.UnixNano() / time.Microsecond.Nanoseconds(),
- Username: in.username,
+ UsernameProto: in.username.EncodeProto(),
Lease: &jobspb.Lease{
NodeID: 7,
},
@@ -1517,14 +1520,16 @@ func TestShowJobs(t *testing.T) {
var out row
var maybeFractionCompleted *float32
var decimalHighWater *apd.Decimal
+ var resultUsername string
sqlDB.QueryRow(t, `
SELECT job_id, job_type, status, created, description, started, finished, modified,
fraction_completed, high_water_timestamp, user_name, ifnull(error, ''), coordinator_id
FROM crdb_internal.jobs WHERE job_id = $1`, in.id).Scan(
&out.id, &out.typ, &out.status, &out.created, &out.description, &out.started,
- &out.finished, &out.modified, &maybeFractionCompleted, &decimalHighWater, &out.username,
+ &out.finished, &out.modified, &maybeFractionCompleted, &decimalHighWater, &resultUsername,
&out.err, &out.coordinatorID,
)
+ out.username = security.MakeSQLUsernameFromPreNormalizedString(resultUsername)
if decimalHighWater != nil {
var err error
@@ -1588,8 +1593,8 @@ func TestShowAutomaticJobs(t *testing.T) {
// system.jobs is part proper SQL columns, part protobuf, so we can't use the
// row struct directly.
inPayload, err := protoutil.Marshal(&jobspb.Payload{
- Username: security.RootUser,
- Details: jobspb.WrapPayloadDetails(in.details),
+ UsernameProto: security.RootUserName().EncodeProto(),
+ Details: jobspb.WrapPayloadDetails(in.details),
})
if err != nil {
t.Fatal(err)
@@ -1807,7 +1812,7 @@ func TestShowJobWhenComplete(t *testing.T) {
defer s.Stopper().Stop(ctx)
registry := s.JobRegistry().(*jobs.Registry)
mockJob := jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Details: jobspb.ImportDetails{},
Progress: jobspb.ImportProgress{},
}
@@ -2151,9 +2156,10 @@ func TestStartableJob(t *testing.T) {
},
}
})
+ woodyP, _ := security.MakeSQLUsernameFromUserInput("Woody Pride", security.UsernameValidation)
rec := jobs.Record{
Description: "There's a snake in my boot!",
- Username: "Woody Pride",
+ Username: woodyP,
DescriptorIDs: []descpb.ID{1, 2, 3},
Details: jobspb.RestoreDetails{},
Progress: jobspb.RestoreProgress{},
diff --git a/pkg/jobs/jobspb/jobs.pb.go b/pkg/jobs/jobspb/jobs.pb.go
index 8e1039503f5e..36a5908cebec 100644
--- a/pkg/jobs/jobspb/jobs.pb.go
+++ b/pkg/jobs/jobspb/jobs.pb.go
@@ -15,6 +15,7 @@ import github_com_cockroachdb_cockroach_pkg_roachpb "github.com/cockroachdb/cock
import github_com_cockroachdb_cockroach_pkg_util_uuid "github.com/cockroachdb/cockroach/pkg/util/uuid"
import github_com_cockroachdb_cockroach_pkg_sql_catalog_descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
import github_com_cockroachdb_cockroach_pkg_sql_sem_tree "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+import github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security"
import bytes "bytes"
@@ -54,7 +55,7 @@ func (x EncryptionMode) String() string {
return proto.EnumName(EncryptionMode_name, int32(x))
}
func (EncryptionMode) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{0}
}
type Status int32
@@ -83,7 +84,7 @@ func (x Status) String() string {
return proto.EnumName(Status_name, int32(x))
}
func (Status) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{1}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{1}
}
type Type int32
@@ -129,7 +130,7 @@ var Type_value = map[string]int32{
}
func (Type) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{2}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{2}
}
type EncryptionInfo_Scheme int32
@@ -149,7 +150,7 @@ func (x EncryptionInfo_Scheme) String() string {
return proto.EnumName(EncryptionInfo_Scheme_name, int32(x))
}
func (EncryptionInfo_Scheme) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{2, 0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{2, 0}
}
type SchemaChangeGCProgress_Status int32
@@ -179,7 +180,7 @@ func (x SchemaChangeGCProgress_Status) String() string {
return proto.EnumName(SchemaChangeGCProgress_Status_name, int32(x))
}
func (SchemaChangeGCProgress_Status) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{16, 0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{16, 0}
}
type Lease struct {
@@ -193,7 +194,7 @@ func (m *Lease) Reset() { *m = Lease{} }
func (m *Lease) String() string { return proto.CompactTextString(m) }
func (*Lease) ProtoMessage() {}
func (*Lease) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{0}
}
func (m *Lease) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -234,7 +235,7 @@ func (m *BackupEncryptionOptions) Reset() { *m = BackupEncryptionOptions
func (m *BackupEncryptionOptions) String() string { return proto.CompactTextString(m) }
func (*BackupEncryptionOptions) ProtoMessage() {}
func (*BackupEncryptionOptions) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{1}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{1}
}
func (m *BackupEncryptionOptions) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -268,7 +269,7 @@ func (m *BackupEncryptionOptions_KMSInfo) Reset() { *m = BackupEncryptio
func (m *BackupEncryptionOptions_KMSInfo) String() string { return proto.CompactTextString(m) }
func (*BackupEncryptionOptions_KMSInfo) ProtoMessage() {}
func (*BackupEncryptionOptions_KMSInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{1, 0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{1, 0}
}
func (m *BackupEncryptionOptions_KMSInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -308,7 +309,7 @@ func (m *EncryptionInfo) Reset() { *m = EncryptionInfo{} }
func (m *EncryptionInfo) String() string { return proto.CompactTextString(m) }
func (*EncryptionInfo) ProtoMessage() {}
func (*EncryptionInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{2}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{2}
}
func (m *EncryptionInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -363,7 +364,7 @@ func (m *BackupDetails) Reset() { *m = BackupDetails{} }
func (m *BackupDetails) String() string { return proto.CompactTextString(m) }
func (*BackupDetails) ProtoMessage() {}
func (*BackupDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{3}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{3}
}
func (m *BackupDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -395,7 +396,7 @@ func (m *BackupProgress) Reset() { *m = BackupProgress{} }
func (m *BackupProgress) String() string { return proto.CompactTextString(m) }
func (*BackupProgress) ProtoMessage() {}
func (*BackupProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{4}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{4}
}
func (m *BackupProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -457,7 +458,7 @@ func (m *RestoreDetails) Reset() { *m = RestoreDetails{} }
func (m *RestoreDetails) String() string { return proto.CompactTextString(m) }
func (*RestoreDetails) ProtoMessage() {}
func (*RestoreDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{5}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{5}
}
func (m *RestoreDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -494,7 +495,7 @@ func (m *RestoreDetails_DescriptorRewrite) Reset() { *m = RestoreDetails
func (m *RestoreDetails_DescriptorRewrite) String() string { return proto.CompactTextString(m) }
func (*RestoreDetails_DescriptorRewrite) ProtoMessage() {}
func (*RestoreDetails_DescriptorRewrite) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{5, 0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{5, 0}
}
func (m *RestoreDetails_DescriptorRewrite) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -527,7 +528,7 @@ func (m *RestoreDetails_BackupLocalityInfo) Reset() { *m = RestoreDetail
func (m *RestoreDetails_BackupLocalityInfo) String() string { return proto.CompactTextString(m) }
func (*RestoreDetails_BackupLocalityInfo) ProtoMessage() {}
func (*RestoreDetails_BackupLocalityInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{5, 1}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{5, 1}
}
func (m *RestoreDetails_BackupLocalityInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -560,7 +561,7 @@ func (m *RestoreProgress) Reset() { *m = RestoreProgress{} }
func (m *RestoreProgress) String() string { return proto.CompactTextString(m) }
func (*RestoreProgress) ProtoMessage() {}
func (*RestoreProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{6}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{6}
}
func (m *RestoreProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -620,7 +621,7 @@ func (m *ImportDetails) Reset() { *m = ImportDetails{} }
func (m *ImportDetails) String() string { return proto.CompactTextString(m) }
func (*ImportDetails) ProtoMessage() {}
func (*ImportDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{7}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{7}
}
func (m *ImportDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -658,7 +659,7 @@ func (m *ImportDetails_Table) Reset() { *m = ImportDetails_Table{} }
func (m *ImportDetails_Table) String() string { return proto.CompactTextString(m) }
func (*ImportDetails_Table) ProtoMessage() {}
func (*ImportDetails_Table) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{7, 0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{7, 0}
}
func (m *ImportDetails_Table) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -701,7 +702,7 @@ func (m *ImportProgress) Reset() { *m = ImportProgress{} }
func (m *ImportProgress) String() string { return proto.CompactTextString(m) }
func (*ImportProgress) ProtoMessage() {}
func (*ImportProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{8}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{8}
}
func (m *ImportProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -735,7 +736,7 @@ func (m *TypeSchemaChangeDetails) Reset() { *m = TypeSchemaChangeDetails
func (m *TypeSchemaChangeDetails) String() string { return proto.CompactTextString(m) }
func (*TypeSchemaChangeDetails) ProtoMessage() {}
func (*TypeSchemaChangeDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{9}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{9}
}
func (m *TypeSchemaChangeDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -768,7 +769,7 @@ func (m *TypeSchemaChangeProgress) Reset() { *m = TypeSchemaChangeProgre
func (m *TypeSchemaChangeProgress) String() string { return proto.CompactTextString(m) }
func (*TypeSchemaChangeProgress) ProtoMessage() {}
func (*TypeSchemaChangeProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{10}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{10}
}
func (m *TypeSchemaChangeProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -801,7 +802,7 @@ func (m *ResumeSpanList) Reset() { *m = ResumeSpanList{} }
func (m *ResumeSpanList) String() string { return proto.CompactTextString(m) }
func (*ResumeSpanList) ProtoMessage() {}
func (*ResumeSpanList) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{11}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{11}
}
func (m *ResumeSpanList) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -836,7 +837,7 @@ func (m *DroppedTableDetails) Reset() { *m = DroppedTableDetails{} }
func (m *DroppedTableDetails) String() string { return proto.CompactTextString(m) }
func (*DroppedTableDetails) ProtoMessage() {}
func (*DroppedTableDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{12}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{12}
}
func (m *DroppedTableDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -894,7 +895,7 @@ func (m *SchemaChangeGCDetails) Reset() { *m = SchemaChangeGCDetails{} }
func (m *SchemaChangeGCDetails) String() string { return proto.CompactTextString(m) }
func (*SchemaChangeGCDetails) ProtoMessage() {}
func (*SchemaChangeGCDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{13}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{13}
}
func (m *SchemaChangeGCDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -928,7 +929,7 @@ func (m *SchemaChangeGCDetails_DroppedIndex) Reset() { *m = SchemaChange
func (m *SchemaChangeGCDetails_DroppedIndex) String() string { return proto.CompactTextString(m) }
func (*SchemaChangeGCDetails_DroppedIndex) ProtoMessage() {}
func (*SchemaChangeGCDetails_DroppedIndex) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{13, 0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{13, 0}
}
func (m *SchemaChangeGCDetails_DroppedIndex) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -962,7 +963,7 @@ func (m *SchemaChangeGCDetails_DroppedID) Reset() { *m = SchemaChangeGCD
func (m *SchemaChangeGCDetails_DroppedID) String() string { return proto.CompactTextString(m) }
func (*SchemaChangeGCDetails_DroppedID) ProtoMessage() {}
func (*SchemaChangeGCDetails_DroppedID) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{13, 1}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{13, 1}
}
func (m *SchemaChangeGCDetails_DroppedID) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1026,7 +1027,7 @@ func (m *SchemaChangeDetails) Reset() { *m = SchemaChangeDetails{} }
func (m *SchemaChangeDetails) String() string { return proto.CompactTextString(m) }
func (*SchemaChangeDetails) ProtoMessage() {}
func (*SchemaChangeDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{14}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{14}
}
func (m *SchemaChangeDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1058,7 +1059,7 @@ func (m *SchemaChangeProgress) Reset() { *m = SchemaChangeProgress{} }
func (m *SchemaChangeProgress) String() string { return proto.CompactTextString(m) }
func (*SchemaChangeProgress) ProtoMessage() {}
func (*SchemaChangeProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{15}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{15}
}
func (m *SchemaChangeProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1094,7 +1095,7 @@ func (m *SchemaChangeGCProgress) Reset() { *m = SchemaChangeGCProgress{}
func (m *SchemaChangeGCProgress) String() string { return proto.CompactTextString(m) }
func (*SchemaChangeGCProgress) ProtoMessage() {}
func (*SchemaChangeGCProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{16}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{16}
}
func (m *SchemaChangeGCProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1128,7 +1129,7 @@ func (m *SchemaChangeGCProgress_IndexProgress) Reset() { *m = SchemaChan
func (m *SchemaChangeGCProgress_IndexProgress) String() string { return proto.CompactTextString(m) }
func (*SchemaChangeGCProgress_IndexProgress) ProtoMessage() {}
func (*SchemaChangeGCProgress_IndexProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{16, 0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{16, 0}
}
func (m *SchemaChangeGCProgress_IndexProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1162,7 +1163,7 @@ func (m *SchemaChangeGCProgress_TableProgress) Reset() { *m = SchemaChan
func (m *SchemaChangeGCProgress_TableProgress) String() string { return proto.CompactTextString(m) }
func (*SchemaChangeGCProgress_TableProgress) ProtoMessage() {}
func (*SchemaChangeGCProgress_TableProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{16, 1}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{16, 1}
}
func (m *SchemaChangeGCProgress_TableProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1195,7 +1196,7 @@ func (m *ChangefeedTarget) Reset() { *m = ChangefeedTarget{} }
func (m *ChangefeedTarget) String() string { return proto.CompactTextString(m) }
func (*ChangefeedTarget) ProtoMessage() {}
func (*ChangefeedTarget) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{17}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{17}
}
func (m *ChangefeedTarget) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1248,7 +1249,7 @@ func (m *ChangefeedDetails) Reset() { *m = ChangefeedDetails{} }
func (m *ChangefeedDetails) String() string { return proto.CompactTextString(m) }
func (*ChangefeedDetails) ProtoMessage() {}
func (*ChangefeedDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{18}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{18}
}
func (m *ChangefeedDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1283,7 +1284,7 @@ func (m *ResolvedSpan) Reset() { *m = ResolvedSpan{} }
func (m *ResolvedSpan) String() string { return proto.CompactTextString(m) }
func (*ResolvedSpan) ProtoMessage() {}
func (*ResolvedSpan) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{19}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{19}
}
func (m *ResolvedSpan) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1326,7 +1327,7 @@ func (m *ChangefeedProgress) Reset() { *m = ChangefeedProgress{} }
func (m *ChangefeedProgress) String() string { return proto.CompactTextString(m) }
func (*ChangefeedProgress) ProtoMessage() {}
func (*ChangefeedProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{20}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{20}
}
func (m *ChangefeedProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1370,7 +1371,7 @@ func (m *CreateStatsDetails) Reset() { *m = CreateStatsDetails{} }
func (m *CreateStatsDetails) String() string { return proto.CompactTextString(m) }
func (*CreateStatsDetails) ProtoMessage() {}
func (*CreateStatsDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{21}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{21}
}
func (m *CreateStatsDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1411,7 +1412,7 @@ func (m *CreateStatsDetails_ColStat) Reset() { *m = CreateStatsDetails_C
func (m *CreateStatsDetails_ColStat) String() string { return proto.CompactTextString(m) }
func (*CreateStatsDetails_ColStat) ProtoMessage() {}
func (*CreateStatsDetails_ColStat) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{21, 0}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{21, 0}
}
func (m *CreateStatsDetails_ColStat) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1443,7 +1444,7 @@ func (m *CreateStatsProgress) Reset() { *m = CreateStatsProgress{} }
func (m *CreateStatsProgress) String() string { return proto.CompactTextString(m) }
func (*CreateStatsProgress) ProtoMessage() {}
func (*CreateStatsProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{22}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{22}
}
func (m *CreateStatsProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1471,8 +1472,8 @@ var xxx_messageInfo_CreateStatsProgress proto.InternalMessageInfo
type Payload struct {
Description string `protobuf:"bytes,1,opt,name=description,proto3" json:"description,omitempty"`
// If empty, the description is assumed to be the statement.
- Statement string `protobuf:"bytes,16,opt,name=statement,proto3" json:"statement,omitempty"`
- Username string `protobuf:"bytes,2,opt,name=username,proto3" json:"username,omitempty"`
+ Statement string `protobuf:"bytes,16,opt,name=statement,proto3" json:"statement,omitempty"`
+ UsernameProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,2,opt,name=username_proto,json=usernameProto,proto3,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"username_proto,omitempty"`
// For consistency with the SQL timestamp type, which has microsecond
// precision, we avoid the timestamp.Timestamp WKT, which has nanosecond
// precision, and use microsecond integers directly.
@@ -1510,7 +1511,7 @@ func (m *Payload) Reset() { *m = Payload{} }
func (m *Payload) String() string { return proto.CompactTextString(m) }
func (*Payload) ProtoMessage() {}
func (*Payload) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{23}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{23}
}
func (m *Payload) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1849,7 +1850,7 @@ func (m *Progress) Reset() { *m = Progress{} }
func (m *Progress) String() string { return proto.CompactTextString(m) }
func (*Progress) ProtoMessage() {}
func (*Progress) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{24}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{24}
}
func (m *Progress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2255,7 +2256,7 @@ func (m *Job) Reset() { *m = Job{} }
func (m *Job) String() string { return proto.CompactTextString(m) }
func (*Job) ProtoMessage() {}
func (*Job) Descriptor() ([]byte, []int) {
- return fileDescriptor_jobs_dfd8362a26101d63, []int{25}
+ return fileDescriptor_jobs_d6cc45c33927cba9, []int{25}
}
func (m *Job) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4137,11 +4138,11 @@ func (m *Payload) MarshalTo(dAtA []byte) (int, error) {
i = encodeVarintJobs(dAtA, i, uint64(len(m.Description)))
i += copy(dAtA[i:], m.Description)
}
- if len(m.Username) > 0 {
+ if len(m.UsernameProto) > 0 {
dAtA[i] = 0x12
i++
- i = encodeVarintJobs(dAtA, i, uint64(len(m.Username)))
- i += copy(dAtA[i:], m.Username)
+ i = encodeVarintJobs(dAtA, i, uint64(len(m.UsernameProto)))
+ i += copy(dAtA[i:], m.UsernameProto)
}
if m.StartedMicros != 0 {
dAtA[i] = 0x18
@@ -5384,7 +5385,7 @@ func (m *Payload) Size() (n int) {
if l > 0 {
n += 1 + l + sovJobs(uint64(l))
}
- l = len(m.Username)
+ l = len(m.UsernameProto)
if l > 0 {
n += 1 + l + sovJobs(uint64(l))
}
@@ -11092,7 +11093,7 @@ func (m *Payload) Unmarshal(dAtA []byte) error {
iNdEx = postIndex
case 2:
if wireType != 2 {
- return fmt.Errorf("proto: wrong wireType = %d for field Username", wireType)
+ return fmt.Errorf("proto: wrong wireType = %d for field UsernameProto", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
@@ -11117,7 +11118,7 @@ func (m *Payload) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
- m.Username = string(dAtA[iNdEx:postIndex])
+ m.UsernameProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 3:
if wireType != 0 {
@@ -12350,268 +12351,270 @@ var (
ErrIntOverflowJobs = fmt.Errorf("proto: integer overflow")
)
-func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_dfd8362a26101d63) }
+func init() { proto.RegisterFile("jobs/jobspb/jobs.proto", fileDescriptor_jobs_d6cc45c33927cba9) }
-var fileDescriptor_jobs_dfd8362a26101d63 = []byte{
- // 4149 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5b, 0xcb, 0x6f, 0x23, 0xc9,
+var fileDescriptor_jobs_d6cc45c33927cba9 = []byte{
+ // 4178 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xc4, 0x5a, 0xcb, 0x6f, 0x23, 0xc9,
0x79, 0x17, 0x1f, 0x22, 0x9b, 0x1f, 0x45, 0xaa, 0x59, 0xd2, 0xcc, 0xd0, 0xcc, 0x5a, 0x54, 0xb8,
- 0x3b, 0x9e, 0xc7, 0xee, 0x52, 0x6b, 0x4d, 0xbc, 0x5e, 0x4f, 0xbc, 0xb3, 0x16, 0x1f, 0x92, 0x48,
- 0x8d, 0x1e, 0xdb, 0x94, 0x66, 0xbd, 0x6b, 0x6c, 0x3a, 0x4d, 0x76, 0x49, 0xea, 0xa8, 0xd9, 0xdd,
+ 0xbb, 0x9e, 0xc7, 0xee, 0x52, 0x6b, 0x4d, 0xbc, 0x5e, 0x4f, 0xbc, 0xb3, 0x16, 0x1f, 0x92, 0x48,
+ 0x8d, 0x1e, 0xd3, 0x94, 0x66, 0xbd, 0x6b, 0x6c, 0x3a, 0x4d, 0x76, 0x49, 0xea, 0xa8, 0xd9, 0xdd,
0xd3, 0xd5, 0x9c, 0x19, 0x19, 0x41, 0x12, 0x38, 0x48, 0x60, 0xcc, 0x29, 0x01, 0x9c, 0x1c, 0x92,
- 0x4c, 0x10, 0x20, 0x31, 0x92, 0x43, 0x80, 0x00, 0x46, 0x90, 0xe4, 0x4f, 0xd8, 0x43, 0x0e, 0xbe,
- 0x04, 0x30, 0x72, 0xa0, 0x13, 0xee, 0x25, 0x87, 0x1c, 0x8c, 0xe4, 0x36, 0xa7, 0xa0, 0x1e, 0xdd,
- 0x6c, 0x52, 0x2f, 0x6a, 0xb4, 0xeb, 0x5c, 0x34, 0x5d, 0x5f, 0x7d, 0xf5, 0xab, 0xd7, 0x57, 0xbf,
- 0xef, 0xab, 0xaf, 0x38, 0x70, 0xfd, 0xb7, 0xec, 0x36, 0x59, 0xa2, 0x7f, 0x9c, 0x36, 0xfb, 0xa7,
- 0xec, 0xb8, 0xb6, 0x67, 0xa3, 0xaf, 0x74, 0xec, 0xce, 0x91, 0x6b, 0x6b, 0x9d, 0xc3, 0x32, 0x79,
- 0x6c, 0x96, 0x59, 0x0d, 0xd7, 0x2a, 0x5c, 0xc3, 0xae, 0x6b, 0xbb, 0x54, 0x9f, 0x7f, 0xf0, 0x16,
- 0x85, 0xf9, 0x03, 0xfb, 0xc0, 0x66, 0x9f, 0x4b, 0xf4, 0x4b, 0x48, 0x11, 0xc3, 0x70, 0xda, 0x4b,
- 0xba, 0xe6, 0x69, 0x42, 0x96, 0xf7, 0x65, 0x86, 0xfd, 0xf6, 0xbe, 0xed, 0x76, 0x35, 0xcf, 0xc7,
- 0x78, 0x9d, 0x3c, 0x36, 0x97, 0x3a, 0x9a, 0xa7, 0x99, 0xf6, 0xc1, 0x92, 0x8e, 0x49, 0xc7, 0x69,
- 0x2f, 0x11, 0xcf, 0xed, 0x75, 0xbc, 0x9e, 0x8b, 0x75, 0xa1, 0x54, 0x3c, 0x45, 0xc9, 0xc3, 0x96,
- 0x66, 0x79, 0x3e, 0x7e, 0xcf, 0x33, 0xcc, 0xa5, 0x43, 0xb3, 0xb3, 0xe4, 0x19, 0x5d, 0x4c, 0x3c,
- 0xad, 0xeb, 0xf0, 0x9a, 0xd2, 0xef, 0xc2, 0xf4, 0x43, 0xac, 0x11, 0x8c, 0x3e, 0x81, 0xa4, 0x65,
- 0xeb, 0x58, 0x35, 0xf4, 0x7c, 0x64, 0x31, 0x72, 0x3b, 0x53, 0x59, 0x19, 0xf4, 0x8b, 0x89, 0x2d,
- 0x5b, 0xc7, 0x8d, 0xda, 0xcb, 0x7e, 0xf1, 0xde, 0x81, 0xe1, 0x1d, 0xf6, 0xda, 0xe5, 0x8e, 0xdd,
- 0x5d, 0x0a, 0x16, 0x42, 0x6f, 0x0f, 0xbf, 0x97, 0x9c, 0xa3, 0x83, 0x25, 0x31, 0x8d, 0x32, 0x6f,
+ 0x0c, 0x10, 0x20, 0x31, 0x92, 0x43, 0x80, 0x00, 0x46, 0x90, 0xe4, 0x4f, 0xd8, 0x43, 0x0e, 0xbe,
+ 0x04, 0x30, 0x72, 0xa0, 0x13, 0xee, 0x25, 0x87, 0x04, 0x30, 0x92, 0xdb, 0x9c, 0x82, 0x7a, 0x74,
+ 0xb3, 0x49, 0xbd, 0xa8, 0xd1, 0xae, 0x73, 0x91, 0x58, 0x5f, 0x7d, 0xf5, 0xab, 0xd7, 0xf7, 0xfd,
+ 0xbe, 0xaf, 0xaa, 0x1a, 0xae, 0xff, 0x96, 0xdd, 0x26, 0x4b, 0xf4, 0x8f, 0xd3, 0x66, 0xff, 0xca,
+ 0x8e, 0x6b, 0x7b, 0x36, 0xfa, 0x4a, 0xc7, 0xee, 0x1c, 0xb9, 0xb6, 0xd6, 0x39, 0x2c, 0x93, 0xc7,
+ 0x66, 0x99, 0xd5, 0x70, 0xad, 0xc2, 0x35, 0xec, 0xba, 0xb6, 0x4b, 0xf5, 0xf9, 0x0f, 0xde, 0xa2,
+ 0x30, 0x7f, 0x60, 0x1f, 0xd8, 0xec, 0xe7, 0x12, 0xfd, 0x25, 0xa4, 0x88, 0x61, 0x38, 0xed, 0x25,
+ 0x5d, 0xf3, 0x34, 0x21, 0xcb, 0xfb, 0x32, 0xc3, 0x7e, 0x67, 0xdf, 0x76, 0xbb, 0x9a, 0xe7, 0x63,
+ 0xbc, 0x4e, 0x1e, 0x9b, 0x4b, 0x1d, 0xcd, 0xd3, 0x4c, 0xfb, 0x60, 0x49, 0xc7, 0xa4, 0xe3, 0xb4,
+ 0x97, 0x88, 0xe7, 0xf6, 0x3a, 0x5e, 0xcf, 0xc5, 0xba, 0x50, 0x2a, 0x9e, 0xa2, 0xe4, 0x61, 0x4b,
+ 0xb3, 0x3c, 0x1f, 0xbf, 0xe7, 0x19, 0xe6, 0xd2, 0xa1, 0xd9, 0x59, 0xf2, 0x8c, 0x2e, 0x26, 0x9e,
+ 0xd6, 0x75, 0x78, 0x4d, 0xe9, 0x77, 0x61, 0xfa, 0x01, 0xd6, 0x08, 0x46, 0x9f, 0x40, 0xd2, 0xb2,
+ 0x75, 0xac, 0x1a, 0x7a, 0x3e, 0xb2, 0x18, 0xb9, 0x95, 0xa9, 0xac, 0x0c, 0xfa, 0xc5, 0xc4, 0x96,
+ 0xad, 0xe3, 0x46, 0xed, 0x65, 0xbf, 0x78, 0xf7, 0xc0, 0xf0, 0x0e, 0x7b, 0xed, 0x72, 0xc7, 0xee,
+ 0x2e, 0x05, 0x0b, 0xa1, 0xb7, 0x87, 0xbf, 0x97, 0x9c, 0xa3, 0x83, 0x25, 0x31, 0x8d, 0x32, 0x6f,
0xa6, 0x24, 0x28, 0x62, 0x43, 0x47, 0xf3, 0x30, 0x8d, 0x1d, 0xbb, 0x73, 0x98, 0x8f, 0x2e, 0x46,
- 0x6e, 0xc7, 0x14, 0x5e, 0xb8, 0x1f, 0xff, 0xaf, 0xbf, 0x2a, 0x46, 0x4a, 0x3f, 0x8e, 0xc2, 0x8d,
+ 0x6e, 0xc5, 0x14, 0x5e, 0xb8, 0x17, 0xff, 0xcf, 0xbf, 0x2c, 0x46, 0x4a, 0x3f, 0x8e, 0xc2, 0x8d,
0x8a, 0xd6, 0x39, 0xea, 0x39, 0x75, 0xab, 0xe3, 0x1e, 0x3b, 0x9e, 0x61, 0x5b, 0xdb, 0xec, 0x2f,
- 0x41, 0x32, 0xc4, 0x8e, 0xf0, 0x31, 0x1b, 0xcf, 0x8c, 0x42, 0x3f, 0xd1, 0xfb, 0x10, 0xef, 0xda,
- 0x3a, 0x66, 0x40, 0xd9, 0xe5, 0x3b, 0xe5, 0x33, 0xf7, 0xa4, 0x3c, 0x44, 0xdb, 0xb4, 0x75, 0xac,
- 0xb0, 0x66, 0xa8, 0x0d, 0xd2, 0x51, 0x97, 0xa8, 0x86, 0xb5, 0x6f, 0xe7, 0x63, 0x8b, 0x91, 0xdb,
- 0xe9, 0xe5, 0xfb, 0xe7, 0x40, 0x9c, 0x31, 0xac, 0xf2, 0xc6, 0x66, 0xab, 0x61, 0xed, 0xdb, 0x95,
- 0xf4, 0xa0, 0x5f, 0x4c, 0x8a, 0x82, 0x92, 0x3c, 0xea, 0x12, 0xfa, 0x51, 0xd8, 0x06, 0x5f, 0x46,
- 0xc7, 0xdf, 0x73, 0x0d, 0x36, 0xfe, 0x94, 0x42, 0x3f, 0xd1, 0x5b, 0x80, 0x30, 0xc7, 0xc3, 0xba,
- 0x4a, 0x0d, 0x40, 0xa5, 0x13, 0x8c, 0xb2, 0x09, 0xca, 0x41, 0x4d, 0x4d, 0xf3, 0xb4, 0x0d, 0x7c,
- 0xcc, 0x57, 0x48, 0xac, 0xd3, 0xef, 0xc5, 0x20, 0x3b, 0x1c, 0x0a, 0x83, 0x5f, 0x87, 0x04, 0xe9,
- 0x1c, 0xe2, 0x2e, 0x66, 0x3d, 0x64, 0x97, 0xdf, 0x99, 0x68, 0x39, 0x68, 0xd3, 0x72, 0x8b, 0xb5,
- 0x53, 0x44, 0x7b, 0x84, 0x20, 0x4e, 0x34, 0xd3, 0x13, 0x03, 0x61, 0xdf, 0xe8, 0xcf, 0x23, 0xb0,
- 0x38, 0x3e, 0xa2, 0xca, 0xf1, 0xc6, 0x66, 0x6b, 0x53, 0x23, 0x1e, 0x76, 0x37, 0xf0, 0x71, 0xa3,
- 0x96, 0x8f, 0x2d, 0xc6, 0x6e, 0xa7, 0x97, 0xb7, 0x27, 0xef, 0xb8, 0x7e, 0x01, 0x62, 0xdd, 0xf2,
- 0xdc, 0x63, 0xe5, 0xc2, 0x8e, 0x0b, 0x2d, 0xb8, 0x39, 0x11, 0x54, 0xd8, 0x86, 0x52, 0xdc, 0x86,
- 0xe6, 0x61, 0xfa, 0x89, 0x66, 0xf6, 0xb0, 0x98, 0x2d, 0x2f, 0xdc, 0x8f, 0xbe, 0x17, 0x29, 0xdd,
- 0x80, 0x04, 0x5f, 0x18, 0x94, 0x81, 0xd4, 0x4a, 0xbd, 0xb5, 0xfc, 0x8d, 0x77, 0xd7, 0xaa, 0x9b,
- 0xf2, 0x94, 0xd8, 0x82, 0xbf, 0x4d, 0x40, 0x86, 0xdb, 0x44, 0x0d, 0x7b, 0x9a, 0x61, 0x12, 0x54,
- 0x01, 0x20, 0x9e, 0xe6, 0x7a, 0x2a, 0x3d, 0x56, 0xac, 0x8f, 0xf4, 0xf2, 0x57, 0x43, 0x8b, 0x41,
- 0x8f, 0x5d, 0xf9, 0xd0, 0xec, 0x94, 0x77, 0xfd, 0x63, 0x57, 0x89, 0x7f, 0xd6, 0x2f, 0x4e, 0x29,
- 0x29, 0xd6, 0x8c, 0x4a, 0xd1, 0x03, 0x90, 0xb0, 0xa5, 0x73, 0x84, 0xe8, 0xe4, 0x08, 0x49, 0x6c,
- 0xe9, 0xac, 0xfd, 0x57, 0xb8, 0x91, 0x51, 0x73, 0x4e, 0x55, 0x92, 0x83, 0x7e, 0x31, 0xb6, 0xa7,
- 0x34, 0xb8, 0xb5, 0xdd, 0x82, 0xd9, 0x36, 0x1b, 0xaf, 0xda, 0xd5, 0x2c, 0x63, 0x1f, 0x13, 0x2f,
- 0x1f, 0x67, 0x73, 0xce, 0x72, 0xf1, 0xa6, 0x90, 0xa2, 0xdf, 0x8f, 0xc0, 0x5c, 0xcf, 0x35, 0x88,
- 0xda, 0x3e, 0x56, 0x4d, 0xbb, 0xa3, 0x99, 0x86, 0x77, 0xac, 0x1e, 0x3d, 0xc9, 0x4f, 0xb3, 0xed,
- 0x7d, 0x70, 0xe1, 0x19, 0x11, 0xeb, 0x51, 0xde, 0x73, 0x0d, 0x52, 0x39, 0x7e, 0x28, 0x10, 0x36,
- 0x9e, 0xb0, 0x2d, 0xa8, 0xcc, 0x0f, 0xfa, 0x45, 0x79, 0x4f, 0x69, 0x84, 0xab, 0x1e, 0x29, 0x72,
- 0x6f, 0x4c, 0x19, 0x69, 0xc1, 0xe1, 0x30, 0x6c, 0x4b, 0xb5, 0xf9, 0x69, 0xcb, 0x27, 0xd8, 0x9a,
- 0x2c, 0x5f, 0xfe, 0x9c, 0x2a, 0x39, 0x7c, 0x82, 0x51, 0xfe, 0x38, 0x02, 0x05, 0x4a, 0x7c, 0xb8,
- 0x43, 0x0f, 0x60, 0x40, 0x86, 0xaa, 0x8b, 0x3b, 0xb6, 0xab, 0xe7, 0x93, 0x74, 0x75, 0x2a, 0xad,
- 0x7f, 0x9f, 0x94, 0xef, 0x18, 0xad, 0xf6, 0x7a, 0x86, 0x5e, 0xde, 0xdb, 0x6b, 0xd4, 0x06, 0xfd,
- 0x62, 0x7e, 0xc7, 0x07, 0x0f, 0xf6, 0x4b, 0x61, 0xd0, 0x4a, 0xde, 0x39, 0xa3, 0x06, 0xbd, 0x07,
- 0xd9, 0x8e, 0x6d, 0x9a, 0xb8, 0xc3, 0xa6, 0xbd, 0xa7, 0x34, 0xf2, 0x12, 0xdb, 0xcb, 0xdc, 0xa0,
- 0x5f, 0xcc, 0x54, 0x83, 0x1a, 0xba, 0xab, 0x99, 0x4e, 0xb8, 0x88, 0x14, 0x98, 0x0d, 0x2d, 0x18,
- 0x63, 0xb5, 0x14, 0x5b, 0xad, 0x3b, 0x13, 0x1f, 0x48, 0x25, 0x8b, 0x47, 0xca, 0x85, 0x2a, 0x5c,
- 0x3b, 0x75, 0x17, 0x2f, 0x3a, 0x48, 0xa9, 0xf0, 0x41, 0x92, 0x21, 0xcb, 0x37, 0x65, 0xc7, 0xb5,
- 0x0f, 0x5c, 0x4c, 0x48, 0xe9, 0x27, 0x59, 0xc8, 0x2a, 0x98, 0x78, 0xb6, 0x8b, 0xfd, 0xc3, 0xf3,
- 0x93, 0x08, 0xcc, 0x51, 0x67, 0xe5, 0x1a, 0x8e, 0x67, 0xbb, 0xaa, 0x8b, 0x9f, 0xba, 0x86, 0x87,
- 0x49, 0x3e, 0xca, 0x8c, 0x6e, 0xe5, 0x9c, 0x29, 0x8c, 0x02, 0x95, 0x6b, 0x01, 0x88, 0x22, 0x30,
- 0xb8, 0xdd, 0x3d, 0xf8, 0xc1, 0xcf, 0x8b, 0xf7, 0x27, 0xda, 0xc7, 0x93, 0xfe, 0xb3, 0xdc, 0xa8,
- 0x29, 0x48, 0x3f, 0x01, 0x8c, 0x5e, 0x83, 0x38, 0xb5, 0x5b, 0x46, 0x7c, 0xa9, 0x8a, 0x34, 0xe8,
- 0x17, 0xe3, 0xd4, 0xb2, 0x15, 0x26, 0x1d, 0x39, 0xcb, 0xf1, 0x57, 0x38, 0xcb, 0x6b, 0x90, 0xf6,
- 0xb4, 0xb6, 0x89, 0x55, 0xda, 0x33, 0x11, 0xc7, 0xef, 0x6b, 0x63, 0x2b, 0x41, 0x1e, 0x9b, 0x6d,
- 0x8d, 0xe0, 0xf2, 0x2e, 0xd5, 0x0c, 0xcd, 0x1d, 0x3c, 0x5f, 0x40, 0xd0, 0x12, 0xa4, 0xed, 0x27,
- 0xd8, 0x75, 0x0d, 0x1d, 0xab, 0x7a, 0x9b, 0x9d, 0xa1, 0x54, 0x25, 0x3b, 0xe8, 0x17, 0x61, 0x5b,
- 0x88, 0x6b, 0x15, 0x05, 0x7c, 0x95, 0x5a, 0x1b, 0x79, 0x30, 0x2f, 0xa8, 0x22, 0x38, 0xff, 0xcc,
- 0x9e, 0x92, 0x6c, 0x08, 0xdf, 0x9e, 0x7c, 0x33, 0xf8, 0xbe, 0xfb, 0xc6, 0xc3, 0xfc, 0x24, 0x9f,
- 0x24, 0x6a, 0x9f, 0xa8, 0x41, 0x6f, 0x42, 0xce, 0x71, 0xb1, 0xa3, 0xb9, 0x58, 0xed, 0xd8, 0x5d,
- 0xc7, 0xc4, 0x1e, 0xd6, 0x99, 0xf5, 0x4b, 0x8a, 0x2c, 0x2a, 0xaa, 0xbe, 0x1c, 0xdd, 0x84, 0x2c,
- 0xf1, 0x34, 0x8f, 0xba, 0x6f, 0x82, 0x5d, 0xaa, 0x99, 0x62, 0x9a, 0x19, 0x26, 0x6d, 0x08, 0x21,
- 0xba, 0x07, 0xd7, 0x86, 0xfb, 0x46, 0x54, 0xa7, 0xd7, 0x36, 0x0d, 0x72, 0x88, 0xf5, 0x3c, 0x30,
- 0xed, 0xf9, 0x50, 0xe5, 0x8e, 0x5f, 0x87, 0x8e, 0x47, 0x4c, 0xb1, 0x43, 0x17, 0x46, 0x3b, 0xc0,
- 0xf9, 0xf4, 0x62, 0xe4, 0xf6, 0x74, 0x65, 0xfd, 0x65, 0xbf, 0x58, 0x9b, 0xd8, 0x8e, 0x08, 0xee,
- 0x2e, 0x79, 0x2e, 0xc6, 0x21, 0xb3, 0xac, 0x0a, 0xbc, 0xb0, 0x45, 0xf9, 0x32, 0xa4, 0x00, 0x0c,
- 0x8f, 0x60, 0x7e, 0xe6, 0x95, 0xd9, 0x2e, 0x84, 0x82, 0x56, 0x20, 0xc9, 0xe3, 0x3f, 0x92, 0xcf,
- 0xb0, 0x0d, 0xfc, 0xd5, 0xb3, 0x6c, 0x88, 0x69, 0x85, 0x76, 0xc9, 0x6f, 0x87, 0x6a, 0x00, 0xde,
- 0xb1, 0xe3, 0x5b, 0x62, 0x96, 0xa1, 0xdc, 0x3c, 0x0b, 0xe5, 0xd8, 0x09, 0x1b, 0x62, 0xca, 0x13,
- 0x65, 0x82, 0x9a, 0x30, 0xc3, 0x42, 0x0c, 0x4d, 0xe0, 0xcc, 0x32, 0x9c, 0x5b, 0x67, 0xe0, 0x30,
- 0xe7, 0xab, 0x85, 0x90, 0xd2, 0x24, 0x90, 0x10, 0xb4, 0x03, 0x59, 0x1a, 0x31, 0x51, 0x4d, 0x81,
- 0x26, 0x33, 0xb4, 0x3b, 0x67, 0xa0, 0xd5, 0x84, 0x72, 0x08, 0x2f, 0xa3, 0x87, 0x64, 0xa4, 0xf0,
- 0xbf, 0x11, 0xc8, 0x9d, 0x20, 0x0f, 0xb4, 0x0b, 0xd1, 0x20, 0x08, 0xa6, 0x9c, 0x1e, 0x65, 0x01,
- 0xf0, 0x55, 0x88, 0x24, 0x6a, 0xe8, 0xe8, 0x00, 0x52, 0xd4, 0x9c, 0x2d, 0x8f, 0x46, 0xd8, 0x51,
- 0x06, 0xde, 0x1c, 0xf4, 0x8b, 0xd2, 0x0e, 0x13, 0x5e, 0xb9, 0x0b, 0x89, 0x83, 0x37, 0x74, 0x54,
- 0x84, 0xb4, 0x67, 0xab, 0xf8, 0x99, 0x41, 0x3c, 0xc3, 0x3a, 0x60, 0x71, 0x81, 0xa4, 0x80, 0x67,
- 0xd7, 0x85, 0xa4, 0xf0, 0x17, 0x51, 0x40, 0x27, 0x4f, 0x29, 0xfa, 0x97, 0x08, 0xbc, 0xe6, 0xc7,
- 0x00, 0xb6, 0x6b, 0x1c, 0x18, 0x96, 0x66, 0x8e, 0x04, 0x03, 0x11, 0xb6, 0xda, 0x9f, 0x5c, 0x85,
- 0x0a, 0x44, 0x80, 0xb0, 0x2d, 0xe0, 0xc7, 0x03, 0x85, 0xd7, 0xa8, 0x07, 0xe5, 0x81, 0xc2, 0x09,
- 0x95, 0x47, 0x4a, 0xbe, 0x77, 0x46, 0xe3, 0xc2, 0x06, 0x7c, 0xf5, 0x5c, 0xe0, 0xcb, 0xf8, 0xae,
- 0xc2, 0x0f, 0x22, 0x70, 0xe3, 0x0c, 0x8f, 0x12, 0xc6, 0xc9, 0x70, 0x9c, 0x0f, 0xc3, 0x38, 0xe9,
- 0xe5, 0x5f, 0xbf, 0x82, 0xd7, 0x0a, 0x0d, 0xa2, 0x19, 0x97, 0x22, 0x72, 0xb4, 0xf4, 0x0e, 0xcc,
- 0x8a, 0x46, 0xbe, 0x1f, 0x45, 0x5f, 0x05, 0x38, 0x34, 0x0e, 0x0e, 0xd5, 0xa7, 0x9a, 0x87, 0x5d,
- 0x71, 0x33, 0x4a, 0x51, 0xc9, 0x47, 0x54, 0x50, 0xfa, 0x37, 0x09, 0x32, 0x8d, 0xae, 0x63, 0xbb,
- 0x9e, 0xef, 0x65, 0x1f, 0x42, 0x82, 0xf9, 0x05, 0x22, 0xf6, 0xaf, 0x7c, 0xce, 0x08, 0x47, 0x5a,
- 0x72, 0xff, 0x22, 0x68, 0x41, 0x60, 0x04, 0xee, 0x2f, 0x7a, 0xaa, 0xfb, 0x7b, 0x1f, 0x12, 0xfc,
- 0xf6, 0x2a, 0x2e, 0x57, 0xc5, 0x50, 0x5f, 0xfe, 0xd5, 0xb0, 0xb1, 0xbd, 0x6a, 0x98, 0x78, 0x95,
- 0xa9, 0xf9, 0xe0, 0xbc, 0x11, 0xfa, 0x1a, 0x48, 0x84, 0x78, 0x2a, 0x31, 0xbe, 0xcf, 0xbd, 0x67,
- 0x8c, 0xdf, 0xb0, 0x5a, 0xad, 0xdd, 0x96, 0xf1, 0x7d, 0xac, 0x24, 0x09, 0xf1, 0xe8, 0x07, 0x2a,
- 0x80, 0xf4, 0x54, 0x33, 0x4d, 0xe6, 0x65, 0xa7, 0xd9, 0x8d, 0x32, 0x28, 0x8f, 0x1e, 0xb3, 0xc4,
- 0x97, 0x7b, 0xcc, 0x84, 0xc3, 0x74, 0x34, 0xef, 0x90, 0x45, 0x8e, 0x29, 0x05, 0xb8, 0x68, 0x47,
- 0xf3, 0x0e, 0x51, 0x1e, 0x92, 0x44, 0xa3, 0xbe, 0x8b, 0xe4, 0xa5, 0xc5, 0xd8, 0xed, 0x19, 0xc5,
- 0x2f, 0xa2, 0x05, 0x60, 0x9e, 0x97, 0x17, 0x99, 0x13, 0x8b, 0x29, 0x21, 0x09, 0x5b, 0x87, 0x23,
- 0xc3, 0x51, 0xf7, 0x8f, 0x08, 0x77, 0x5a, 0x62, 0x1d, 0x8e, 0x0c, 0x67, 0x75, 0x83, 0x28, 0x49,
- 0x5a, 0xb9, 0x7a, 0x44, 0x68, 0x78, 0x6f, 0x58, 0x07, 0x98, 0x78, 0xaa, 0x6e, 0xb8, 0xb8, 0xe3,
- 0x99, 0xc7, 0xcc, 0x61, 0x49, 0x4a, 0x96, 0x8b, 0x6b, 0x42, 0x8a, 0xee, 0x80, 0x3c, 0xee, 0x66,
- 0x99, 0xa3, 0x91, 0x94, 0xd9, 0x31, 0x2f, 0x4b, 0x55, 0xf9, 0x56, 0x87, 0x1c, 0x67, 0x86, 0xab,
- 0x72, 0xf9, 0xd0, 0x67, 0x96, 0x61, 0xce, 0xd1, 0x5c, 0x82, 0xd5, 0x76, 0xcf, 0xd2, 0x4d, 0xac,
- 0x72, 0xae, 0xce, 0x67, 0x99, 0x76, 0x8e, 0x55, 0x55, 0x58, 0x0d, 0xa7, 0xf5, 0x8b, 0x62, 0xef,
- 0xeb, 0xff, 0x0f, 0xb1, 0x77, 0xe1, 0xc7, 0x51, 0x98, 0x66, 0x76, 0x8e, 0xee, 0x43, 0x9c, 0x6e,
- 0xb3, 0xb8, 0xc4, 0x4d, 0x1a, 0x73, 0xb1, 0x36, 0xf4, 0xfa, 0x6c, 0x69, 0x5d, 0x9c, 0x47, 0xcc,
- 0x08, 0xd8, 0x37, 0xba, 0x01, 0x49, 0x82, 0x1f, 0xab, 0x4f, 0x34, 0x33, 0x3f, 0xc7, 0x76, 0x38,
- 0x41, 0xf0, 0xe3, 0x47, 0x9a, 0x89, 0xae, 0x41, 0xc2, 0x20, 0xaa, 0x85, 0x9f, 0xe6, 0xe7, 0xd9,
- 0x4a, 0x4d, 0x1b, 0x64, 0x0b, 0x3f, 0x65, 0xb4, 0xad, 0xb9, 0x07, 0xd8, 0x53, 0x3b, 0xb6, 0x49,
- 0xf2, 0xd7, 0xe8, 0x01, 0xa3, 0x21, 0x1d, 0x15, 0x55, 0x6d, 0x93, 0xa0, 0x5f, 0x81, 0xd4, 0x53,
- 0x8d, 0xa8, 0xb8, 0xeb, 0x78, 0xc7, 0x6c, 0xb1, 0x24, 0x6a, 0xf6, 0xa4, 0x4e, 0xcb, 0xcd, 0xb8,
- 0x14, 0x95, 0x63, 0xcd, 0xb8, 0x14, 0x93, 0xe3, 0xcd, 0xb8, 0x14, 0x97, 0xa7, 0x9b, 0x71, 0x69,
- 0x5a, 0x4e, 0x34, 0xe3, 0x52, 0x42, 0x4e, 0x36, 0xe3, 0x52, 0x52, 0x96, 0x9a, 0x71, 0x49, 0x92,
- 0x53, 0xcd, 0xb8, 0x94, 0x92, 0xa1, 0x19, 0x97, 0x40, 0x4e, 0x37, 0xe3, 0x52, 0x5a, 0x9e, 0x69,
- 0xc6, 0xa5, 0x19, 0x39, 0xd3, 0x8c, 0x4b, 0x19, 0x39, 0xdb, 0x8c, 0x4b, 0x59, 0x79, 0xb6, 0x19,
- 0x97, 0x66, 0x65, 0xb9, 0x19, 0x97, 0x64, 0x39, 0xd7, 0x8c, 0x4b, 0x39, 0x19, 0x95, 0xfe, 0x3b,
- 0x02, 0x59, 0xce, 0x0e, 0x01, 0x13, 0xbd, 0x09, 0x39, 0x66, 0xaf, 0x86, 0x75, 0xa0, 0x3a, 0x42,
- 0xc8, 0x38, 0x26, 0xaa, 0xc8, 0x7e, 0x45, 0xa0, 0xfc, 0x3a, 0x64, 0x5c, 0xac, 0xe9, 0x43, 0xc5,
- 0x28, 0x53, 0x9c, 0xa1, 0xc2, 0x40, 0xe9, 0x26, 0x64, 0x19, 0x11, 0x0e, 0xb5, 0x62, 0x4c, 0x2b,
- 0xc3, 0xa4, 0x81, 0x5a, 0x05, 0x32, 0xc4, 0xd1, 0xac, 0xa1, 0x56, 0x9c, 0x11, 0xdb, 0x8d, 0x53,
- 0xc8, 0xa6, 0xe5, 0x68, 0x96, 0x20, 0x99, 0x19, 0xda, 0x26, 0x4c, 0xa3, 0x2e, 0x26, 0xbd, 0x2e,
- 0x56, 0x1d, 0x9b, 0xc7, 0xd9, 0x31, 0x25, 0xc5, 0x25, 0x3b, 0x36, 0x29, 0xfd, 0x36, 0xdc, 0xa0,
- 0x31, 0x0d, 0x37, 0xdc, 0xea, 0xa1, 0x66, 0x1d, 0x04, 0xb7, 0x16, 0x0d, 0x92, 0x2c, 0x2e, 0x0a,
- 0x42, 0x84, 0xf5, 0x41, 0xbf, 0x98, 0xa0, 0xda, 0x57, 0x26, 0x97, 0x04, 0x05, 0x6e, 0xe8, 0xa5,
- 0x02, 0xe4, 0xc7, 0x7b, 0x0f, 0xee, 0x51, 0x0a, 0xbb, 0x46, 0xf5, 0xba, 0x98, 0x4e, 0xed, 0xa1,
- 0x41, 0x3c, 0xf4, 0x1d, 0x98, 0x11, 0x53, 0xa1, 0x33, 0xf4, 0x69, 0xfe, 0x82, 0xd5, 0x48, 0xbb,
- 0x01, 0x08, 0x29, 0xfd, 0x63, 0x04, 0xe6, 0x6a, 0xae, 0xed, 0x38, 0x58, 0x17, 0xf6, 0xcd, 0xa7,
- 0xea, 0x9b, 0x75, 0x24, 0x64, 0xd6, 0x5b, 0x10, 0x6d, 0xd4, 0x44, 0xfc, 0xf2, 0xe0, 0xaa, 0x61,
- 0x51, 0xa3, 0x86, 0xbe, 0x05, 0x09, 0x1a, 0xbe, 0xf7, 0x08, 0x73, 0x19, 0xd9, 0x13, 0x81, 0x6a,
- 0xd8, 0x3d, 0xb5, 0x98, 0xa2, 0x22, 0x1a, 0x94, 0xfe, 0x30, 0x01, 0xd7, 0xc2, 0x6b, 0xb4, 0x56,
- 0xf5, 0x07, 0xfe, 0x29, 0x24, 0x0d, 0x4b, 0xc7, 0xcf, 0x02, 0xa7, 0xf7, 0xfe, 0x79, 0xa8, 0xa7,
- 0x41, 0x94, 0xc5, 0x7a, 0x34, 0x28, 0x8c, 0x1f, 0x1a, 0x0b, 0x4c, 0xf4, 0xdd, 0xc0, 0xa5, 0xf2,
- 0xab, 0xea, 0xfd, 0x57, 0x46, 0xaf, 0x8d, 0xb9, 0xd7, 0x11, 0xef, 0x15, 0x63, 0x2e, 0xf0, 0xcb,
- 0xf1, 0x5e, 0x2d, 0xc8, 0x19, 0x96, 0x87, 0x5d, 0x13, 0x6b, 0x4f, 0x28, 0x19, 0xd3, 0xee, 0xc5,
- 0x8d, 0x75, 0x52, 0xea, 0x93, 0x43, 0x00, 0x9c, 0x42, 0x3f, 0x85, 0xb9, 0x30, 0xa8, 0xbf, 0x05,
- 0xe7, 0xdf, 0x62, 0xd9, 0x0a, 0x0f, 0x61, 0xfd, 0xcb, 0x62, 0x08, 0xa8, 0xc1, 0x71, 0x0a, 0x7f,
- 0x1a, 0x81, 0x99, 0xf0, 0xb6, 0x20, 0x03, 0x24, 0xd6, 0x87, 0x7f, 0x16, 0x63, 0x95, 0x2d, 0xea,
- 0x27, 0x59, 0x25, 0x5b, 0xab, 0x0f, 0x5e, 0x79, 0xad, 0x38, 0x84, 0xd8, 0xf2, 0x86, 0x4e, 0xc9,
- 0x57, 0x77, 0x6d, 0x67, 0x98, 0xa5, 0x8b, 0x29, 0x12, 0x15, 0x50, 0x7f, 0x52, 0xf8, 0x1d, 0x48,
- 0x05, 0x1b, 0x1a, 0xba, 0x3d, 0xc4, 0xbe, 0xc0, 0xdb, 0xc3, 0x79, 0xfd, 0x97, 0x7e, 0x9e, 0x80,
- 0xb9, 0xd3, 0xa8, 0xea, 0x63, 0x90, 0x43, 0xcc, 0xa0, 0x9a, 0x06, 0xf1, 0x84, 0xc5, 0xde, 0x39,
- 0x3f, 0x4c, 0x0d, 0xd1, 0x8b, 0xd8, 0x8f, 0xac, 0x3b, 0x4a, 0x3a, 0xdf, 0x83, 0xac, 0xce, 0xa7,
- 0xac, 0x8a, 0xa3, 0x10, 0xbb, 0x30, 0xba, 0x3c, 0x85, 0x62, 0x04, 0x7a, 0x46, 0x0f, 0x55, 0x11,
- 0x96, 0x8d, 0xf4, 0xd1, 0x83, 0x1b, 0x9f, 0xa1, 0x33, 0xfb, 0xcc, 0x54, 0x5a, 0x83, 0x7e, 0x31,
- 0x27, 0xb0, 0xfc, 0x2b, 0xde, 0x95, 0xd7, 0x38, 0xa7, 0x8f, 0x01, 0xea, 0x94, 0xe8, 0x69, 0x3d,
- 0xed, 0x78, 0x7a, 0x48, 0xf4, 0xd4, 0x52, 0xaf, 0x4e, 0xf4, 0xf4, 0xb3, 0xa1, 0xa3, 0x3f, 0x88,
- 0x40, 0x8e, 0xe7, 0x7b, 0xba, 0x3d, 0x4f, 0xe3, 0x49, 0x3c, 0x3f, 0x6a, 0xfd, 0x78, 0xd0, 0x2f,
- 0xce, 0xb2, 0x05, 0xd9, 0x14, 0x75, 0xac, 0xdb, 0xca, 0xab, 0x76, 0x3b, 0x44, 0x11, 0x91, 0x5c,
- 0x20, 0xd0, 0xd1, 0x06, 0x64, 0x79, 0x08, 0xae, 0xd2, 0x20, 0xd4, 0xb0, 0x2d, 0x16, 0xce, 0x66,
- 0x2a, 0x6f, 0xbc, 0xec, 0x17, 0x17, 0x4f, 0xb1, 0x2c, 0x1e, 0xbd, 0x3f, 0xe2, 0xba, 0x4a, 0x66,
- 0x3f, 0x5c, 0x44, 0x1d, 0xc8, 0x04, 0xa6, 0x71, 0xec, 0x88, 0xe8, 0xf7, 0xea, 0xce, 0x62, 0xc6,
- 0xb7, 0x11, 0x8a, 0x89, 0x0e, 0x60, 0xd6, 0xef, 0x84, 0x87, 0x9d, 0x24, 0x9f, 0xfa, 0x42, 0xba,
- 0xf1, 0xcd, 0x9a, 0xcf, 0x9a, 0x88, 0x8b, 0xd8, 0x75, 0x98, 0x3f, 0xd5, 0x1b, 0xff, 0xd9, 0x34,
- 0x5c, 0x1f, 0x65, 0xf8, 0x20, 0xc2, 0x50, 0xc7, 0x7d, 0xd0, 0x07, 0x13, 0x7b, 0x09, 0x1f, 0x83,
- 0xd3, 0x90, 0x5f, 0x1a, 0xf7, 0x42, 0x9f, 0x8e, 0x79, 0xa1, 0x57, 0xc0, 0x67, 0xe6, 0x35, 0x86,
- 0x2f, 0x40, 0x0b, 0xff, 0x1a, 0x81, 0xcc, 0x48, 0xff, 0xbf, 0x4c, 0xba, 0xdd, 0x09, 0xa2, 0x02,
- 0xfe, 0xd0, 0xf7, 0xde, 0xe5, 0xe7, 0x36, 0x1a, 0x2c, 0x14, 0xfe, 0x39, 0x02, 0x99, 0x91, 0xe9,
- 0x7e, 0x49, 0x44, 0xfd, 0x85, 0x8f, 0xbc, 0xf4, 0x4d, 0x48, 0x70, 0x09, 0x42, 0x90, 0xfd, 0x68,
- 0xa5, 0xb1, 0xdb, 0xd8, 0x5a, 0x53, 0x57, 0xb7, 0x15, 0x75, 0xad, 0x2a, 0x4f, 0xa1, 0x19, 0x90,
- 0x6a, 0xf5, 0x87, 0x75, 0x2a, 0x94, 0x23, 0x28, 0x0d, 0x49, 0x56, 0xaa, 0xd7, 0xe4, 0x68, 0xa9,
- 0x02, 0x32, 0xc7, 0xde, 0xc7, 0x94, 0x5a, 0xe9, 0x45, 0x82, 0xde, 0xd9, 0x28, 0x2c, 0xee, 0xd2,
- 0x18, 0x83, 0x3a, 0x13, 0x35, 0x14, 0xe1, 0xe5, 0x82, 0x2a, 0xea, 0x56, 0xb6, 0xb4, 0x2e, 0x2e,
- 0xfd, 0x53, 0x1c, 0x72, 0x43, 0x10, 0xdf, 0xb1, 0xd0, 0x0b, 0xaa, 0x61, 0x1d, 0xa9, 0xc3, 0x77,
- 0x27, 0x7e, 0x41, 0x35, 0xac, 0xa3, 0x3d, 0xa5, 0xa1, 0x24, 0x69, 0xe5, 0x9e, 0x6b, 0xa0, 0x26,
- 0xc4, 0x6d, 0xc7, 0xf3, 0x03, 0xf4, 0x77, 0xcf, 0x59, 0x8a, 0x13, 0x7d, 0x94, 0xb7, 0x1d, 0x8f,
- 0x27, 0x5d, 0x14, 0x86, 0x81, 0xfe, 0x3e, 0x02, 0x49, 0x7e, 0x1b, 0x22, 0xf9, 0x04, 0xc3, 0xfb,
- 0xd6, 0xa5, 0xf0, 0xf8, 0x02, 0x88, 0x97, 0x81, 0x8f, 0xa8, 0xa9, 0xbf, 0xec, 0x17, 0x73, 0xe3,
- 0x0b, 0x44, 0xae, 0xf8, 0x64, 0xe0, 0x0f, 0x11, 0x35, 0x79, 0xb2, 0x7a, 0xb8, 0xd0, 0x8c, 0x52,
- 0x27, 0x7c, 0x0f, 0xc8, 0x8c, 0x6c, 0x44, 0xe1, 0x00, 0x66, 0xc2, 0xa3, 0x3f, 0x25, 0x0b, 0xb5,
- 0x32, 0x9a, 0x85, 0x7a, 0x73, 0xa2, 0x95, 0xe1, 0x98, 0xe1, 0xd4, 0xd7, 0x37, 0x21, 0x15, 0x2c,
- 0xfb, 0x65, 0x72, 0x66, 0x9c, 0x25, 0x83, 0x4b, 0xe8, 0xb4, 0x9c, 0x28, 0xfd, 0x43, 0x04, 0x66,
- 0x14, 0x4c, 0x6c, 0xf3, 0x09, 0xd6, 0x69, 0xd4, 0x80, 0xbe, 0x0e, 0x71, 0x1a, 0x85, 0x88, 0xfb,
- 0xf5, 0x05, 0xd7, 0x13, 0xa6, 0x8a, 0x56, 0x20, 0x15, 0x64, 0x09, 0x2e, 0xf3, 0x34, 0x3a, 0x6c,
- 0x85, 0xee, 0x80, 0xdc, 0xb6, 0x7b, 0x96, 0xae, 0xb9, 0xc7, 0xaa, 0x8b, 0xb5, 0xce, 0x21, 0xd6,
- 0x45, 0x46, 0x74, 0xd6, 0x97, 0x2b, 0x5c, 0x5c, 0xfa, 0x61, 0x14, 0xd0, 0x70, 0x71, 0x42, 0x34,
- 0x41, 0x63, 0x1f, 0x36, 0x0f, 0x71, 0xc1, 0x8a, 0x9e, 0x9a, 0xc3, 0x1e, 0x0b, 0xa1, 0x82, 0x89,
- 0xfb, 0x5b, 0xea, 0x86, 0x64, 0x04, 0xfd, 0xc9, 0xf9, 0xb9, 0x90, 0x18, 0xcb, 0x85, 0x30, 0x2b,
- 0xfd, 0xa5, 0xe6, 0x43, 0x84, 0xbb, 0xfb, 0x9f, 0x38, 0xa0, 0xaa, 0x8b, 0x35, 0x0f, 0x53, 0xe6,
- 0x21, 0xe7, 0xdd, 0x07, 0x2b, 0x30, 0xcd, 0x2f, 0x0f, 0xd1, 0xcb, 0x5c, 0x1e, 0xc4, 0xa2, 0xf0,
- 0xa6, 0xe8, 0x37, 0x60, 0xa6, 0x63, 0x9b, 0xbd, 0xae, 0xa5, 0xb2, 0x97, 0x1c, 0x11, 0x4a, 0x7e,
- 0xe3, 0x3c, 0x23, 0x3e, 0x31, 0xb8, 0x72, 0xd5, 0x36, 0x69, 0xd9, 0xbf, 0xdf, 0x72, 0x40, 0xa6,
- 0x81, 0x5e, 0x83, 0x54, 0x70, 0xa0, 0x58, 0x10, 0x99, 0x52, 0x86, 0x02, 0xb4, 0x0c, 0xd3, 0x1a,
- 0x51, 0xed, 0x7d, 0x16, 0xe5, 0x5d, 0x64, 0x61, 0x4a, 0x5c, 0x23, 0xdb, 0xfb, 0xe8, 0x1e, 0x64,
- 0xf6, 0x1f, 0xf3, 0xc8, 0x97, 0x13, 0x28, 0x7f, 0x60, 0x9b, 0x1d, 0xf4, 0x8b, 0xe9, 0xd5, 0x0f,
- 0xd9, 0x64, 0x29, 0x7d, 0x2a, 0xe9, 0xfd, 0xc7, 0x41, 0x01, 0xdd, 0x85, 0x5c, 0x57, 0x7b, 0xa6,
- 0xee, 0xbb, 0x5a, 0x47, 0x84, 0x7a, 0x26, 0x67, 0x85, 0x88, 0x32, 0xdb, 0xd5, 0x9e, 0xad, 0x0a,
+ 0x41, 0x32, 0xc4, 0x8e, 0xf0, 0x31, 0x1b, 0xcf, 0x8c, 0x42, 0x7f, 0xa2, 0x0f, 0x20, 0xde, 0xb5,
+ 0x75, 0xcc, 0x80, 0xb2, 0xcb, 0xb7, 0xcb, 0x67, 0xee, 0x49, 0x79, 0x88, 0xb6, 0x69, 0xeb, 0x58,
+ 0x61, 0xcd, 0x50, 0x1b, 0xa4, 0xa3, 0x2e, 0x51, 0x0d, 0x6b, 0xdf, 0xce, 0xc7, 0x16, 0x23, 0xb7,
+ 0xd2, 0xcb, 0xf7, 0xce, 0x81, 0x38, 0x63, 0x58, 0xe5, 0x8d, 0xcd, 0x56, 0xc3, 0xda, 0xb7, 0x2b,
+ 0xe9, 0x41, 0xbf, 0x98, 0x14, 0x05, 0x25, 0x79, 0xd4, 0x25, 0xf4, 0x47, 0x61, 0x1b, 0x7c, 0x19,
+ 0x1d, 0x7f, 0xcf, 0x35, 0xd8, 0xf8, 0x53, 0x0a, 0xfd, 0x89, 0xde, 0x06, 0x84, 0x39, 0x1e, 0xd6,
+ 0x55, 0x6a, 0x00, 0x2a, 0x9d, 0x60, 0x94, 0x4d, 0x50, 0x0e, 0x6a, 0x6a, 0x9a, 0xa7, 0x6d, 0xe0,
+ 0x63, 0xbe, 0x42, 0x62, 0x9d, 0x7e, 0x2f, 0x06, 0xd9, 0xe1, 0x50, 0x18, 0xfc, 0x3a, 0x24, 0x48,
+ 0xe7, 0x10, 0x77, 0x31, 0xeb, 0x21, 0xbb, 0xfc, 0xee, 0x44, 0xcb, 0x41, 0x9b, 0x96, 0x5b, 0xac,
+ 0x9d, 0x22, 0xda, 0x23, 0x04, 0x71, 0xa2, 0x99, 0x9e, 0x18, 0x08, 0xfb, 0x8d, 0xfe, 0x3c, 0x02,
+ 0x8b, 0xe3, 0x23, 0xaa, 0x1c, 0x6f, 0x6c, 0xb6, 0x36, 0x35, 0xe2, 0x61, 0x77, 0x03, 0x1f, 0x37,
+ 0x6a, 0xf9, 0xd8, 0x62, 0xec, 0x56, 0x7a, 0x79, 0x7b, 0xf2, 0x8e, 0xeb, 0x17, 0x20, 0xd6, 0x2d,
+ 0xcf, 0x3d, 0x56, 0x2e, 0xec, 0xb8, 0xd0, 0x82, 0x37, 0x27, 0x82, 0x0a, 0xdb, 0x50, 0x8a, 0xdb,
+ 0xd0, 0x3c, 0x4c, 0x3f, 0xd1, 0xcc, 0x1e, 0x16, 0xb3, 0xe5, 0x85, 0x7b, 0xd1, 0xf7, 0x23, 0xa5,
+ 0x1b, 0x90, 0xe0, 0x0b, 0x83, 0x32, 0x90, 0x5a, 0xa9, 0xb7, 0x96, 0xbf, 0xf1, 0xde, 0x5a, 0x75,
+ 0x53, 0x9e, 0x12, 0x5b, 0xf0, 0x37, 0x09, 0xc8, 0x70, 0x9b, 0xa8, 0x61, 0x4f, 0x33, 0x4c, 0x82,
+ 0x2a, 0x00, 0xc4, 0xd3, 0x5c, 0x4f, 0xa5, 0x6e, 0xc5, 0xfa, 0x48, 0x2f, 0x7f, 0x35, 0xb4, 0x18,
+ 0xd4, 0xed, 0xca, 0x87, 0x66, 0xa7, 0xbc, 0xeb, 0xbb, 0x5d, 0x25, 0xfe, 0x59, 0xbf, 0x38, 0xa5,
+ 0xa4, 0x58, 0x33, 0x2a, 0x45, 0xf7, 0x41, 0xc2, 0x96, 0xce, 0x11, 0xa2, 0x93, 0x23, 0x24, 0xb1,
+ 0xa5, 0xb3, 0xf6, 0x5f, 0xe1, 0x46, 0x46, 0xcd, 0x39, 0x55, 0x49, 0x0e, 0xfa, 0xc5, 0xd8, 0x9e,
+ 0xd2, 0xe0, 0xd6, 0x76, 0x13, 0x66, 0xdb, 0x6c, 0xbc, 0x6a, 0x57, 0xb3, 0x8c, 0x7d, 0x4c, 0xbc,
+ 0x7c, 0x9c, 0xcd, 0x39, 0xcb, 0xc5, 0x9b, 0x42, 0x8a, 0x7e, 0x3f, 0x02, 0x73, 0x3d, 0xd7, 0x20,
+ 0x6a, 0xfb, 0x58, 0x35, 0xed, 0x8e, 0x66, 0x1a, 0xde, 0xb1, 0x7a, 0xf4, 0x24, 0x3f, 0xcd, 0xb6,
+ 0xf7, 0xfe, 0x85, 0x3e, 0x22, 0xd6, 0xa3, 0xbc, 0xe7, 0x1a, 0xa4, 0x72, 0xfc, 0x40, 0x20, 0x6c,
+ 0x3c, 0x61, 0x5b, 0x50, 0x99, 0x1f, 0xf4, 0x8b, 0xf2, 0x9e, 0xd2, 0x08, 0x57, 0x3d, 0x52, 0xe4,
+ 0xde, 0x98, 0x32, 0xd2, 0x02, 0xe7, 0x30, 0x6c, 0x4b, 0xb5, 0xb9, 0xb7, 0xe5, 0x13, 0x6c, 0x4d,
+ 0x96, 0x2f, 0xef, 0xa7, 0x4a, 0x0e, 0x9f, 0x60, 0x94, 0x3f, 0x8e, 0x40, 0x81, 0x12, 0x1f, 0xee,
+ 0x50, 0x07, 0x0c, 0xc8, 0x50, 0x75, 0x71, 0xc7, 0x76, 0xf5, 0x7c, 0x92, 0xae, 0x4e, 0xa5, 0xf5,
+ 0x6f, 0x93, 0xf2, 0x1d, 0xa3, 0xd5, 0x5e, 0xcf, 0xd0, 0xcb, 0x7b, 0x7b, 0x8d, 0xda, 0xa0, 0x5f,
+ 0xcc, 0xef, 0xf8, 0xe0, 0xc1, 0x7e, 0x29, 0x0c, 0x5a, 0xc9, 0x3b, 0x67, 0xd4, 0xa0, 0xf7, 0x21,
+ 0xdb, 0xb1, 0x4d, 0x13, 0x77, 0xd8, 0xb4, 0xf7, 0x94, 0x46, 0x5e, 0x62, 0x7b, 0x99, 0x1b, 0xf4,
+ 0x8b, 0x99, 0x6a, 0x50, 0x43, 0x77, 0x35, 0xd3, 0x09, 0x17, 0x91, 0x02, 0xb3, 0xa1, 0x05, 0x63,
+ 0xac, 0x96, 0x62, 0xab, 0x75, 0x7b, 0x62, 0x87, 0x54, 0xb2, 0x78, 0xa4, 0x5c, 0xa8, 0xc2, 0xb5,
+ 0x53, 0x77, 0xf1, 0x22, 0x47, 0x4a, 0x85, 0x1d, 0x49, 0x86, 0x2c, 0xdf, 0x94, 0x1d, 0xd7, 0x3e,
+ 0x70, 0x31, 0x21, 0xa5, 0x9f, 0x64, 0x21, 0xab, 0x60, 0xe2, 0xd9, 0x2e, 0xf6, 0x9d, 0xe7, 0x27,
+ 0x11, 0x98, 0xa3, 0xc1, 0xca, 0x35, 0x1c, 0xcf, 0x76, 0x55, 0x17, 0x3f, 0x75, 0x0d, 0x0f, 0x93,
+ 0x7c, 0x94, 0x19, 0xdd, 0xca, 0x39, 0x53, 0x18, 0x05, 0x2a, 0xd7, 0x02, 0x10, 0x45, 0x60, 0x70,
+ 0xbb, 0xbb, 0xff, 0x83, 0x9f, 0x17, 0xef, 0x4d, 0xb4, 0x8f, 0x27, 0xe3, 0x67, 0xb9, 0x51, 0x53,
+ 0x90, 0x7e, 0x02, 0x18, 0xbd, 0x06, 0x71, 0x6a, 0xb7, 0x8c, 0xf8, 0x52, 0x15, 0x69, 0xd0, 0x2f,
+ 0xc6, 0xa9, 0x65, 0x2b, 0x4c, 0x3a, 0xe2, 0xcb, 0xf1, 0x57, 0xf0, 0xe5, 0x35, 0x48, 0x7b, 0x5a,
+ 0xdb, 0xc4, 0x2a, 0xed, 0x99, 0x08, 0xf7, 0xfb, 0xda, 0xd8, 0x4a, 0x90, 0xc7, 0x66, 0x5b, 0x23,
+ 0xb8, 0xbc, 0x4b, 0x35, 0x43, 0x73, 0x07, 0xcf, 0x17, 0x10, 0xb4, 0x04, 0x69, 0xfb, 0x09, 0x76,
+ 0x5d, 0x43, 0xc7, 0xaa, 0xde, 0x66, 0x3e, 0x94, 0xaa, 0x64, 0x07, 0xfd, 0x22, 0x6c, 0x0b, 0x71,
+ 0xad, 0xa2, 0x80, 0xaf, 0x52, 0x6b, 0x23, 0x0f, 0xe6, 0x05, 0x55, 0x04, 0xfe, 0xcf, 0xec, 0x29,
+ 0xc9, 0x86, 0xf0, 0xed, 0xc9, 0x37, 0x83, 0xef, 0xbb, 0x6f, 0x3c, 0x2c, 0x4e, 0xf2, 0x49, 0xa2,
+ 0xf6, 0x89, 0x1a, 0xf4, 0x16, 0xe4, 0x1c, 0x17, 0x3b, 0x9a, 0x8b, 0xd5, 0x8e, 0xdd, 0x75, 0x4c,
+ 0xec, 0x61, 0x9d, 0x59, 0xbf, 0xa4, 0xc8, 0xa2, 0xa2, 0xea, 0xcb, 0xd1, 0x9b, 0x90, 0x25, 0x9e,
+ 0xe6, 0xd1, 0xf0, 0x4d, 0xb0, 0x4b, 0x35, 0x53, 0x4c, 0x33, 0xc3, 0xa4, 0x0d, 0x21, 0x44, 0x77,
+ 0xe1, 0xda, 0x70, 0xdf, 0x88, 0xea, 0xf4, 0xda, 0xa6, 0x41, 0x0e, 0xb1, 0x9e, 0x07, 0xa6, 0x3d,
+ 0x1f, 0xaa, 0xdc, 0xf1, 0xeb, 0xd0, 0xf1, 0x88, 0x29, 0x76, 0xe8, 0xc2, 0x68, 0x07, 0x38, 0x9f,
+ 0x5e, 0x8c, 0xdc, 0x9a, 0xae, 0xac, 0xbf, 0xec, 0x17, 0x6b, 0x13, 0xdb, 0x11, 0xc1, 0xdd, 0x25,
+ 0xcf, 0xc5, 0x38, 0x64, 0x96, 0x55, 0x81, 0x17, 0xb6, 0x28, 0x5f, 0x86, 0x14, 0x80, 0xa1, 0x0b,
+ 0xe6, 0x67, 0x5e, 0x99, 0xed, 0x42, 0x28, 0x68, 0x05, 0x92, 0x3c, 0xff, 0x23, 0xf9, 0x0c, 0xdb,
+ 0xc0, 0x5f, 0x3d, 0xcb, 0x86, 0x98, 0x56, 0x68, 0x97, 0xfc, 0x76, 0xa8, 0x06, 0xe0, 0x1d, 0x3b,
+ 0xbe, 0x25, 0x66, 0x19, 0xca, 0x9b, 0x67, 0xa1, 0x1c, 0x3b, 0x61, 0x43, 0x4c, 0x79, 0xa2, 0x4c,
+ 0x50, 0x13, 0x66, 0x58, 0x8a, 0xa1, 0x09, 0x9c, 0x59, 0x86, 0x73, 0xf3, 0x0c, 0x1c, 0x16, 0x7c,
+ 0xb5, 0x10, 0x52, 0x9a, 0x04, 0x12, 0x82, 0x76, 0x20, 0x4b, 0x33, 0x26, 0xaa, 0x29, 0xd0, 0x64,
+ 0x86, 0x76, 0xfb, 0x0c, 0xb4, 0x9a, 0x50, 0x0e, 0xe1, 0x65, 0xf4, 0x90, 0x8c, 0x14, 0xfe, 0x37,
+ 0x02, 0xb9, 0x13, 0xe4, 0x81, 0x76, 0x21, 0x1a, 0x24, 0xc1, 0x94, 0xd3, 0xa3, 0x2c, 0x01, 0xbe,
+ 0x0a, 0x91, 0x44, 0x0d, 0x1d, 0x1d, 0x40, 0x8a, 0x9a, 0xb3, 0xe5, 0xd1, 0x0c, 0x3b, 0xca, 0xc0,
+ 0x9b, 0x83, 0x7e, 0x51, 0xda, 0x61, 0xc2, 0x2b, 0x77, 0x21, 0x71, 0xf0, 0x86, 0x8e, 0x8a, 0x90,
+ 0xf6, 0x6c, 0x15, 0x3f, 0x33, 0x88, 0x67, 0x58, 0x07, 0x2c, 0x2f, 0x90, 0x14, 0xf0, 0xec, 0xba,
+ 0x90, 0x14, 0xfe, 0x22, 0x0a, 0xe8, 0xa4, 0x97, 0xa2, 0x7f, 0x8e, 0xc0, 0x6b, 0x7e, 0x0e, 0x60,
+ 0xbb, 0xc6, 0x81, 0x61, 0x69, 0xe6, 0x48, 0x32, 0x10, 0x61, 0xab, 0xfd, 0xc9, 0x55, 0xa8, 0x40,
+ 0x24, 0x08, 0xdb, 0x02, 0x7e, 0x3c, 0x51, 0x78, 0x8d, 0x46, 0x50, 0x9e, 0x28, 0x9c, 0x50, 0x79,
+ 0xa4, 0xe4, 0x7b, 0x67, 0x34, 0x2e, 0x6c, 0xc0, 0x57, 0xcf, 0x05, 0xbe, 0x4c, 0xec, 0x2a, 0xfc,
+ 0x20, 0x02, 0x37, 0xce, 0x88, 0x28, 0x61, 0x9c, 0x0c, 0xc7, 0x79, 0x18, 0xc6, 0x49, 0x2f, 0xff,
+ 0xfa, 0x15, 0xa2, 0x56, 0x68, 0x10, 0xcd, 0xb8, 0x14, 0x91, 0xa3, 0xa5, 0x77, 0x61, 0x56, 0x34,
+ 0xf2, 0xe3, 0x28, 0xfa, 0x2a, 0xc0, 0xa1, 0x71, 0x70, 0xa8, 0x3e, 0xd5, 0x3c, 0xec, 0x8a, 0x93,
+ 0x51, 0x8a, 0x4a, 0x3e, 0xa2, 0x82, 0xd2, 0xbf, 0x4a, 0x90, 0x69, 0x74, 0x1d, 0xdb, 0xf5, 0xfc,
+ 0x28, 0xfb, 0x00, 0x12, 0x2c, 0x2e, 0x10, 0xb1, 0x7f, 0xe5, 0x73, 0x46, 0x38, 0xd2, 0x92, 0xc7,
+ 0x17, 0x41, 0x0b, 0x02, 0x23, 0x08, 0x7f, 0xd1, 0x53, 0xc3, 0xdf, 0x07, 0x90, 0xe0, 0xa7, 0x57,
+ 0x71, 0xb8, 0x2a, 0x86, 0xfa, 0xf2, 0x8f, 0x86, 0x8d, 0xed, 0x55, 0xc3, 0xc4, 0xab, 0x4c, 0xcd,
+ 0x07, 0xe7, 0x8d, 0xd0, 0xd7, 0x40, 0x22, 0xc4, 0x53, 0x89, 0xf1, 0x7d, 0x1e, 0x3d, 0x63, 0xfc,
+ 0x84, 0xd5, 0x6a, 0xed, 0xb6, 0x8c, 0xef, 0x63, 0x25, 0x49, 0x88, 0x47, 0x7f, 0xa0, 0x02, 0x48,
+ 0x4f, 0x35, 0xd3, 0x64, 0x51, 0x76, 0x9a, 0x9d, 0x28, 0x83, 0xf2, 0xa8, 0x9b, 0x25, 0xbe, 0x5c,
+ 0x37, 0x13, 0x01, 0xd3, 0xd1, 0xbc, 0x43, 0x96, 0x39, 0xa6, 0x14, 0xe0, 0xa2, 0x1d, 0xcd, 0x3b,
+ 0x44, 0x79, 0x48, 0x12, 0x8d, 0xc6, 0x2e, 0x92, 0x97, 0x16, 0x63, 0xb7, 0x66, 0x14, 0xbf, 0x88,
+ 0x16, 0x80, 0x45, 0x5e, 0x5e, 0x64, 0x41, 0x2c, 0xa6, 0x84, 0x24, 0x6c, 0x1d, 0x8e, 0x0c, 0x47,
+ 0xdd, 0x3f, 0x22, 0x3c, 0x68, 0x89, 0x75, 0x38, 0x32, 0x9c, 0xd5, 0x0d, 0xa2, 0x24, 0x69, 0xe5,
+ 0xea, 0x11, 0xa1, 0xe9, 0xbd, 0x61, 0x1d, 0x60, 0xe2, 0xa9, 0xba, 0xe1, 0xe2, 0x8e, 0x67, 0x1e,
+ 0xb3, 0x80, 0x25, 0x29, 0x59, 0x2e, 0xae, 0x09, 0x29, 0xba, 0x0d, 0xf2, 0x78, 0x98, 0x65, 0x81,
+ 0x46, 0x52, 0x66, 0xc7, 0xa2, 0x2c, 0x55, 0xe5, 0x5b, 0x1d, 0x0a, 0x9c, 0x19, 0xae, 0xca, 0xe5,
+ 0xc3, 0x98, 0x59, 0x86, 0x39, 0x47, 0x73, 0x09, 0x56, 0xdb, 0x3d, 0x4b, 0x37, 0xb1, 0xca, 0xb9,
+ 0x3a, 0x9f, 0x65, 0xda, 0x39, 0x56, 0x55, 0x61, 0x35, 0x9c, 0xd6, 0x2f, 0xca, 0xbd, 0xaf, 0xff,
+ 0x3f, 0xe4, 0xde, 0x85, 0x1f, 0x47, 0x61, 0x9a, 0xd9, 0x39, 0xba, 0x07, 0x71, 0xba, 0xcd, 0xe2,
+ 0x10, 0x37, 0x69, 0xce, 0xc5, 0xda, 0xd0, 0xe3, 0xb3, 0xa5, 0x75, 0x71, 0x1e, 0x31, 0x23, 0x60,
+ 0xbf, 0xd1, 0x0d, 0x48, 0x12, 0xfc, 0x58, 0x7d, 0xa2, 0x99, 0xf9, 0x39, 0xb6, 0xc3, 0x09, 0x82,
+ 0x1f, 0x3f, 0xd2, 0x4c, 0x74, 0x0d, 0x12, 0x06, 0x51, 0x2d, 0xfc, 0x34, 0x3f, 0xcf, 0x56, 0x6a,
+ 0xda, 0x20, 0x5b, 0xf8, 0x29, 0xa3, 0x6d, 0xcd, 0x3d, 0xc0, 0x9e, 0xda, 0xb1, 0x4d, 0x92, 0xbf,
+ 0x46, 0x1d, 0x8c, 0xa6, 0x74, 0x54, 0x54, 0xb5, 0x4d, 0x82, 0x7e, 0x05, 0x52, 0x4f, 0x35, 0xa2,
+ 0xe2, 0xae, 0xe3, 0x1d, 0xb3, 0xc5, 0x92, 0xa8, 0xd9, 0x93, 0x3a, 0x2d, 0x37, 0xe3, 0x52, 0x54,
+ 0x8e, 0x35, 0xe3, 0x52, 0x4c, 0x8e, 0x37, 0xe3, 0x52, 0x5c, 0x9e, 0x6e, 0xc6, 0xa5, 0x69, 0x39,
+ 0xd1, 0x8c, 0x4b, 0x09, 0x39, 0xd9, 0x8c, 0x4b, 0x49, 0x59, 0x6a, 0xc6, 0x25, 0x49, 0x4e, 0x35,
+ 0xe3, 0x52, 0x4a, 0x86, 0x66, 0x5c, 0x02, 0x39, 0xdd, 0x8c, 0x4b, 0x69, 0x79, 0xa6, 0x19, 0x97,
+ 0x66, 0xe4, 0x4c, 0x33, 0x2e, 0x65, 0xe4, 0x6c, 0x33, 0x2e, 0x65, 0xe5, 0xd9, 0x66, 0x5c, 0x9a,
+ 0x95, 0xe5, 0x66, 0x5c, 0x92, 0xe5, 0x5c, 0x33, 0x2e, 0xe5, 0x64, 0x54, 0xfa, 0xaf, 0x08, 0x64,
+ 0x39, 0x3b, 0x04, 0x4c, 0xf4, 0x16, 0xe4, 0x98, 0xbd, 0x1a, 0xd6, 0x81, 0xea, 0x08, 0x21, 0xe3,
+ 0x98, 0xa8, 0x22, 0xfb, 0x15, 0x81, 0xf2, 0xeb, 0x90, 0x71, 0xb1, 0xa6, 0x0f, 0x15, 0xa3, 0x4c,
+ 0x71, 0x86, 0x0a, 0x03, 0xa5, 0x37, 0x21, 0xcb, 0x88, 0x70, 0xa8, 0x15, 0x63, 0x5a, 0x19, 0x26,
+ 0x0d, 0xd4, 0x2a, 0x90, 0x21, 0x8e, 0x66, 0x0d, 0xb5, 0xe2, 0x8c, 0xd8, 0x6e, 0x9c, 0x42, 0x36,
+ 0x2d, 0x47, 0xb3, 0x04, 0xc9, 0xcc, 0xd0, 0x36, 0x61, 0x1a, 0x75, 0x31, 0xe9, 0x75, 0xb1, 0xea,
+ 0xd8, 0x3c, 0xcf, 0x8e, 0x29, 0x29, 0x2e, 0xd9, 0xb1, 0x49, 0xe9, 0xb7, 0xe1, 0x06, 0xcd, 0x69,
+ 0xb8, 0xe1, 0x56, 0x0f, 0x35, 0xeb, 0x20, 0x38, 0xb5, 0x68, 0x90, 0x64, 0x79, 0x51, 0x90, 0x22,
+ 0xac, 0x0f, 0xfa, 0xc5, 0x04, 0xd5, 0xbe, 0x32, 0xb9, 0x24, 0x28, 0x70, 0x43, 0x2f, 0x15, 0x20,
+ 0x3f, 0xde, 0x7b, 0x70, 0x8e, 0x52, 0xd8, 0x31, 0xaa, 0xd7, 0xc5, 0x74, 0x6a, 0x0f, 0x0c, 0xe2,
+ 0xa1, 0xef, 0xc0, 0x8c, 0x98, 0x0a, 0x9d, 0xa1, 0x4f, 0xf3, 0x17, 0xac, 0x46, 0xda, 0x0d, 0x40,
+ 0x48, 0xe9, 0x1f, 0x22, 0x30, 0x57, 0x73, 0x6d, 0xc7, 0xc1, 0xba, 0xb0, 0x6f, 0x3e, 0x55, 0xdf,
+ 0xac, 0x23, 0x21, 0xb3, 0xde, 0x82, 0x68, 0xa3, 0x26, 0xf2, 0x97, 0xfb, 0x57, 0x4d, 0x8b, 0x1a,
+ 0x35, 0xf4, 0x2d, 0x48, 0xd0, 0xf4, 0xbd, 0x47, 0x58, 0xc8, 0xc8, 0x9e, 0x48, 0x54, 0xc3, 0xe1,
+ 0xa9, 0xc5, 0x14, 0x15, 0xd1, 0xa0, 0xf4, 0x87, 0x09, 0xb8, 0x16, 0x5e, 0xa3, 0xb5, 0xaa, 0x3f,
+ 0xf0, 0x4f, 0x21, 0x69, 0x58, 0x3a, 0x7e, 0x16, 0x04, 0xbd, 0x0f, 0xce, 0x43, 0x3d, 0x0d, 0xa2,
+ 0x2c, 0xd6, 0xa3, 0x41, 0x61, 0xfc, 0xd4, 0x58, 0x60, 0xa2, 0xef, 0x06, 0x21, 0x95, 0x1f, 0x55,
+ 0xef, 0xbd, 0x32, 0x7a, 0x6d, 0x2c, 0xbc, 0x8e, 0x44, 0xaf, 0x18, 0x0b, 0x81, 0x5f, 0x4e, 0xf4,
+ 0x6a, 0x41, 0xce, 0xb0, 0x3c, 0xec, 0x9a, 0x58, 0x7b, 0x42, 0xc9, 0x98, 0x76, 0x2f, 0x4e, 0xac,
+ 0x93, 0x52, 0x9f, 0x1c, 0x02, 0xe0, 0x14, 0xfa, 0x29, 0xcc, 0x85, 0x41, 0xfd, 0x2d, 0x38, 0xff,
+ 0x14, 0xcb, 0x56, 0x78, 0x08, 0xeb, 0x1f, 0x16, 0x43, 0x40, 0x0d, 0x8e, 0x53, 0xf8, 0xd3, 0x08,
+ 0xcc, 0x84, 0xb7, 0x05, 0x19, 0x20, 0xb1, 0x3e, 0x7c, 0x5f, 0x8c, 0x55, 0xb6, 0x68, 0x9c, 0x64,
+ 0x95, 0x6c, 0xad, 0x3e, 0x7c, 0xe5, 0xb5, 0xe2, 0x10, 0x62, 0xcb, 0x1b, 0x3a, 0x25, 0x5f, 0xdd,
+ 0xb5, 0x9d, 0xe1, 0x2d, 0x5d, 0x4c, 0x91, 0xa8, 0x80, 0xc6, 0x93, 0xc2, 0xef, 0x40, 0x2a, 0xd8,
+ 0xd0, 0xd0, 0xe9, 0x21, 0xf6, 0x05, 0x9e, 0x1e, 0xce, 0xeb, 0xbf, 0xf4, 0xf3, 0x04, 0xcc, 0x9d,
+ 0x46, 0x55, 0x1f, 0x83, 0x1c, 0x62, 0x06, 0xd5, 0x34, 0x88, 0x27, 0x2c, 0xf6, 0xf6, 0xf9, 0x69,
+ 0x6a, 0x88, 0x5e, 0xc4, 0x7e, 0x64, 0xdd, 0x51, 0xd2, 0xf9, 0x1e, 0x64, 0x75, 0x3e, 0x65, 0x55,
+ 0xb8, 0x42, 0xec, 0xc2, 0xec, 0xf2, 0x14, 0x8a, 0x11, 0xe8, 0x19, 0x3d, 0x54, 0x45, 0xd8, 0x6d,
+ 0xa4, 0x8f, 0x1e, 0x9c, 0xf8, 0x0c, 0x9d, 0xd9, 0x67, 0xa6, 0xd2, 0x1a, 0xf4, 0x8b, 0x39, 0x81,
+ 0xe5, 0x1f, 0xf1, 0xae, 0xbc, 0xc6, 0x39, 0x7d, 0x0c, 0x50, 0xa7, 0x44, 0x4f, 0xeb, 0x69, 0xc7,
+ 0xd3, 0x43, 0xa2, 0xa7, 0x96, 0x7a, 0x75, 0xa2, 0xa7, 0x3f, 0x1b, 0x3a, 0xfa, 0x83, 0x08, 0xe4,
+ 0xf8, 0x7d, 0x4f, 0xb7, 0xe7, 0x69, 0xfc, 0x12, 0xcf, 0xcf, 0x5a, 0x3f, 0x1e, 0xf4, 0x8b, 0xb3,
+ 0x6c, 0x41, 0x36, 0x45, 0x1d, 0xeb, 0xb6, 0xf2, 0xaa, 0xdd, 0x0e, 0x51, 0x44, 0x26, 0x17, 0x08,
+ 0x74, 0xb4, 0x01, 0x59, 0x9e, 0x82, 0xab, 0x34, 0x09, 0x35, 0x6c, 0x8b, 0xa5, 0xb3, 0x99, 0xca,
+ 0x1b, 0x2f, 0xfb, 0xc5, 0xc5, 0x53, 0x2c, 0x8b, 0x67, 0xef, 0x8f, 0xb8, 0xae, 0x92, 0xd9, 0x0f,
+ 0x17, 0x51, 0x07, 0x32, 0x81, 0x69, 0x1c, 0x3b, 0x22, 0xfb, 0xbd, 0x7a, 0xb0, 0x98, 0xf1, 0x6d,
+ 0x84, 0x62, 0xa2, 0x03, 0x98, 0xf5, 0x3b, 0xe1, 0x69, 0x27, 0xc9, 0xa7, 0xbe, 0x90, 0x6e, 0x7c,
+ 0xb3, 0xe6, 0xb3, 0x26, 0xe2, 0x20, 0x76, 0x1d, 0xe6, 0x4f, 0x8d, 0xc6, 0x7f, 0x36, 0x0d, 0xd7,
+ 0x47, 0x19, 0x3e, 0xc8, 0x30, 0xd4, 0xf1, 0x18, 0xf4, 0xe1, 0xc4, 0x51, 0xc2, 0xc7, 0xe0, 0x34,
+ 0xe4, 0x97, 0xc6, 0xa3, 0xd0, 0xa7, 0x63, 0x51, 0xe8, 0x15, 0xf0, 0x99, 0x79, 0x8d, 0xe1, 0x0b,
+ 0xd0, 0xc2, 0xbf, 0x44, 0x20, 0x33, 0xd2, 0xff, 0x2f, 0x93, 0x6e, 0x77, 0x82, 0xac, 0x80, 0x3f,
+ 0xf4, 0xbd, 0x7f, 0xf9, 0xb9, 0x8d, 0x26, 0x0b, 0x85, 0x7f, 0x8a, 0x40, 0x66, 0x64, 0xba, 0x5f,
+ 0x12, 0x51, 0x7f, 0xe1, 0x23, 0x2f, 0x7d, 0x13, 0x12, 0x5c, 0x82, 0x10, 0x64, 0x3f, 0x5a, 0x69,
+ 0xec, 0x36, 0xb6, 0xd6, 0xd4, 0xd5, 0x6d, 0x45, 0x5d, 0xab, 0xca, 0x53, 0x68, 0x06, 0xa4, 0x5a,
+ 0xfd, 0x41, 0x9d, 0x0a, 0xe5, 0x08, 0x4a, 0x43, 0x92, 0x95, 0xea, 0x35, 0x39, 0x5a, 0xaa, 0x80,
+ 0xcc, 0xb1, 0xf7, 0x31, 0xa5, 0x56, 0x7a, 0x90, 0xa0, 0x67, 0x36, 0x0a, 0x8b, 0xbb, 0x34, 0xc7,
+ 0xa0, 0xc1, 0x44, 0x0d, 0x65, 0x78, 0xb9, 0xa0, 0x8a, 0x86, 0x95, 0x2d, 0xad, 0x8b, 0x4b, 0xff,
+ 0x18, 0x87, 0xdc, 0x10, 0xc4, 0x0f, 0x2c, 0xf4, 0x80, 0x6a, 0x58, 0x47, 0xea, 0xf0, 0xdd, 0x89,
+ 0x1f, 0x50, 0x0d, 0xeb, 0x68, 0x4f, 0x69, 0x28, 0x49, 0x5a, 0xb9, 0xe7, 0x1a, 0xa8, 0x09, 0x71,
+ 0xdb, 0xf1, 0xfc, 0x04, 0xfd, 0xbd, 0x73, 0x96, 0xe2, 0x44, 0x1f, 0xe5, 0x6d, 0xc7, 0xe3, 0x97,
+ 0x2e, 0x0a, 0xc3, 0x40, 0x7f, 0x17, 0x81, 0x24, 0x3f, 0x0d, 0x91, 0x7c, 0x82, 0xe1, 0x7d, 0xeb,
+ 0x52, 0x78, 0x7c, 0x01, 0xc4, 0xcb, 0xc0, 0x47, 0xd4, 0xd4, 0x5f, 0xf6, 0x8b, 0xb9, 0xf1, 0x05,
+ 0x22, 0x57, 0x7c, 0x32, 0xf0, 0x87, 0x88, 0x9a, 0xfc, 0xb2, 0x7a, 0xb8, 0xd0, 0x8c, 0x52, 0x27,
+ 0x7c, 0x0f, 0xc8, 0x8c, 0x6c, 0x44, 0xe1, 0x00, 0x66, 0xc2, 0xa3, 0x3f, 0xe5, 0x16, 0x6a, 0x65,
+ 0xf4, 0x16, 0xea, 0xad, 0x89, 0x56, 0x86, 0x63, 0x86, 0xaf, 0xbe, 0xbe, 0x09, 0xa9, 0x60, 0xd9,
+ 0x2f, 0x73, 0x67, 0xc6, 0x59, 0x32, 0x38, 0x84, 0x4e, 0xcb, 0x89, 0xd2, 0xdf, 0x47, 0x60, 0x46,
+ 0xc1, 0xc4, 0x36, 0x9f, 0x60, 0x9d, 0x66, 0x0d, 0xe8, 0xeb, 0x10, 0xa7, 0x59, 0x88, 0x38, 0x5f,
+ 0x5f, 0x70, 0x3c, 0x61, 0xaa, 0x68, 0x05, 0x52, 0xc1, 0x2d, 0xc1, 0x65, 0x9e, 0x46, 0x87, 0xad,
+ 0xd0, 0x6d, 0x90, 0xdb, 0x76, 0xcf, 0xd2, 0x35, 0xf7, 0x58, 0x75, 0xb1, 0xd6, 0x39, 0xc4, 0xba,
+ 0xb8, 0x11, 0x9d, 0xf5, 0xe5, 0x0a, 0x17, 0x97, 0x7e, 0x18, 0x05, 0x34, 0x5c, 0x9c, 0x10, 0x4d,
+ 0xd0, 0xdc, 0x87, 0xcd, 0x43, 0x1c, 0xb0, 0xa2, 0xa7, 0xde, 0x61, 0x8f, 0xa5, 0x50, 0xc1, 0xc4,
+ 0xfd, 0x2d, 0x75, 0x43, 0x32, 0x82, 0xfe, 0xe4, 0xfc, 0xbb, 0x90, 0x18, 0xbb, 0x0b, 0x61, 0x56,
+ 0xfa, 0x4b, 0xbd, 0x0f, 0x11, 0xe1, 0xee, 0x7f, 0xe2, 0x80, 0xaa, 0x2e, 0xd6, 0x3c, 0x4c, 0x99,
+ 0x87, 0x9c, 0x77, 0x1e, 0xac, 0xc0, 0x34, 0x3f, 0x3c, 0x44, 0x2f, 0x73, 0x78, 0x10, 0x8b, 0xc2,
+ 0x9b, 0xa2, 0xdf, 0x80, 0x99, 0x8e, 0x6d, 0xf6, 0xba, 0x96, 0xca, 0x5e, 0x72, 0x44, 0x2a, 0xf9,
+ 0x8d, 0xf3, 0x8c, 0xf8, 0xc4, 0xe0, 0xca, 0x55, 0xdb, 0xa4, 0x65, 0xff, 0x7c, 0xcb, 0x01, 0x99,
+ 0x06, 0x7a, 0x0d, 0x52, 0x81, 0x43, 0xb1, 0x24, 0x32, 0xa5, 0x0c, 0x05, 0x68, 0x19, 0xa6, 0x35,
+ 0xa2, 0xda, 0xfb, 0x2c, 0xcb, 0xbb, 0xc8, 0xc2, 0x94, 0xb8, 0x46, 0xb6, 0xf7, 0xd1, 0x5d, 0xc8,
+ 0xec, 0x3f, 0xe6, 0x99, 0x2f, 0x27, 0x50, 0xfe, 0xc0, 0x36, 0x3b, 0xe8, 0x17, 0xd3, 0xab, 0x0f,
+ 0xd9, 0x64, 0x29, 0x7d, 0x2a, 0xe9, 0xfd, 0xc7, 0x41, 0x01, 0xdd, 0x81, 0x5c, 0x57, 0x7b, 0xa6,
+ 0xee, 0xbb, 0x5a, 0x47, 0xa4, 0x7a, 0x26, 0x67, 0x85, 0x88, 0x32, 0xdb, 0xd5, 0x9e, 0xad, 0x0a,
0x79, 0x43, 0x37, 0x71, 0xe1, 0x17, 0x11, 0x48, 0x8a, 0x19, 0x21, 0x07, 0x40, 0x2c, 0x8f, 0xa1,
- 0xf3, 0x60, 0x22, 0x53, 0xf9, 0x70, 0xd0, 0x2f, 0xa6, 0xaa, 0x4c, 0xda, 0xa8, 0x91, 0x97, 0xfd,
- 0xe2, 0x77, 0x5e, 0x95, 0xb4, 0x7c, 0x10, 0x25, 0xc5, 0x3b, 0x69, 0xe8, 0x2c, 0x5b, 0x73, 0xa8,
+ 0xf3, 0x64, 0x22, 0x53, 0x79, 0x38, 0xe8, 0x17, 0x53, 0x55, 0x26, 0x6d, 0xd4, 0xc8, 0xcb, 0x7e,
+ 0xf1, 0x3b, 0xaf, 0x4a, 0x5a, 0x3e, 0x88, 0x92, 0xe2, 0x9d, 0x34, 0x74, 0x76, 0x5b, 0x73, 0xa8,
0x11, 0xf5, 0xd0, 0x20, 0x9e, 0x7d, 0xe0, 0x6a, 0x5d, 0xb6, 0xb9, 0x92, 0x32, 0x73, 0xa8, 0x91,
- 0x75, 0x5f, 0x86, 0x0a, 0x34, 0x1c, 0x78, 0xc2, 0x1f, 0xe2, 0xf8, 0x91, 0x0a, 0xca, 0x68, 0x19,
- 0xae, 0x05, 0x8d, 0x55, 0x3a, 0xe9, 0x76, 0xaf, 0x73, 0x84, 0x99, 0x27, 0xa0, 0x9c, 0x35, 0x17,
- 0x54, 0x6e, 0x6a, 0xcf, 0x2a, 0xbc, 0xaa, 0x74, 0x0d, 0xe6, 0x42, 0xdb, 0x1a, 0x84, 0x58, 0x7f,
- 0x09, 0x90, 0xdc, 0xd1, 0x8e, 0x4d, 0x5b, 0xd3, 0xd1, 0x22, 0xa4, 0xfd, 0x07, 0x34, 0x1a, 0xa4,
- 0x72, 0x3b, 0x0c, 0x8b, 0xe8, 0xa0, 0x7a, 0x04, 0xbb, 0x6c, 0x4f, 0x38, 0x4b, 0x05, 0x65, 0xf1,
- 0x7e, 0x48, 0xc7, 0xa7, 0x76, 0x8d, 0x8e, 0x6b, 0xf3, 0x94, 0x43, 0x8c, 0xb1, 0x2d, 0x95, 0x6e,
- 0x32, 0x21, 0xba, 0x05, 0xb3, 0xfb, 0x86, 0xc5, 0x52, 0x9c, 0xbe, 0x1e, 0x4b, 0x46, 0x2b, 0x59,
- 0x5f, 0x2c, 0x14, 0x9f, 0x40, 0x36, 0xf4, 0x66, 0x48, 0xf7, 0x26, 0xc1, 0xf6, 0x66, 0x7b, 0xd0,
- 0x2f, 0x66, 0x86, 0xb6, 0xce, 0xf7, 0xe7, 0x2a, 0x4e, 0x25, 0x33, 0xec, 0x86, 0xee, 0xce, 0x3c,
- 0x4c, 0xb3, 0x9f, 0x99, 0xf1, 0x9f, 0x09, 0x28, 0xbc, 0x80, 0xde, 0x85, 0x69, 0x13, 0x6b, 0x04,
- 0x8b, 0x5f, 0x00, 0x2c, 0x9e, 0x73, 0x7a, 0xd8, 0x0f, 0xbe, 0x14, 0xae, 0x8e, 0x2a, 0x90, 0xe0,
- 0x49, 0x6b, 0x96, 0x6a, 0x4e, 0x2f, 0xdf, 0x9e, 0xf4, 0xc7, 0x1e, 0xeb, 0x53, 0x8a, 0x68, 0x89,
- 0xea, 0x90, 0x74, 0xf9, 0x3b, 0x05, 0x4b, 0x40, 0x5f, 0x78, 0xbf, 0x0c, 0x3d, 0x83, 0xac, 0x4f,
- 0x29, 0x7e, 0x5b, 0xb4, 0xeb, 0x3f, 0x16, 0x72, 0x1a, 0x16, 0x6f, 0xa1, 0xe5, 0x09, 0x23, 0xa8,
- 0x21, 0xe0, 0x08, 0x0a, 0x9d, 0xa0, 0xc1, 0x32, 0x97, 0x2c, 0x8f, 0x7d, 0xfe, 0x04, 0x47, 0x1e,
- 0x40, 0xe8, 0x04, 0x79, 0x4b, 0xb4, 0x05, 0xd0, 0x09, 0x5c, 0x03, 0xcb, 0x70, 0xa7, 0x97, 0xdf,
- 0xba, 0x4c, 0xf8, 0xb1, 0x3e, 0xa5, 0x84, 0x10, 0xd0, 0x87, 0x90, 0xee, 0x0c, 0x6d, 0x3d, 0x3f,
- 0xcb, 0x00, 0xdf, 0xbe, 0x14, 0xe1, 0xad, 0x53, 0x92, 0x1b, 0x4a, 0x47, 0x49, 0x4e, 0x1e, 0x27,
- 0xb9, 0x3a, 0x64, 0x44, 0x2a, 0x80, 0xff, 0x42, 0x31, 0x9f, 0x63, 0x1c, 0x1b, 0xb6, 0x12, 0xff,
- 0x37, 0x8c, 0xe5, 0xba, 0xd5, 0xb1, 0x75, 0xac, 0xd7, 0x69, 0x59, 0x11, 0xb9, 0x45, 0x56, 0x20,
- 0x68, 0x0d, 0xb2, 0x1d, 0x13, 0x6b, 0x56, 0xcf, 0xf1, 0x71, 0xd0, 0x84, 0x38, 0x19, 0xd1, 0x4e,
- 0x00, 0x6d, 0x01, 0xda, 0x67, 0x0f, 0x8c, 0xe1, 0x51, 0xb1, 0x44, 0xf9, 0x24, 0x60, 0x32, 0x6b,
- 0xab, 0x0c, 0x47, 0x86, 0xde, 0x80, 0x8c, 0x65, 0x5b, 0x1d, 0xcd, 0xea, 0x60, 0x93, 0xb9, 0x23,
- 0x9e, 0x5b, 0x1f, 0x15, 0xa2, 0x4f, 0x20, 0x4b, 0x46, 0x62, 0xee, 0xfc, 0x35, 0xd6, 0xe3, 0x3b,
- 0x97, 0x4d, 0xe0, 0xad, 0x4f, 0x29, 0x63, 0x48, 0xe8, 0x37, 0x41, 0xf6, 0xc6, 0x92, 0xb6, 0x2c,
- 0x4b, 0x7f, 0xfe, 0x63, 0xfe, 0x19, 0x59, 0xe6, 0xf5, 0x29, 0xe5, 0x04, 0x5a, 0x25, 0x05, 0x49,
- 0x9d, 0x57, 0x07, 0xc9, 0xfd, 0xa4, 0x2c, 0x95, 0x7e, 0x91, 0x00, 0x29, 0x88, 0x56, 0x96, 0x00,
- 0x05, 0xfe, 0x65, 0xf8, 0x8b, 0x0a, 0x4a, 0x94, 0xd1, 0xf5, 0x29, 0x25, 0xe7, 0xd7, 0x0d, 0x7f,
- 0x54, 0x71, 0x0b, 0x66, 0xbb, 0xb6, 0x6e, 0xec, 0x1b, 0x43, 0xb6, 0xe3, 0xe9, 0xa5, 0xac, 0x2f,
- 0x16, 0x6c, 0xf7, 0x60, 0xe4, 0xe1, 0x31, 0x36, 0x81, 0xaf, 0x5c, 0x9f, 0x0a, 0xbd, 0x4c, 0x52,
- 0xf6, 0x75, 0x7b, 0x96, 0x65, 0x58, 0x07, 0xaa, 0xb8, 0x20, 0x71, 0x4f, 0x9c, 0x11, 0x52, 0x71,
- 0xc7, 0xa9, 0x8e, 0xd1, 0xd1, 0x9d, 0x0b, 0xe9, 0xc8, 0x9f, 0xfb, 0x7a, 0x24, 0xe0, 0xa3, 0xd5,
- 0x71, 0x3e, 0xba, 0x7b, 0x31, 0x1f, 0x85, 0x60, 0x02, 0x42, 0xda, 0x3b, 0x95, 0x90, 0x96, 0x26,
- 0xb4, 0x96, 0x10, 0xe2, 0x28, 0x23, 0x55, 0xc7, 0x18, 0xe9, 0xce, 0x85, 0x8c, 0x14, 0x9e, 0xa3,
- 0xa0, 0xa4, 0xed, 0x53, 0x28, 0xe9, 0xed, 0x89, 0x28, 0x29, 0x04, 0x16, 0xe6, 0x24, 0xe5, 0x34,
- 0x4e, 0x2a, 0x4f, 0xc6, 0x49, 0x21, 0xc8, 0x11, 0x52, 0xfa, 0xde, 0x89, 0x03, 0x27, 0x33, 0xd8,
- 0xaf, 0x5f, 0xfa, 0x56, 0xbc, 0x1e, 0x39, 0x71, 0xe2, 0xb4, 0x53, 0x4e, 0x5c, 0x8e, 0xc1, 0xdf,
- 0xbb, 0xc4, 0x89, 0x0b, 0x75, 0x70, 0xf2, 0xc8, 0x01, 0x48, 0xfe, 0x2b, 0x53, 0xe8, 0xf8, 0x95,
- 0x7e, 0x14, 0x81, 0x58, 0xd3, 0x6e, 0xa3, 0xec, 0x30, 0x85, 0xc0, 0x2e, 0xff, 0x1f, 0x0c, 0xd5,
- 0x45, 0x3c, 0xfc, 0xfa, 0x39, 0x23, 0xf1, 0x7b, 0x56, 0x82, 0x46, 0xe8, 0xdb, 0x90, 0x74, 0x78,
- 0xac, 0x23, 0x4e, 0x58, 0xe9, 0xbc, 0xf6, 0x5c, 0x53, 0xf1, 0x9b, 0xdc, 0xbd, 0x13, 0xfe, 0x85,
- 0xf0, 0xa6, 0xad, 0x63, 0x94, 0x05, 0xd8, 0xd1, 0x08, 0x71, 0x0e, 0x5d, 0x8d, 0x60, 0x79, 0x0a,
- 0x25, 0x21, 0xb6, 0xb1, 0xd9, 0x92, 0x23, 0x77, 0xbf, 0x1b, 0x4e, 0x2a, 0xd4, 0x94, 0x95, 0xc6,
- 0x56, 0x63, 0x6b, 0x4d, 0xdd, 0x5a, 0xd9, 0xac, 0xb7, 0xe4, 0x29, 0x94, 0x87, 0xf9, 0x8f, 0x56,
- 0x1a, 0xbb, 0x22, 0xcb, 0xa0, 0x36, 0xb6, 0x76, 0xeb, 0xca, 0xa3, 0x95, 0x87, 0x72, 0x04, 0x5d,
- 0x07, 0xa4, 0x6c, 0x57, 0x37, 0x5a, 0xb5, 0x8a, 0x5a, 0xdd, 0xde, 0xdc, 0x59, 0xa9, 0xee, 0x36,
- 0xb6, 0xb7, 0xe4, 0x28, 0x92, 0x20, 0x5e, 0xdb, 0xde, 0xaa, 0xcb, 0x70, 0xf7, 0x47, 0x31, 0x88,
- 0xd3, 0x35, 0x46, 0x6f, 0x40, 0x7a, 0x6f, 0xab, 0xb5, 0x53, 0xaf, 0x36, 0x56, 0x1b, 0xf5, 0x9a,
- 0x3c, 0x55, 0x98, 0x7b, 0xfe, 0x62, 0x71, 0x96, 0x56, 0xed, 0x59, 0xc4, 0xc1, 0x1d, 0x46, 0x2e,
- 0xa8, 0x00, 0x89, 0xca, 0x4a, 0x75, 0x63, 0x6f, 0x47, 0x8e, 0x14, 0xb2, 0xcf, 0x5f, 0x2c, 0x02,
- 0x55, 0xe0, 0x07, 0x1b, 0xbd, 0x06, 0x49, 0xa5, 0xde, 0xda, 0xdd, 0x56, 0xea, 0x72, 0xb4, 0x30,
- 0xfb, 0xfc, 0xc5, 0x62, 0x9a, 0x56, 0x8a, 0xf3, 0x8a, 0x6e, 0x41, 0xa6, 0x55, 0x5d, 0xaf, 0x6f,
- 0xae, 0xa8, 0xd5, 0xf5, 0x95, 0xad, 0xb5, 0xba, 0x1c, 0x2b, 0xcc, 0x3f, 0x7f, 0xb1, 0x28, 0x8f,
- 0x6f, 0x30, 0xed, 0xa2, 0xb1, 0xb9, 0xb3, 0xad, 0xec, 0xca, 0xf1, 0x61, 0x17, 0xfc, 0x5c, 0xa1,
- 0x12, 0x00, 0x6f, 0xbd, 0x5a, 0xaf, 0xd7, 0xe4, 0xe9, 0x02, 0x7a, 0xfe, 0x62, 0x31, 0x4b, 0xeb,
- 0x87, 0xc7, 0x05, 0xdd, 0x84, 0x99, 0xaa, 0x52, 0x5f, 0xd9, 0xad, 0xab, 0xad, 0xdd, 0x95, 0xdd,
- 0x96, 0x9c, 0x18, 0xce, 0x24, 0x74, 0x04, 0x50, 0x19, 0x72, 0x2b, 0x7b, 0xbb, 0xdb, 0xea, 0x88,
- 0x6e, 0xb2, 0x70, 0xe3, 0xf9, 0x8b, 0xc5, 0x39, 0xaa, 0xbb, 0xd2, 0xf3, 0xec, 0xb0, 0xfe, 0x5b,
- 0x20, 0x8f, 0x8c, 0x5f, 0x5d, 0xab, 0xca, 0x52, 0xe1, 0xfa, 0xf3, 0x17, 0x8b, 0x68, 0x7c, 0x0a,
- 0x6b, 0x55, 0xf4, 0x6b, 0x70, 0x7d, 0xf7, 0xe3, 0x9d, 0x7a, 0xad, 0xde, 0xaa, 0xaa, 0xa3, 0xd3,
- 0x4e, 0x15, 0xf2, 0xcf, 0x5f, 0x2c, 0xce, 0xd3, 0x36, 0xe3, 0xed, 0x0a, 0xd2, 0x0f, 0xff, 0x7a,
- 0x61, 0xea, 0xef, 0xfe, 0x66, 0x61, 0xaa, 0x72, 0xfb, 0xb3, 0xff, 0x5c, 0x98, 0xfa, 0x6c, 0xb0,
- 0x10, 0xf9, 0xe9, 0x60, 0x21, 0xf2, 0xb3, 0xc1, 0x42, 0xe4, 0x3f, 0x06, 0x0b, 0x91, 0x3f, 0xfa,
- 0x7c, 0x61, 0xea, 0xa7, 0x9f, 0x2f, 0x4c, 0xfd, 0xec, 0xf3, 0x85, 0xa9, 0x4f, 0x12, 0xdc, 0xac,
- 0xda, 0x09, 0xf6, 0x1f, 0x03, 0xee, 0xfd, 0x5f, 0x00, 0x00, 0x00, 0xff, 0xff, 0xe4, 0x80, 0xaf,
- 0xb2, 0x08, 0x31, 0x00, 0x00,
+ 0x75, 0x5f, 0x86, 0x0a, 0x34, 0x1d, 0x78, 0xc2, 0x1f, 0xe2, 0xb8, 0x4b, 0x05, 0x65, 0xb4, 0x0c,
+ 0xd7, 0x82, 0xc6, 0x2a, 0x9d, 0x74, 0xbb, 0xd7, 0x39, 0xc2, 0x2c, 0x12, 0x50, 0xce, 0x9a, 0x0b,
+ 0x2a, 0x37, 0xb5, 0x67, 0x15, 0x5e, 0x55, 0xba, 0x06, 0x73, 0xa1, 0x6d, 0x0d, 0x52, 0xac, 0xff,
+ 0x06, 0x48, 0xee, 0x68, 0xc7, 0xa6, 0xad, 0xe9, 0x68, 0x11, 0xd2, 0xfe, 0x03, 0x1a, 0x4d, 0x52,
+ 0xb9, 0x1d, 0x86, 0x45, 0xc8, 0x80, 0x6c, 0x8f, 0x60, 0x97, 0xee, 0x89, 0xca, 0x3e, 0x70, 0xe2,
+ 0x5c, 0x55, 0xa9, 0xbc, 0xec, 0x17, 0xef, 0x4f, 0xb6, 0x44, 0xb8, 0xd3, 0x73, 0x0d, 0xef, 0xb8,
+ 0xdc, 0x7a, 0xf8, 0x60, 0x4f, 0x40, 0x51, 0x47, 0xb2, 0x95, 0x4c, 0x2f, 0x5c, 0x14, 0xcf, 0x91,
+ 0x74, 0xba, 0x6a, 0xd7, 0xe8, 0xb8, 0x36, 0xbf, 0xc1, 0x88, 0x31, 0xf2, 0xa6, 0xd2, 0x4d, 0x26,
+ 0x44, 0x37, 0x61, 0x76, 0xdf, 0xb0, 0xd8, 0x8d, 0xa9, 0xaf, 0xc7, 0xee, 0xb6, 0x95, 0xac, 0x2f,
+ 0x16, 0x8a, 0x4f, 0x20, 0x1b, 0x7a, 0x82, 0xa4, 0x5b, 0x9d, 0x60, 0x5b, 0xbd, 0x3d, 0xe8, 0x17,
+ 0x33, 0x43, 0xd7, 0xe1, 0xdb, 0x7d, 0x95, 0x18, 0x95, 0x19, 0x76, 0x43, 0x37, 0x7b, 0x1e, 0xa6,
+ 0xd9, 0x57, 0x6b, 0xfc, 0xab, 0x03, 0x85, 0x17, 0xd0, 0x7b, 0x30, 0x6d, 0x62, 0x8d, 0x60, 0xf1,
+ 0x41, 0xc1, 0xe2, 0x39, 0xce, 0xc8, 0xbe, 0x1f, 0x53, 0xb8, 0x3a, 0xaa, 0x40, 0x82, 0xdf, 0x81,
+ 0xb3, 0x9b, 0xeb, 0xf4, 0xf2, 0xad, 0x49, 0xbf, 0x1d, 0x59, 0x9f, 0x52, 0x44, 0x4b, 0x54, 0x87,
+ 0xa4, 0xcb, 0x9f, 0x3d, 0xd8, 0x7d, 0xf6, 0x85, 0xc7, 0xd5, 0xd0, 0xab, 0xca, 0xfa, 0x94, 0xe2,
+ 0xb7, 0x45, 0xbb, 0xfe, 0xdb, 0x23, 0x67, 0x75, 0xf1, 0xb4, 0x5a, 0x9e, 0x30, 0x21, 0x1b, 0x02,
+ 0x8e, 0xa0, 0xd0, 0x09, 0x1a, 0xec, 0x22, 0x94, 0x5d, 0x8b, 0x9f, 0x3f, 0xc1, 0x91, 0xf7, 0x14,
+ 0x3a, 0x41, 0xde, 0x12, 0x6d, 0x01, 0x74, 0x82, 0x48, 0xc3, 0x2e, 0xcc, 0xd3, 0xcb, 0x6f, 0x5f,
+ 0x26, 0x9b, 0x59, 0x9f, 0x52, 0x42, 0x08, 0xe8, 0x21, 0xa4, 0x3b, 0x43, 0xd7, 0xc9, 0xcf, 0x32,
+ 0xc0, 0x77, 0x2e, 0xc5, 0x9f, 0xeb, 0x94, 0x33, 0x87, 0xd2, 0x51, 0xce, 0x94, 0xc7, 0x39, 0xb3,
+ 0x0e, 0x19, 0x71, 0xb3, 0xc0, 0x3f, 0x78, 0xcc, 0xe7, 0x18, 0x65, 0x87, 0xad, 0xc4, 0xff, 0x24,
+ 0xb2, 0x5c, 0xb7, 0x3a, 0xb6, 0x8e, 0xf5, 0x3a, 0x2d, 0x2b, 0xe2, 0xaa, 0x92, 0x15, 0x08, 0x5a,
+ 0x83, 0x6c, 0xc7, 0xc4, 0x9a, 0xd5, 0x73, 0x7c, 0x1c, 0x34, 0x21, 0x4e, 0x46, 0xb4, 0x13, 0x40,
+ 0x5b, 0x80, 0xf6, 0xd9, 0x7b, 0x65, 0x78, 0x54, 0xec, 0xde, 0x7d, 0x12, 0x30, 0x99, 0xb5, 0x55,
+ 0x86, 0x23, 0x43, 0x6f, 0x40, 0xc6, 0xb2, 0xad, 0x8e, 0x66, 0x75, 0xb0, 0xc9, 0xa2, 0x1b, 0xbf,
+ 0xaa, 0x1f, 0x15, 0xa2, 0x4f, 0x20, 0x4b, 0x46, 0x52, 0xf8, 0xfc, 0x35, 0xd6, 0xe3, 0xbb, 0x97,
+ 0xbd, 0x0f, 0x5c, 0x9f, 0x52, 0xc6, 0x90, 0xd0, 0x6f, 0x82, 0xec, 0x8d, 0xdd, 0x01, 0xb3, 0x4b,
+ 0xff, 0xf3, 0xbf, 0x0d, 0x38, 0xe3, 0xd2, 0x7a, 0x7d, 0x4a, 0x39, 0x81, 0x56, 0x49, 0x41, 0x52,
+ 0xe7, 0xd5, 0xc1, 0x5b, 0x41, 0x52, 0x96, 0x4a, 0xbf, 0x48, 0x80, 0x14, 0x24, 0x3f, 0x4b, 0x80,
+ 0x82, 0x70, 0x35, 0xfc, 0x40, 0x83, 0xf2, 0x6e, 0x74, 0x7d, 0x4a, 0xc9, 0xf9, 0x75, 0xc3, 0x6f,
+ 0x34, 0x6e, 0xc2, 0x6c, 0xd7, 0xd6, 0x8d, 0x7d, 0x63, 0xc8, 0x76, 0xfc, 0xb6, 0x2a, 0xeb, 0x8b,
+ 0x05, 0xdb, 0xdd, 0x1f, 0x79, 0xc7, 0x8c, 0x4d, 0x10, 0x7a, 0xd7, 0xa7, 0x42, 0x0f, 0x9d, 0x94,
+ 0x7d, 0xdd, 0x9e, 0x65, 0x19, 0xd6, 0x81, 0x2a, 0xce, 0x5b, 0x3c, 0xb0, 0x67, 0x84, 0x54, 0x1c,
+ 0x99, 0xaa, 0x63, 0x74, 0x74, 0xfb, 0x42, 0x3a, 0xf2, 0xe7, 0xbe, 0x1e, 0x09, 0xf8, 0x68, 0x75,
+ 0x9c, 0x8f, 0xee, 0x5c, 0xcc, 0x47, 0x21, 0x98, 0x80, 0x90, 0xf6, 0x4e, 0x25, 0xa4, 0xa5, 0x09,
+ 0xad, 0x25, 0x84, 0x38, 0xca, 0x48, 0xd5, 0x31, 0x46, 0xba, 0x7d, 0x21, 0x23, 0x85, 0xe7, 0x28,
+ 0x28, 0x69, 0xfb, 0x14, 0x4a, 0x7a, 0x67, 0x22, 0x4a, 0x0a, 0x81, 0x85, 0x39, 0x49, 0x39, 0x8d,
+ 0x93, 0xca, 0x93, 0x71, 0x52, 0x08, 0x72, 0x84, 0x94, 0xbe, 0x77, 0xc2, 0xe1, 0x64, 0x06, 0xfb,
+ 0xf5, 0x4b, 0x1f, 0xb2, 0xd7, 0x23, 0x27, 0x3c, 0x4e, 0x3b, 0xc5, 0xe3, 0x72, 0x0c, 0xfe, 0xee,
+ 0x25, 0x3c, 0x2e, 0xd4, 0xc1, 0x49, 0x97, 0x03, 0x90, 0xfc, 0x47, 0xab, 0x90, 0xfb, 0x95, 0x7e,
+ 0x14, 0x81, 0x58, 0xd3, 0x6e, 0xa3, 0xec, 0xf0, 0x46, 0x82, 0xdd, 0x25, 0x7c, 0x38, 0x54, 0x17,
+ 0xe9, 0xf5, 0xeb, 0xe7, 0x8c, 0xc4, 0xef, 0x59, 0x09, 0x1a, 0xa1, 0x6f, 0x43, 0xd2, 0xe1, 0xa9,
+ 0x93, 0xf0, 0xb0, 0xd2, 0x79, 0xed, 0xb9, 0xa6, 0xe2, 0x37, 0xb9, 0x73, 0x3b, 0xfc, 0xc1, 0xf1,
+ 0xa6, 0xad, 0x63, 0x94, 0x05, 0xd8, 0xd1, 0x08, 0x71, 0x0e, 0x5d, 0x8d, 0x60, 0x79, 0x0a, 0x25,
+ 0x21, 0xb6, 0xb1, 0xd9, 0x92, 0x23, 0x77, 0xbe, 0x1b, 0xbe, 0xa3, 0xa8, 0x29, 0x2b, 0x8d, 0xad,
+ 0xc6, 0xd6, 0x9a, 0xba, 0xb5, 0xb2, 0x59, 0x6f, 0xc9, 0x53, 0x28, 0x0f, 0xf3, 0x1f, 0xad, 0x34,
+ 0x76, 0xc5, 0xa5, 0x85, 0xda, 0xd8, 0xda, 0xad, 0x2b, 0x8f, 0x56, 0x1e, 0xc8, 0x11, 0x74, 0x1d,
+ 0x90, 0xb2, 0x5d, 0xdd, 0x68, 0xd5, 0x2a, 0x6a, 0x75, 0x7b, 0x73, 0x67, 0xa5, 0xba, 0xdb, 0xd8,
+ 0xde, 0x92, 0xa3, 0x48, 0x82, 0x78, 0x6d, 0x7b, 0xab, 0x2e, 0xc3, 0x9d, 0x1f, 0xc5, 0x20, 0x4e,
+ 0xd7, 0x18, 0xbd, 0x01, 0xe9, 0xbd, 0xad, 0xd6, 0x4e, 0xbd, 0xda, 0x58, 0x6d, 0xd4, 0x6b, 0xf2,
+ 0x54, 0x61, 0xee, 0xf9, 0x8b, 0xc5, 0x59, 0x5a, 0xb5, 0x67, 0x11, 0x07, 0x77, 0x18, 0xb9, 0xa0,
+ 0x02, 0x24, 0x2a, 0x2b, 0xd5, 0x8d, 0xbd, 0x1d, 0x39, 0x52, 0xc8, 0x3e, 0x7f, 0xb1, 0x08, 0x54,
+ 0x81, 0x3b, 0x36, 0x7a, 0x0d, 0x92, 0x4a, 0xbd, 0xb5, 0xbb, 0xad, 0xd4, 0xe5, 0x68, 0x61, 0xf6,
+ 0xf9, 0x8b, 0xc5, 0x34, 0xad, 0x14, 0xfe, 0x8a, 0x6e, 0x42, 0xa6, 0x55, 0x5d, 0xaf, 0x6f, 0xae,
+ 0xa8, 0xd5, 0xf5, 0x95, 0xad, 0xb5, 0xba, 0x1c, 0x2b, 0xcc, 0x3f, 0x7f, 0xb1, 0x28, 0x8f, 0x6f,
+ 0x30, 0xed, 0xa2, 0xb1, 0xb9, 0xb3, 0xad, 0xec, 0xca, 0xf1, 0x61, 0x17, 0xdc, 0xaf, 0x50, 0x09,
+ 0x80, 0xb7, 0x5e, 0xad, 0xd7, 0x6b, 0xf2, 0x74, 0x01, 0x3d, 0x7f, 0xb1, 0x98, 0xa5, 0xf5, 0x43,
+ 0x77, 0x41, 0x6f, 0xc2, 0x4c, 0x55, 0xa9, 0xaf, 0xec, 0xd6, 0xd5, 0xd6, 0xee, 0xca, 0x6e, 0x4b,
+ 0x4e, 0x0c, 0x67, 0x12, 0x72, 0x01, 0x54, 0x86, 0xdc, 0xca, 0xde, 0xee, 0xb6, 0x3a, 0xa2, 0x9b,
+ 0x2c, 0xdc, 0x78, 0xfe, 0x62, 0x71, 0x8e, 0xea, 0xae, 0xf4, 0x3c, 0x3b, 0xac, 0xff, 0x36, 0xc8,
+ 0x23, 0xe3, 0x57, 0xd7, 0xaa, 0xb2, 0x54, 0xb8, 0xfe, 0xfc, 0xc5, 0x22, 0x1a, 0x9f, 0xc2, 0x5a,
+ 0x15, 0xfd, 0x1a, 0x5c, 0xdf, 0xfd, 0x78, 0xa7, 0x5e, 0xab, 0xb7, 0xaa, 0xea, 0xe8, 0xb4, 0x53,
+ 0x85, 0xfc, 0xf3, 0x17, 0x8b, 0xf3, 0xb4, 0xcd, 0x78, 0xbb, 0x82, 0xf4, 0xc3, 0xbf, 0x5a, 0x98,
+ 0xfa, 0xdb, 0xbf, 0x5e, 0x98, 0xaa, 0xdc, 0xfa, 0xec, 0x3f, 0x16, 0xa6, 0x3e, 0x1b, 0x2c, 0x44,
+ 0x7e, 0x3a, 0x58, 0x88, 0xfc, 0x6c, 0xb0, 0x10, 0xf9, 0xf7, 0xc1, 0x42, 0xe4, 0x8f, 0x3e, 0x5f,
+ 0x98, 0xfa, 0xe9, 0xe7, 0x0b, 0x53, 0x3f, 0xfb, 0x7c, 0x61, 0xea, 0x93, 0x04, 0x37, 0xab, 0x76,
+ 0x82, 0xa5, 0xe1, 0x77, 0xff, 0x2f, 0x00, 0x00, 0xff, 0xff, 0xc6, 0x85, 0x9f, 0x4e, 0x57, 0x31,
+ 0x00, 0x00,
}
diff --git a/pkg/jobs/jobspb/jobs.proto b/pkg/jobs/jobspb/jobs.proto
index a0329c3cce8a..aabf6f0f4ff3 100644
--- a/pkg/jobs/jobspb/jobs.proto
+++ b/pkg/jobs/jobspb/jobs.proto
@@ -493,7 +493,7 @@ message Payload {
string description = 1;
// If empty, the description is assumed to be the statement.
string statement = 16;
- string username = 2;
+ string username_proto = 2 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"];
// For consistency with the SQL timestamp type, which has microsecond
// precision, we avoid the timestamp.Timestamp WKT, which has nanosecond
// precision, and use microsecond integers directly.
diff --git a/pkg/jobs/jobsprotectedts/jobs_protected_ts_test.go b/pkg/jobs/jobsprotectedts/jobs_protected_ts_test.go
index 31fd69003a7d..20e56b37fa8e 100644
--- a/pkg/jobs/jobsprotectedts/jobs_protected_ts_test.go
+++ b/pkg/jobs/jobsprotectedts/jobs_protected_ts_test.go
@@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts/ptpb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/testutils"
@@ -56,7 +57,7 @@ func TestJobsProtectedTimestamp(t *testing.T) {
return jobs.Record{
Description: "testing",
Statement: "SELECT 1",
- Username: "root",
+ Username: security.RootUserName(),
Details: jobspb.SchemaChangeGCDetails{
Tables: []jobspb.SchemaChangeGCDetails_DroppedID{
{
diff --git a/pkg/jobs/registry.go b/pkg/jobs/registry.go
index efc821e90e94..1f3843979a9b 100644
--- a/pkg/jobs/registry.go
+++ b/pkg/jobs/registry.go
@@ -179,7 +179,7 @@ type Registry struct {
// subpackage like sqlbase is difficult because of the amount of sql-only
// stuff that PlanHookState exports. One other choice is to merge this package
// back into the sql package. There's maybe a better way that I'm unaware of.
-type planHookMaker func(opName, user string) (interface{}, func())
+type planHookMaker func(opName string, user security.SQLUsername) (interface{}, func())
// PreventAdoptionFile is the name of the file which, if present in the first
// on-disk store, will prevent the adoption of background jobs by that node.
@@ -348,7 +348,7 @@ func (r *Registry) Run(ctx context.Context, ex sqlutil.InternalExecutor, jobs []
ctx,
"poll-show-jobs",
nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
query,
)
if err != nil {
@@ -389,7 +389,7 @@ func (r *Registry) NewJob(record Record) *Job {
job.mu.payload = jobspb.Payload{
Description: record.Description,
Statement: record.Statement,
- Username: record.Username,
+ UsernameProto: record.Username.EncodeProto(),
DescriptorIDs: record.DescriptorIDs,
Details: jobspb.WrapPayloadDetails(record.Details),
Noncancelable: record.NonCancelable,
@@ -620,7 +620,7 @@ func (r *Registry) Start(
removeClaimsFromDeadSessions := func(ctx context.Context, s sqlliveness.Session) {
if _, err := r.ex.QueryRowEx(
ctx, "expire-sessions", nil,
- sessiondata.InternalExecutorOverride{User: security.RootUser}, `
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()}, `
UPDATE system.jobs
SET claim_session_id = NULL
WHERE claim_session_id <> $1
diff --git a/pkg/jobs/registry_external_test.go b/pkg/jobs/registry_external_test.go
index b1408cc5856d..e7435991de62 100644
--- a/pkg/jobs/registry_external_test.go
+++ b/pkg/jobs/registry_external_test.go
@@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness"
@@ -51,7 +52,7 @@ func TestRoundtripJob(t *testing.T) {
storedJob := registry.NewJob(jobs.Record{
Description: "beep boop",
- Username: "robot",
+ Username: security.MakeSQLUsernameFromPreNormalizedString("robot"),
DescriptorIDs: descpb.IDs{42},
Details: jobspb.RestoreDetails{},
Progress: jobspb.RestoreProgress{},
diff --git a/pkg/jobs/registry_test.go b/pkg/jobs/registry_test.go
index 749ec9ea518a..d9de130fe758 100644
--- a/pkg/jobs/registry_test.go
+++ b/pkg/jobs/registry_test.go
@@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
@@ -40,7 +41,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
)
-func FakePHS(opName, user string) (interface{}, func()) {
+func FakePHS(opName string, user security.SQLUsername) (interface{}, func()) {
return nil, func() {}
}
diff --git a/pkg/jobs/schedule_control_test.go b/pkg/jobs/schedule_control_test.go
index d525bc53cc61..c7e487e247cb 100644
--- a/pkg/jobs/schedule_control_test.go
+++ b/pkg/jobs/schedule_control_test.go
@@ -142,7 +142,7 @@ func TestJobsControlForSchedules(t *testing.T) {
record := Record{
Description: "fake job",
- Username: "test",
+ Username: security.TestUserName(),
Details: jobspb.ImportDetails{},
Progress: jobspb.ImportProgress{},
}
@@ -208,7 +208,7 @@ func TestJobsControlForSchedules(t *testing.T) {
context.Background(),
"test-num-effected",
nil,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
jobControl,
)
require.NoError(t, err)
@@ -246,7 +246,7 @@ func TestFilterJobsControlForSchedules(t *testing.T) {
record := Record{
Description: "fake job",
- Username: "test",
+ Username: security.TestUserName(),
Details: jobspb.ImportDetails{},
Progress: jobspb.ImportProgress{},
}
@@ -285,7 +285,7 @@ func TestFilterJobsControlForSchedules(t *testing.T) {
context.Background(),
"test-num-effected",
nil,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
jobControl,
)
require.NoError(t, err)
diff --git a/pkg/jobs/scheduled_job.go b/pkg/jobs/scheduled_job.go
index bd1c72bb04e7..76c431197290 100644
--- a/pkg/jobs/scheduled_job.go
+++ b/pkg/jobs/scheduled_job.go
@@ -38,7 +38,7 @@ import (
type scheduledJobRecord struct {
ScheduleID int64 `col:"schedule_id"`
ScheduleLabel string `col:"schedule_name"`
- Owner string `col:"owner"`
+ Owner security.SQLUsername `col:"owner"`
NextRun time.Time `col:"next_run"`
ScheduleState jobspb.ScheduleState `col:"schedule_state"`
ScheduleExpr string `col:"schedule_expr"`
@@ -85,7 +85,7 @@ func LoadScheduledJob(
txn *kv.Txn,
) (*ScheduledJob, error) {
rows, cols, err := ex.QueryWithCols(ctx, "lookup-schedule", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
fmt.Sprintf("SELECT * FROM %s WHERE schedule_id = %d",
env.ScheduledJobsTableName(), id))
@@ -123,12 +123,12 @@ func (j *ScheduledJob) SetScheduleLabel(label string) {
}
// Owner returns schedule owner.
-func (j *ScheduledJob) Owner() string {
+func (j *ScheduledJob) Owner() security.SQLUsername {
return j.rec.Owner
}
// SetOwner updates schedule owner.
-func (j *ScheduledJob) SetOwner(owner string) {
+func (j *ScheduledJob) SetOwner(owner security.SQLUsername) {
j.rec.Owner = owner
j.markDirty("owner")
}
@@ -311,8 +311,25 @@ func (j *ScheduledJob) InitFromDatums(datums []tree.Datum, cols []colinfo.Result
// But, be paranoid and double check.
rv := reflect.ValueOf(native)
if !rv.Type().AssignableTo(field.Type()) {
- return errors.Newf("value of type %T cannot be assigned to %s",
- native, field.Type().String())
+ // Is this the owner field? This needs special treatment.
+ ok := false
+ if col.Name == "owner" {
+ // The owner field has type SQLUsername, but the datum is a
+ // simple string. So we need to convert.
+ //
+ // TODO(someone): We need a more generic mechanism than this
+ // naive go reflect stuff here.
+ var s string
+ s, ok = native.(string)
+ if ok {
+ // Replace the value by one of the right type.
+ rv = reflect.ValueOf(security.MakeSQLUsernameFromPreNormalizedString(s))
+ }
+ }
+ if !ok {
+ return errors.Newf("value of type %T cannot be assigned to %s",
+ native, field.Type().String())
+ }
}
field.Set(rv)
}
@@ -346,7 +363,7 @@ func (j *ScheduledJob) Create(ctx context.Context, ex sqlutil.InternalExecutor,
}
rows, retCols, err := ex.QueryWithCols(ctx, "sched-create", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
fmt.Sprintf("INSERT INTO %s (%s) VALUES(%s) RETURNING schedule_id",
j.env.ScheduledJobsTableName(), strings.Join(cols, ","), generatePlaceholders(len(qargs))),
qargs...,
@@ -384,7 +401,7 @@ func (j *ScheduledJob) Update(ctx context.Context, ex sqlutil.InternalExecutor,
}
n, err := ex.ExecEx(ctx, "sched-update", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
fmt.Sprintf("UPDATE %s SET (%s) = (%s) WHERE schedule_id = %d",
j.env.ScheduledJobsTableName(), strings.Join(cols, ","),
generatePlaceholders(len(qargs)), j.ScheduleID()),
@@ -418,7 +435,7 @@ func (j *ScheduledJob) marshalChanges() ([]string, []interface{}, error) {
case `schedule_name`:
arg = tree.NewDString(j.rec.ScheduleLabel)
case `owner`:
- arg = tree.NewDString(j.rec.Owner)
+ arg = tree.NewDString(j.rec.Owner.Normalized())
case `next_run`:
if (j.rec.NextRun == time.Time{}) {
arg = tree.DNull
diff --git a/pkg/jobs/testutils_test.go b/pkg/jobs/testutils_test.go
index e973aef34615..5d74b80345a3 100644
--- a/pkg/jobs/testutils_test.go
+++ b/pkg/jobs/testutils_test.go
@@ -105,7 +105,7 @@ func newTestHelperForTables(
func (h *testHelper) newScheduledJob(t *testing.T, scheduleLabel, sql string) *ScheduledJob {
j := NewScheduledJob(h.env)
j.SetScheduleLabel(scheduleLabel)
- j.SetOwner("test")
+ j.SetOwner(security.TestUserName())
any, err := types.MarshalAny(&jobspb.SqlStatementExecutionArg{Statement: sql})
require.NoError(t, err)
j.SetExecutionDetails(InlineExecutorName, jobspb.ExecutionArguments{Args: any})
@@ -119,7 +119,7 @@ func (h *testHelper) newScheduledJobForExecutor(
) *ScheduledJob {
j := NewScheduledJob(h.env)
j.SetScheduleLabel(scheduleLabel)
- j.SetOwner("test")
+ j.SetOwner(security.TestUserName())
j.SetExecutionDetails(executorName, jobspb.ExecutionArguments{Args: executorArgs})
return j
}
@@ -129,7 +129,7 @@ func (h *testHelper) loadSchedule(t *testing.T, id int64) *ScheduledJob {
j := NewScheduledJob(h.env)
rows, cols, err := h.cfg.InternalExecutor.QueryWithCols(
context.Background(), "sched-load", nil,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
fmt.Sprintf(
"SELECT * FROM %s WHERE schedule_id = %d",
h.env.ScheduledJobsTableName(), id),
@@ -159,7 +159,7 @@ func registerScopedScheduledJobExecutor(name string, ex ScheduledJobExecutor) fu
func addFakeJob(t *testing.T, h *testHelper, scheduleID int64, status Status, txn *kv.Txn) int64 {
payload := []byte("fake payload")
datums, err := h.cfg.InternalExecutor.QueryRowEx(context.Background(), "fake-job", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
fmt.Sprintf(`
INSERT INTO %s (created_by_type, created_by_id, status, payload)
VALUES ($1, $2, $3, $4)
diff --git a/pkg/jobs/update.go b/pkg/jobs/update.go
index 558fd215a1b8..c1ee9d05f7a0 100644
--- a/pkg/jobs/update.go
+++ b/pkg/jobs/update.go
@@ -113,7 +113,7 @@ func (j *Job) Update(ctx context.Context, updateFn UpdateFn) error {
var err error
var row tree.Datums
row, err = j.registry.ex.QueryRowEx(
- ctx, "log-job", txn, sessiondata.InternalExecutorOverride{User: security.RootUser},
+ ctx, "log-job", txn, sessiondata.InternalExecutorOverride{User: security.RootUserName()},
stmt, *j.id,
)
if err != nil {
diff --git a/pkg/kv/client_test.go b/pkg/kv/client_test.go
index abb07b4b666a..b2348569804f 100644
--- a/pkg/kv/client_test.go
+++ b/pkg/kv/client_test.go
@@ -28,7 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
- "github.com/cockroachdb/cockroach/pkg/server"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
@@ -43,7 +43,7 @@ import (
)
// testUser has valid client certs.
-var testUser = server.TestUser
+var testUser = security.TestUser
// checkKVs verifies that a KeyValue slice contains the expected keys and
// values. The values can be either integers or strings; the expected results
diff --git a/pkg/kv/kvserver/batcheval/eval_context.go b/pkg/kv/kvserver/batcheval/eval_context.go
index 17417dcd96d9..eb3db03ee689 100644
--- a/pkg/kv/kvserver/batcheval/eval_context.go
+++ b/pkg/kv/kvserver/batcheval/eval_context.go
@@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
@@ -97,7 +98,7 @@ type EvalContext interface {
GetDescAndLease(context.Context) (roachpb.RangeDescriptor, roachpb.Lease)
GetExternalStorage(ctx context.Context, dest roachpb.ExternalStorage) (cloud.ExternalStorage, error)
- GetExternalStorageFromURI(ctx context.Context, uri string, user string) (cloud.ExternalStorage,
+ GetExternalStorageFromURI(ctx context.Context, uri string, user security.SQLUsername) (cloud.ExternalStorage,
error)
}
@@ -222,7 +223,7 @@ func (m *mockEvalCtxImpl) GetExternalStorage(
}
func (m *mockEvalCtxImpl) GetExternalStorageFromURI(
- ctx context.Context, uri string, user string,
+ ctx context.Context, uri string, user security.SQLUsername,
) (cloud.ExternalStorage, error) {
panic("unimplemented")
}
diff --git a/pkg/kv/kvserver/client_rangefeed_test.go b/pkg/kv/kvserver/client_rangefeed_test.go
index 3b6d75c5eca6..76c2730722c5 100644
--- a/pkg/kv/kvserver/client_rangefeed_test.go
+++ b/pkg/kv/kvserver/client_rangefeed_test.go
@@ -69,7 +69,7 @@ func TestRangefeedWorksOnSystemRangesUnconditionally(t *testing.T) {
require.GreaterOrEqual(t, keys.MaxReservedDescID, junkDescriptorID)
junkDescriptorKey := catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, junkDescriptorID)
junkDescriptor := dbdesc.NewInitial(
- junkDescriptorID, "junk", security.AdminRole)
+ junkDescriptorID, "junk", security.AdminRoleName())
require.NoError(t, db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
if err := txn.SetSystemConfigTrigger(true /* forSystemTenant */); err != nil {
return err
diff --git a/pkg/kv/kvserver/gossip_test.go b/pkg/kv/kvserver/gossip_test.go
index 2b7fea130f63..642a0e144c76 100644
--- a/pkg/kv/kvserver/gossip_test.go
+++ b/pkg/kv/kvserver/gossip_test.go
@@ -219,13 +219,13 @@ func TestGossipAfterAbortOfSystemConfigTransactionAfterFailureDueToIntents(t *te
txB := db.NewTxn(ctx, "b")
require.NoError(t, txA.SetSystemConfigTrigger(true /* forSystemTenant */))
- db1000 := dbdesc.NewInitial(1000, "1000", security.AdminRole)
+ db1000 := dbdesc.NewInitial(1000, "1000", security.AdminRoleName())
require.NoError(t, txA.Put(ctx,
keys.SystemSQLCodec.DescMetadataKey(1000),
db1000.DescriptorProto()))
require.NoError(t, txB.SetSystemConfigTrigger(true /* forSystemTenant */))
- db2000 := dbdesc.NewInitial(2000, "2000", security.AdminRole)
+ db2000 := dbdesc.NewInitial(2000, "2000", security.AdminRoleName())
require.NoError(t, txB.Put(ctx,
keys.SystemSQLCodec.DescMetadataKey(2000),
db2000.DescriptorProto()))
diff --git a/pkg/kv/kvserver/log.go b/pkg/kv/kvserver/log.go
index 226db8138e37..e25c4b20b16b 100644
--- a/pkg/kv/kvserver/log.go
+++ b/pkg/kv/kvserver/log.go
@@ -80,7 +80,7 @@ func (s *Store) insertRangeLogEvent(
}
rows, err := s.cfg.SQLExecutor.ExecEx(ctx, "log-range-event", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
insertEventTableStmt, args...)
if err != nil {
return err
diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage.go b/pkg/kv/kvserver/protectedts/ptstorage/storage.go
index 1cf02cda89e7..7cec76aecc54 100644
--- a/pkg/kv/kvserver/protectedts/ptstorage/storage.go
+++ b/pkg/kv/kvserver/protectedts/ptstorage/storage.go
@@ -78,7 +78,7 @@ func (p *storage) Protect(ctx context.Context, txn *kv.Txn, r *ptpb.Record) erro
}
s := makeSettings(p.settings)
rows, err := p.ex.QueryEx(ctx, "protectedts-protect", txn,
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
protectQuery,
s.maxSpans, s.maxBytes, len(r.Spans),
r.ID.GetBytesMut(), r.Timestamp.AsOfSystemTime(),
@@ -114,7 +114,7 @@ func (p *storage) GetRecord(ctx context.Context, txn *kv.Txn, id uuid.UUID) (*pt
return nil, errNoTxn
}
row, err := p.ex.QueryRowEx(ctx, "protectedts-GetRecord", txn,
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
getRecordQuery, id.GetBytesMut())
if err != nil {
return nil, errors.Wrapf(err, "failed to read record %v", id)
@@ -134,7 +134,7 @@ func (p *storage) MarkVerified(ctx context.Context, txn *kv.Txn, id uuid.UUID) e
return errNoTxn
}
rows, err := p.ex.QueryEx(ctx, "protectedts-MarkVerified", txn,
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
markVerifiedQuery, id.GetBytesMut())
if err != nil {
return errors.Wrapf(err, "failed to mark record %v as verified", id)
@@ -150,7 +150,7 @@ func (p *storage) Release(ctx context.Context, txn *kv.Txn, id uuid.UUID) error
return errNoTxn
}
rows, err := p.ex.QueryEx(ctx, "protectedts-Release", txn,
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
releaseQuery, id.GetBytesMut())
if err != nil {
return errors.Wrapf(err, "failed to release record %v", id)
@@ -166,7 +166,7 @@ func (p *storage) GetMetadata(ctx context.Context, txn *kv.Txn) (ptpb.Metadata,
return ptpb.Metadata{}, errNoTxn
}
row, err := p.ex.QueryRowEx(ctx, "protectedts-GetMetadata", txn,
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
getMetadataQuery)
if err != nil {
return ptpb.Metadata{}, errors.Wrap(err, "failed to read metadata")
@@ -199,7 +199,7 @@ func (p *storage) GetState(ctx context.Context, txn *kv.Txn) (ptpb.State, error)
func (p *storage) getRecords(ctx context.Context, txn *kv.Txn) ([]ptpb.Record, error) {
rows, err := p.ex.QueryEx(ctx, "protectedts-GetRecords", txn,
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
getRecordsQuery)
if err != nil {
return nil, errors.Wrap(err, "failed to read records")
diff --git a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go
index dd8987d462b3..a4d7f9214f4f 100644
--- a/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go
+++ b/pkg/kv/kvserver/protectedts/ptstorage/storage_test.go
@@ -491,7 +491,7 @@ func TestCorruptData(t *testing.T) {
ie := tc.Server(0).InternalExecutor().(sqlutil.InternalExecutor)
affected, err := ie.ExecEx(
ctx, "corrupt-data", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
"UPDATE system.protected_ts_records SET spans = $1 WHERE id = $2",
[]byte("junk"), rec.ID.String())
require.NoError(t, err)
@@ -541,7 +541,7 @@ func TestCorruptData(t *testing.T) {
ie := tc.Server(0).InternalExecutor().(sqlutil.InternalExecutor)
affected, err := ie.ExecEx(
ctx, "corrupt-data", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
"UPDATE system.protected_ts_records SET ts = $1 WHERE id = $2",
d.String(), rec.ID.String())
require.NoError(t, err)
diff --git a/pkg/kv/kvserver/replica.go b/pkg/kv/kvserver/replica.go
index bdf55b62fd49..6998d39db018 100644
--- a/pkg/kv/kvserver/replica.go
+++ b/pkg/kv/kvserver/replica.go
@@ -34,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/tenantrate"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
@@ -1692,7 +1693,7 @@ func (r *Replica) GetExternalStorage(
// GetExternalStorageFromURI returns an ExternalStorage object, based on the given URI.
func (r *Replica) GetExternalStorageFromURI(
- ctx context.Context, uri string, user string,
+ ctx context.Context, uri string, user security.SQLUsername,
) (cloud.ExternalStorage, error) {
return r.store.cfg.ExternalStorageFromURI(ctx, uri, user)
}
diff --git a/pkg/kv/kvserver/replica_eval_context_span.go b/pkg/kv/kvserver/replica_eval_context_span.go
index c7fb650a4b9c..60b5cb2f6fca 100644
--- a/pkg/kv/kvserver/replica_eval_context_span.go
+++ b/pkg/kv/kvserver/replica_eval_context_span.go
@@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
@@ -226,7 +227,7 @@ func (rec *SpanSetReplicaEvalContext) GetExternalStorage(
// GetExternalStorageFromURI returns an ExternalStorage object, based on the given URI.
func (rec *SpanSetReplicaEvalContext) GetExternalStorageFromURI(
- ctx context.Context, uri string, user string,
+ ctx context.Context, uri string, user security.SQLUsername,
) (cloud.ExternalStorage, error) {
return rec.i.GetExternalStorageFromURI(ctx, uri, user)
}
diff --git a/pkg/kv/kvserver/reports/constraint_stats_report_test.go b/pkg/kv/kvserver/reports/constraint_stats_report_test.go
index 1118d57663ab..6c45cd351a99 100644
--- a/pkg/kv/kvserver/reports/constraint_stats_report_test.go
+++ b/pkg/kv/kvserver/reports/constraint_stats_report_test.go
@@ -762,7 +762,7 @@ func compileTestCase(tc baseReportTestCase) (compiledTestCase, error) {
}
}
sysCfgBuilder.addDBDesc(dbID,
- &dbdesc.NewInitial(descpb.ID(dbID), db.name, security.AdminRole).Immutable)
+ &dbdesc.NewInitial(descpb.ID(dbID), db.name, security.AdminRoleName()).Immutable)
for _, table := range db.tables {
tableID := objectCounter
diff --git a/pkg/kv/kvserver/reports/reporter.go b/pkg/kv/kvserver/reports/reporter.go
index 109a32d5863d..05eb3b5c0f56 100644
--- a/pkg/kv/kvserver/reports/reporter.go
+++ b/pkg/kv/kvserver/reports/reporter.go
@@ -784,7 +784,7 @@ func getReportGenerationTime(
ctx,
"get-previous-timestamp",
txn,
- sessiondata.InternalExecutorOverride{User: security.NodeUser},
+ sessiondata.InternalExecutorOverride{User: security.NodeUserName()},
"select generated from system.reports_meta where id = $1",
rid,
)
diff --git a/pkg/roachpb/api.pb.go b/pkg/roachpb/api.pb.go
index b12b64dbd8a9..4ed07e35c6e3 100644
--- a/pkg/roachpb/api.pb.go
+++ b/pkg/roachpb/api.pb.go
@@ -72,7 +72,7 @@ func (x ReadConsistencyType) String() string {
return proto.EnumName(ReadConsistencyType_name, int32(x))
}
func (ReadConsistencyType) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{0}
+ return fileDescriptor_api_763908915cb85439, []int{0}
}
// ScanFormat is an enumeration of the available response formats for MVCCScan
@@ -100,7 +100,7 @@ func (x ScanFormat) String() string {
return proto.EnumName(ScanFormat_name, int32(x))
}
func (ScanFormat) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{1}
+ return fileDescriptor_api_763908915cb85439, []int{1}
}
type ChecksumMode int32
@@ -147,7 +147,7 @@ func (x ChecksumMode) String() string {
return proto.EnumName(ChecksumMode_name, int32(x))
}
func (ChecksumMode) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{2}
+ return fileDescriptor_api_763908915cb85439, []int{2}
}
// PushTxnType determines what action to take when pushing a transaction.
@@ -178,7 +178,7 @@ func (x PushTxnType) String() string {
return proto.EnumName(PushTxnType_name, int32(x))
}
func (PushTxnType) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{3}
+ return fileDescriptor_api_763908915cb85439, []int{3}
}
type ExternalStorageProvider int32
@@ -219,7 +219,7 @@ func (x ExternalStorageProvider) String() string {
return proto.EnumName(ExternalStorageProvider_name, int32(x))
}
func (ExternalStorageProvider) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{4}
+ return fileDescriptor_api_763908915cb85439, []int{4}
}
type MVCCFilter int32
@@ -242,7 +242,7 @@ func (x MVCCFilter) String() string {
return proto.EnumName(MVCCFilter_name, int32(x))
}
func (MVCCFilter) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{5}
+ return fileDescriptor_api_763908915cb85439, []int{5}
}
type ResponseHeader_ResumeReason int32
@@ -268,7 +268,7 @@ func (x ResponseHeader_ResumeReason) String() string {
return proto.EnumName(ResponseHeader_ResumeReason_name, int32(x))
}
func (ResponseHeader_ResumeReason) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{1, 0}
+ return fileDescriptor_api_763908915cb85439, []int{1, 0}
}
type CheckConsistencyResponse_Status int32
@@ -310,7 +310,7 @@ func (x CheckConsistencyResponse_Status) String() string {
return proto.EnumName(CheckConsistencyResponse_Status_name, int32(x))
}
func (CheckConsistencyResponse_Status) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{25, 0}
+ return fileDescriptor_api_763908915cb85439, []int{25, 0}
}
// RequestHeader is supplied with every storage node request.
@@ -331,7 +331,7 @@ func (m *RequestHeader) Reset() { *m = RequestHeader{} }
func (m *RequestHeader) String() string { return proto.CompactTextString(m) }
func (*RequestHeader) ProtoMessage() {}
func (*RequestHeader) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{0}
+ return fileDescriptor_api_763908915cb85439, []int{0}
}
func (m *RequestHeader) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -402,7 +402,7 @@ func (m *ResponseHeader) Reset() { *m = ResponseHeader{} }
func (m *ResponseHeader) String() string { return proto.CompactTextString(m) }
func (*ResponseHeader) ProtoMessage() {}
func (*ResponseHeader) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{1}
+ return fileDescriptor_api_763908915cb85439, []int{1}
}
func (m *ResponseHeader) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -436,7 +436,7 @@ func (m *GetRequest) Reset() { *m = GetRequest{} }
func (m *GetRequest) String() string { return proto.CompactTextString(m) }
func (*GetRequest) ProtoMessage() {}
func (*GetRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{2}
+ return fileDescriptor_api_763908915cb85439, []int{2}
}
func (m *GetRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -479,7 +479,7 @@ func (m *GetResponse) Reset() { *m = GetResponse{} }
func (m *GetResponse) String() string { return proto.CompactTextString(m) }
func (*GetResponse) ProtoMessage() {}
func (*GetResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{3}
+ return fileDescriptor_api_763908915cb85439, []int{3}
}
func (m *GetResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -522,7 +522,7 @@ func (m *PutRequest) Reset() { *m = PutRequest{} }
func (m *PutRequest) String() string { return proto.CompactTextString(m) }
func (*PutRequest) ProtoMessage() {}
func (*PutRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{4}
+ return fileDescriptor_api_763908915cb85439, []int{4}
}
func (m *PutRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -556,7 +556,7 @@ func (m *PutResponse) Reset() { *m = PutResponse{} }
func (m *PutResponse) String() string { return proto.CompactTextString(m) }
func (*PutResponse) ProtoMessage() {}
func (*PutResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{5}
+ return fileDescriptor_api_763908915cb85439, []int{5}
}
func (m *PutResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -638,7 +638,7 @@ func (m *ConditionalPutRequest) Reset() { *m = ConditionalPutRequest{} }
func (m *ConditionalPutRequest) String() string { return proto.CompactTextString(m) }
func (*ConditionalPutRequest) ProtoMessage() {}
func (*ConditionalPutRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{6}
+ return fileDescriptor_api_763908915cb85439, []int{6}
}
func (m *ConditionalPutRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -673,7 +673,7 @@ func (m *ConditionalPutResponse) Reset() { *m = ConditionalPutResponse{}
func (m *ConditionalPutResponse) String() string { return proto.CompactTextString(m) }
func (*ConditionalPutResponse) ProtoMessage() {}
func (*ConditionalPutResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{7}
+ return fileDescriptor_api_763908915cb85439, []int{7}
}
func (m *ConditionalPutResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -719,7 +719,7 @@ func (m *InitPutRequest) Reset() { *m = InitPutRequest{} }
func (m *InitPutRequest) String() string { return proto.CompactTextString(m) }
func (*InitPutRequest) ProtoMessage() {}
func (*InitPutRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{8}
+ return fileDescriptor_api_763908915cb85439, []int{8}
}
func (m *InitPutRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -753,7 +753,7 @@ func (m *InitPutResponse) Reset() { *m = InitPutResponse{} }
func (m *InitPutResponse) String() string { return proto.CompactTextString(m) }
func (*InitPutResponse) ProtoMessage() {}
func (*InitPutResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{9}
+ return fileDescriptor_api_763908915cb85439, []int{9}
}
func (m *InitPutResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -793,7 +793,7 @@ func (m *IncrementRequest) Reset() { *m = IncrementRequest{} }
func (m *IncrementRequest) String() string { return proto.CompactTextString(m) }
func (*IncrementRequest) ProtoMessage() {}
func (*IncrementRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{10}
+ return fileDescriptor_api_763908915cb85439, []int{10}
}
func (m *IncrementRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -830,7 +830,7 @@ func (m *IncrementResponse) Reset() { *m = IncrementResponse{} }
func (m *IncrementResponse) String() string { return proto.CompactTextString(m) }
func (*IncrementResponse) ProtoMessage() {}
func (*IncrementResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{11}
+ return fileDescriptor_api_763908915cb85439, []int{11}
}
func (m *IncrementResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -864,7 +864,7 @@ func (m *DeleteRequest) Reset() { *m = DeleteRequest{} }
func (m *DeleteRequest) String() string { return proto.CompactTextString(m) }
func (*DeleteRequest) ProtoMessage() {}
func (*DeleteRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{12}
+ return fileDescriptor_api_763908915cb85439, []int{12}
}
func (m *DeleteRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -898,7 +898,7 @@ func (m *DeleteResponse) Reset() { *m = DeleteResponse{} }
func (m *DeleteResponse) String() string { return proto.CompactTextString(m) }
func (*DeleteResponse) ProtoMessage() {}
func (*DeleteResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{13}
+ return fileDescriptor_api_763908915cb85439, []int{13}
}
func (m *DeleteResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -950,7 +950,7 @@ func (m *DeleteRangeRequest) Reset() { *m = DeleteRangeRequest{} }
func (m *DeleteRangeRequest) String() string { return proto.CompactTextString(m) }
func (*DeleteRangeRequest) ProtoMessage() {}
func (*DeleteRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{14}
+ return fileDescriptor_api_763908915cb85439, []int{14}
}
func (m *DeleteRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -987,7 +987,7 @@ func (m *DeleteRangeResponse) Reset() { *m = DeleteRangeResponse{} }
func (m *DeleteRangeResponse) String() string { return proto.CompactTextString(m) }
func (*DeleteRangeResponse) ProtoMessage() {}
func (*DeleteRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{15}
+ return fileDescriptor_api_763908915cb85439, []int{15}
}
func (m *DeleteRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1042,7 +1042,7 @@ func (m *ClearRangeRequest) Reset() { *m = ClearRangeRequest{} }
func (m *ClearRangeRequest) String() string { return proto.CompactTextString(m) }
func (*ClearRangeRequest) ProtoMessage() {}
func (*ClearRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{16}
+ return fileDescriptor_api_763908915cb85439, []int{16}
}
func (m *ClearRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1076,7 +1076,7 @@ func (m *ClearRangeResponse) Reset() { *m = ClearRangeResponse{} }
func (m *ClearRangeResponse) String() string { return proto.CompactTextString(m) }
func (*ClearRangeResponse) ProtoMessage() {}
func (*ClearRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{17}
+ return fileDescriptor_api_763908915cb85439, []int{17}
}
func (m *ClearRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1117,7 +1117,7 @@ func (m *RevertRangeRequest) Reset() { *m = RevertRangeRequest{} }
func (m *RevertRangeRequest) String() string { return proto.CompactTextString(m) }
func (*RevertRangeRequest) ProtoMessage() {}
func (*RevertRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{18}
+ return fileDescriptor_api_763908915cb85439, []int{18}
}
func (m *RevertRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1151,7 +1151,7 @@ func (m *RevertRangeResponse) Reset() { *m = RevertRangeResponse{} }
func (m *RevertRangeResponse) String() string { return proto.CompactTextString(m) }
func (*RevertRangeResponse) ProtoMessage() {}
func (*RevertRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{19}
+ return fileDescriptor_api_763908915cb85439, []int{19}
}
func (m *RevertRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1202,7 +1202,7 @@ func (m *ScanRequest) Reset() { *m = ScanRequest{} }
func (m *ScanRequest) String() string { return proto.CompactTextString(m) }
func (*ScanRequest) ProtoMessage() {}
func (*ScanRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{20}
+ return fileDescriptor_api_763908915cb85439, []int{20}
}
func (m *ScanRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1254,7 +1254,7 @@ func (m *ScanResponse) Reset() { *m = ScanResponse{} }
func (m *ScanResponse) String() string { return proto.CompactTextString(m) }
func (*ScanResponse) ProtoMessage() {}
func (*ScanResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{21}
+ return fileDescriptor_api_763908915cb85439, []int{21}
}
func (m *ScanResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1305,7 +1305,7 @@ func (m *ReverseScanRequest) Reset() { *m = ReverseScanRequest{} }
func (m *ReverseScanRequest) String() string { return proto.CompactTextString(m) }
func (*ReverseScanRequest) ProtoMessage() {}
func (*ReverseScanRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{22}
+ return fileDescriptor_api_763908915cb85439, []int{22}
}
func (m *ReverseScanRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1357,7 +1357,7 @@ func (m *ReverseScanResponse) Reset() { *m = ReverseScanResponse{} }
func (m *ReverseScanResponse) String() string { return proto.CompactTextString(m) }
func (*ReverseScanResponse) ProtoMessage() {}
func (*ReverseScanResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{23}
+ return fileDescriptor_api_763908915cb85439, []int{23}
}
func (m *ReverseScanResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1410,7 +1410,7 @@ func (m *CheckConsistencyRequest) Reset() { *m = CheckConsistencyRequest
func (m *CheckConsistencyRequest) String() string { return proto.CompactTextString(m) }
func (*CheckConsistencyRequest) ProtoMessage() {}
func (*CheckConsistencyRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{24}
+ return fileDescriptor_api_763908915cb85439, []int{24}
}
func (m *CheckConsistencyRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1447,7 +1447,7 @@ func (m *CheckConsistencyResponse) Reset() { *m = CheckConsistencyRespon
func (m *CheckConsistencyResponse) String() string { return proto.CompactTextString(m) }
func (*CheckConsistencyResponse) ProtoMessage() {}
func (*CheckConsistencyResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{25}
+ return fileDescriptor_api_763908915cb85439, []int{25}
}
func (m *CheckConsistencyResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1491,7 +1491,7 @@ func (m *CheckConsistencyResponse_Result) Reset() { *m = CheckConsistenc
func (m *CheckConsistencyResponse_Result) String() string { return proto.CompactTextString(m) }
func (*CheckConsistencyResponse_Result) ProtoMessage() {}
func (*CheckConsistencyResponse_Result) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{25, 0}
+ return fileDescriptor_api_763908915cb85439, []int{25, 0}
}
func (m *CheckConsistencyResponse_Result) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1539,7 +1539,7 @@ func (m *RecomputeStatsRequest) Reset() { *m = RecomputeStatsRequest{} }
func (m *RecomputeStatsRequest) String() string { return proto.CompactTextString(m) }
func (*RecomputeStatsRequest) ProtoMessage() {}
func (*RecomputeStatsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{26}
+ return fileDescriptor_api_763908915cb85439, []int{26}
}
func (m *RecomputeStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1575,7 +1575,7 @@ func (m *RecomputeStatsResponse) Reset() { *m = RecomputeStatsResponse{}
func (m *RecomputeStatsResponse) String() string { return proto.CompactTextString(m) }
func (*RecomputeStatsResponse) ProtoMessage() {}
func (*RecomputeStatsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{27}
+ return fileDescriptor_api_763908915cb85439, []int{27}
}
func (m *RecomputeStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1686,7 +1686,7 @@ func (m *EndTxnRequest) Reset() { *m = EndTxnRequest{} }
func (m *EndTxnRequest) String() string { return proto.CompactTextString(m) }
func (*EndTxnRequest) ProtoMessage() {}
func (*EndTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{28}
+ return fileDescriptor_api_763908915cb85439, []int{28}
}
func (m *EndTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1732,7 +1732,7 @@ func (m *EndTxnResponse) Reset() { *m = EndTxnResponse{} }
func (m *EndTxnResponse) String() string { return proto.CompactTextString(m) }
func (*EndTxnResponse) ProtoMessage() {}
func (*EndTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{29}
+ return fileDescriptor_api_763908915cb85439, []int{29}
}
func (m *EndTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1793,7 +1793,7 @@ func (m *AdminSplitRequest) Reset() { *m = AdminSplitRequest{} }
func (m *AdminSplitRequest) String() string { return proto.CompactTextString(m) }
func (*AdminSplitRequest) ProtoMessage() {}
func (*AdminSplitRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{30}
+ return fileDescriptor_api_763908915cb85439, []int{30}
}
func (m *AdminSplitRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1828,7 +1828,7 @@ func (m *AdminSplitResponse) Reset() { *m = AdminSplitResponse{} }
func (m *AdminSplitResponse) String() string { return proto.CompactTextString(m) }
func (*AdminSplitResponse) ProtoMessage() {}
func (*AdminSplitResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{31}
+ return fileDescriptor_api_763908915cb85439, []int{31}
}
func (m *AdminSplitResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1867,7 +1867,7 @@ func (m *AdminUnsplitRequest) Reset() { *m = AdminUnsplitRequest{} }
func (m *AdminUnsplitRequest) String() string { return proto.CompactTextString(m) }
func (*AdminUnsplitRequest) ProtoMessage() {}
func (*AdminUnsplitRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{32}
+ return fileDescriptor_api_763908915cb85439, []int{32}
}
func (m *AdminUnsplitRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1902,7 +1902,7 @@ func (m *AdminUnsplitResponse) Reset() { *m = AdminUnsplitResponse{} }
func (m *AdminUnsplitResponse) String() string { return proto.CompactTextString(m) }
func (*AdminUnsplitResponse) ProtoMessage() {}
func (*AdminUnsplitResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{33}
+ return fileDescriptor_api_763908915cb85439, []int{33}
}
func (m *AdminUnsplitResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1945,7 +1945,7 @@ func (m *AdminMergeRequest) Reset() { *m = AdminMergeRequest{} }
func (m *AdminMergeRequest) String() string { return proto.CompactTextString(m) }
func (*AdminMergeRequest) ProtoMessage() {}
func (*AdminMergeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{34}
+ return fileDescriptor_api_763908915cb85439, []int{34}
}
func (m *AdminMergeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1980,7 +1980,7 @@ func (m *AdminMergeResponse) Reset() { *m = AdminMergeResponse{} }
func (m *AdminMergeResponse) String() string { return proto.CompactTextString(m) }
func (*AdminMergeResponse) ProtoMessage() {}
func (*AdminMergeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{35}
+ return fileDescriptor_api_763908915cb85439, []int{35}
}
func (m *AdminMergeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2018,7 +2018,7 @@ func (m *AdminTransferLeaseRequest) Reset() { *m = AdminTransferLeaseReq
func (m *AdminTransferLeaseRequest) String() string { return proto.CompactTextString(m) }
func (*AdminTransferLeaseRequest) ProtoMessage() {}
func (*AdminTransferLeaseRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{36}
+ return fileDescriptor_api_763908915cb85439, []int{36}
}
func (m *AdminTransferLeaseRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2051,7 +2051,7 @@ func (m *AdminTransferLeaseResponse) Reset() { *m = AdminTransferLeaseRe
func (m *AdminTransferLeaseResponse) String() string { return proto.CompactTextString(m) }
func (*AdminTransferLeaseResponse) ProtoMessage() {}
func (*AdminTransferLeaseResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{37}
+ return fileDescriptor_api_763908915cb85439, []int{37}
}
func (m *AdminTransferLeaseResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2086,7 +2086,7 @@ func (m *ReplicationChange) Reset() { *m = ReplicationChange{} }
func (m *ReplicationChange) String() string { return proto.CompactTextString(m) }
func (*ReplicationChange) ProtoMessage() {}
func (*ReplicationChange) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{38}
+ return fileDescriptor_api_763908915cb85439, []int{38}
}
func (m *ReplicationChange) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2144,7 +2144,7 @@ func (m *AdminChangeReplicasRequest) Reset() { *m = AdminChangeReplicasR
func (m *AdminChangeReplicasRequest) String() string { return proto.CompactTextString(m) }
func (*AdminChangeReplicasRequest) ProtoMessage() {}
func (*AdminChangeReplicasRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{39}
+ return fileDescriptor_api_763908915cb85439, []int{39}
}
func (m *AdminChangeReplicasRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2179,7 +2179,7 @@ func (m *AdminChangeReplicasResponse) Reset() { *m = AdminChangeReplicas
func (m *AdminChangeReplicasResponse) String() string { return proto.CompactTextString(m) }
func (*AdminChangeReplicasResponse) ProtoMessage() {}
func (*AdminChangeReplicasResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{40}
+ return fileDescriptor_api_763908915cb85439, []int{40}
}
func (m *AdminChangeReplicasResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2216,7 +2216,7 @@ func (m *AdminRelocateRangeRequest) Reset() { *m = AdminRelocateRangeReq
func (m *AdminRelocateRangeRequest) String() string { return proto.CompactTextString(m) }
func (*AdminRelocateRangeRequest) ProtoMessage() {}
func (*AdminRelocateRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{41}
+ return fileDescriptor_api_763908915cb85439, []int{41}
}
func (m *AdminRelocateRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2249,7 +2249,7 @@ func (m *AdminRelocateRangeResponse) Reset() { *m = AdminRelocateRangeRe
func (m *AdminRelocateRangeResponse) String() string { return proto.CompactTextString(m) }
func (*AdminRelocateRangeResponse) ProtoMessage() {}
func (*AdminRelocateRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{42}
+ return fileDescriptor_api_763908915cb85439, []int{42}
}
func (m *AdminRelocateRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2288,7 +2288,7 @@ func (m *HeartbeatTxnRequest) Reset() { *m = HeartbeatTxnRequest{} }
func (m *HeartbeatTxnRequest) String() string { return proto.CompactTextString(m) }
func (*HeartbeatTxnRequest) ProtoMessage() {}
func (*HeartbeatTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{43}
+ return fileDescriptor_api_763908915cb85439, []int{43}
}
func (m *HeartbeatTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2325,7 +2325,7 @@ func (m *HeartbeatTxnResponse) Reset() { *m = HeartbeatTxnResponse{} }
func (m *HeartbeatTxnResponse) String() string { return proto.CompactTextString(m) }
func (*HeartbeatTxnResponse) ProtoMessage() {}
func (*HeartbeatTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{44}
+ return fileDescriptor_api_763908915cb85439, []int{44}
}
func (m *HeartbeatTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2363,7 +2363,7 @@ func (m *GCRequest) Reset() { *m = GCRequest{} }
func (m *GCRequest) String() string { return proto.CompactTextString(m) }
func (*GCRequest) ProtoMessage() {}
func (*GCRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{45}
+ return fileDescriptor_api_763908915cb85439, []int{45}
}
func (m *GCRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2397,7 +2397,7 @@ func (m *GCRequest_GCKey) Reset() { *m = GCRequest_GCKey{} }
func (m *GCRequest_GCKey) String() string { return proto.CompactTextString(m) }
func (*GCRequest_GCKey) ProtoMessage() {}
func (*GCRequest_GCKey) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{45, 0}
+ return fileDescriptor_api_763908915cb85439, []int{45, 0}
}
func (m *GCRequest_GCKey) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2431,7 +2431,7 @@ func (m *GCResponse) Reset() { *m = GCResponse{} }
func (m *GCResponse) String() string { return proto.CompactTextString(m) }
func (*GCResponse) ProtoMessage() {}
func (*GCResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{46}
+ return fileDescriptor_api_763908915cb85439, []int{46}
}
func (m *GCResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2500,7 +2500,7 @@ func (m *PushTxnRequest) Reset() { *m = PushTxnRequest{} }
func (m *PushTxnRequest) String() string { return proto.CompactTextString(m) }
func (*PushTxnRequest) ProtoMessage() {}
func (*PushTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{47}
+ return fileDescriptor_api_763908915cb85439, []int{47}
}
func (m *PushTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2543,7 +2543,7 @@ func (m *PushTxnResponse) Reset() { *m = PushTxnResponse{} }
func (m *PushTxnResponse) String() string { return proto.CompactTextString(m) }
func (*PushTxnResponse) ProtoMessage() {}
func (*PushTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{48}
+ return fileDescriptor_api_763908915cb85439, []int{48}
}
func (m *PushTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2590,7 +2590,7 @@ func (m *RecoverTxnRequest) Reset() { *m = RecoverTxnRequest{} }
func (m *RecoverTxnRequest) String() string { return proto.CompactTextString(m) }
func (*RecoverTxnRequest) ProtoMessage() {}
func (*RecoverTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{49}
+ return fileDescriptor_api_763908915cb85439, []int{49}
}
func (m *RecoverTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2626,7 +2626,7 @@ func (m *RecoverTxnResponse) Reset() { *m = RecoverTxnResponse{} }
func (m *RecoverTxnResponse) String() string { return proto.CompactTextString(m) }
func (*RecoverTxnResponse) ProtoMessage() {}
func (*RecoverTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{50}
+ return fileDescriptor_api_763908915cb85439, []int{50}
}
func (m *RecoverTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2670,7 +2670,7 @@ func (m *QueryTxnRequest) Reset() { *m = QueryTxnRequest{} }
func (m *QueryTxnRequest) String() string { return proto.CompactTextString(m) }
func (*QueryTxnRequest) ProtoMessage() {}
func (*QueryTxnRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{51}
+ return fileDescriptor_api_763908915cb85439, []int{51}
}
func (m *QueryTxnRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2714,7 +2714,7 @@ func (m *QueryTxnResponse) Reset() { *m = QueryTxnResponse{} }
func (m *QueryTxnResponse) String() string { return proto.CompactTextString(m) }
func (*QueryTxnResponse) ProtoMessage() {}
func (*QueryTxnResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{52}
+ return fileDescriptor_api_763908915cb85439, []int{52}
}
func (m *QueryTxnResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2774,7 +2774,7 @@ func (m *QueryIntentRequest) Reset() { *m = QueryIntentRequest{} }
func (m *QueryIntentRequest) String() string { return proto.CompactTextString(m) }
func (*QueryIntentRequest) ProtoMessage() {}
func (*QueryIntentRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{53}
+ return fileDescriptor_api_763908915cb85439, []int{53}
}
func (m *QueryIntentRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2810,7 +2810,7 @@ func (m *QueryIntentResponse) Reset() { *m = QueryIntentResponse{} }
func (m *QueryIntentResponse) String() string { return proto.CompactTextString(m) }
func (*QueryIntentResponse) ProtoMessage() {}
func (*QueryIntentResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{54}
+ return fileDescriptor_api_763908915cb85439, []int{54}
}
func (m *QueryIntentResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2857,7 +2857,7 @@ func (m *ResolveIntentRequest) Reset() { *m = ResolveIntentRequest{} }
func (m *ResolveIntentRequest) String() string { return proto.CompactTextString(m) }
func (*ResolveIntentRequest) ProtoMessage() {}
func (*ResolveIntentRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{55}
+ return fileDescriptor_api_763908915cb85439, []int{55}
}
func (m *ResolveIntentRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2892,7 +2892,7 @@ func (m *ResolveIntentResponse) Reset() { *m = ResolveIntentResponse{} }
func (m *ResolveIntentResponse) String() string { return proto.CompactTextString(m) }
func (*ResolveIntentResponse) ProtoMessage() {}
func (*ResolveIntentResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{56}
+ return fileDescriptor_api_763908915cb85439, []int{56}
}
func (m *ResolveIntentResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2943,7 +2943,7 @@ func (m *ResolveIntentRangeRequest) Reset() { *m = ResolveIntentRangeReq
func (m *ResolveIntentRangeRequest) String() string { return proto.CompactTextString(m) }
func (*ResolveIntentRangeRequest) ProtoMessage() {}
func (*ResolveIntentRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{57}
+ return fileDescriptor_api_763908915cb85439, []int{57}
}
func (m *ResolveIntentRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2978,7 +2978,7 @@ func (m *ResolveIntentRangeResponse) Reset() { *m = ResolveIntentRangeRe
func (m *ResolveIntentRangeResponse) String() string { return proto.CompactTextString(m) }
func (*ResolveIntentRangeResponse) ProtoMessage() {}
func (*ResolveIntentRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{58}
+ return fileDescriptor_api_763908915cb85439, []int{58}
}
func (m *ResolveIntentRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3015,7 +3015,7 @@ func (m *MergeRequest) Reset() { *m = MergeRequest{} }
func (m *MergeRequest) String() string { return proto.CompactTextString(m) }
func (*MergeRequest) ProtoMessage() {}
func (*MergeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{59}
+ return fileDescriptor_api_763908915cb85439, []int{59}
}
func (m *MergeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3049,7 +3049,7 @@ func (m *MergeResponse) Reset() { *m = MergeResponse{} }
func (m *MergeResponse) String() string { return proto.CompactTextString(m) }
func (*MergeResponse) ProtoMessage() {}
func (*MergeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{60}
+ return fileDescriptor_api_763908915cb85439, []int{60}
}
func (m *MergeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3094,7 +3094,7 @@ func (m *TruncateLogRequest) Reset() { *m = TruncateLogRequest{} }
func (m *TruncateLogRequest) String() string { return proto.CompactTextString(m) }
func (*TruncateLogRequest) ProtoMessage() {}
func (*TruncateLogRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{61}
+ return fileDescriptor_api_763908915cb85439, []int{61}
}
func (m *TruncateLogRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3128,7 +3128,7 @@ func (m *TruncateLogResponse) Reset() { *m = TruncateLogResponse{} }
func (m *TruncateLogResponse) String() string { return proto.CompactTextString(m) }
func (*TruncateLogResponse) ProtoMessage() {}
func (*TruncateLogResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{62}
+ return fileDescriptor_api_763908915cb85439, []int{62}
}
func (m *TruncateLogResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3172,7 +3172,7 @@ func (m *RequestLeaseRequest) Reset() { *m = RequestLeaseRequest{} }
func (m *RequestLeaseRequest) String() string { return proto.CompactTextString(m) }
func (*RequestLeaseRequest) ProtoMessage() {}
func (*RequestLeaseRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{63}
+ return fileDescriptor_api_763908915cb85439, []int{63}
}
func (m *RequestLeaseRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3221,7 +3221,7 @@ func (m *TransferLeaseRequest) Reset() { *m = TransferLeaseRequest{} }
func (m *TransferLeaseRequest) String() string { return proto.CompactTextString(m) }
func (*TransferLeaseRequest) ProtoMessage() {}
func (*TransferLeaseRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{64}
+ return fileDescriptor_api_763908915cb85439, []int{64}
}
func (m *TransferLeaseRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3258,7 +3258,7 @@ func (m *LeaseInfoRequest) Reset() { *m = LeaseInfoRequest{} }
func (m *LeaseInfoRequest) String() string { return proto.CompactTextString(m) }
func (*LeaseInfoRequest) ProtoMessage() {}
func (*LeaseInfoRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{65}
+ return fileDescriptor_api_763908915cb85439, []int{65}
}
func (m *LeaseInfoRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3295,7 +3295,7 @@ func (m *LeaseInfoResponse) Reset() { *m = LeaseInfoResponse{} }
func (m *LeaseInfoResponse) String() string { return proto.CompactTextString(m) }
func (*LeaseInfoResponse) ProtoMessage() {}
func (*LeaseInfoResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{66}
+ return fileDescriptor_api_763908915cb85439, []int{66}
}
func (m *LeaseInfoResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3330,7 +3330,7 @@ func (m *RequestLeaseResponse) Reset() { *m = RequestLeaseResponse{} }
func (m *RequestLeaseResponse) String() string { return proto.CompactTextString(m) }
func (*RequestLeaseResponse) ProtoMessage() {}
func (*RequestLeaseResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{67}
+ return fileDescriptor_api_763908915cb85439, []int{67}
}
func (m *RequestLeaseResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3385,7 +3385,7 @@ func (m *ComputeChecksumRequest) Reset() { *m = ComputeChecksumRequest{}
func (m *ComputeChecksumRequest) String() string { return proto.CompactTextString(m) }
func (*ComputeChecksumRequest) ProtoMessage() {}
func (*ComputeChecksumRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{68}
+ return fileDescriptor_api_763908915cb85439, []int{68}
}
func (m *ComputeChecksumRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3422,7 +3422,7 @@ func (m *ComputeChecksumResponse) Reset() { *m = ComputeChecksumResponse
func (m *ComputeChecksumResponse) String() string { return proto.CompactTextString(m) }
func (*ComputeChecksumResponse) ProtoMessage() {}
func (*ComputeChecksumResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{69}
+ return fileDescriptor_api_763908915cb85439, []int{69}
}
func (m *ComputeChecksumResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3462,7 +3462,7 @@ func (m *ExternalStorage) Reset() { *m = ExternalStorage{} }
func (m *ExternalStorage) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage) ProtoMessage() {}
func (*ExternalStorage) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{70}
+ return fileDescriptor_api_763908915cb85439, []int{70}
}
func (m *ExternalStorage) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3496,7 +3496,7 @@ func (m *ExternalStorage_LocalFilePath) Reset() { *m = ExternalStorage_L
func (m *ExternalStorage_LocalFilePath) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_LocalFilePath) ProtoMessage() {}
func (*ExternalStorage_LocalFilePath) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{70, 0}
+ return fileDescriptor_api_763908915cb85439, []int{70, 0}
}
func (m *ExternalStorage_LocalFilePath) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3529,7 +3529,7 @@ func (m *ExternalStorage_Http) Reset() { *m = ExternalStorage_Http{} }
func (m *ExternalStorage_Http) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_Http) ProtoMessage() {}
func (*ExternalStorage_Http) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{70, 1}
+ return fileDescriptor_api_763908915cb85439, []int{70, 1}
}
func (m *ExternalStorage_Http) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3569,7 +3569,7 @@ func (m *ExternalStorage_S3) Reset() { *m = ExternalStorage_S3{} }
func (m *ExternalStorage_S3) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_S3) ProtoMessage() {}
func (*ExternalStorage_S3) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{70, 2}
+ return fileDescriptor_api_763908915cb85439, []int{70, 2}
}
func (m *ExternalStorage_S3) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3608,7 +3608,7 @@ func (m *ExternalStorage_GCS) Reset() { *m = ExternalStorage_GCS{} }
func (m *ExternalStorage_GCS) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_GCS) ProtoMessage() {}
func (*ExternalStorage_GCS) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{70, 3}
+ return fileDescriptor_api_763908915cb85439, []int{70, 3}
}
func (m *ExternalStorage_GCS) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3644,7 +3644,7 @@ func (m *ExternalStorage_Azure) Reset() { *m = ExternalStorage_Azure{} }
func (m *ExternalStorage_Azure) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_Azure) ProtoMessage() {}
func (*ExternalStorage_Azure) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{70, 4}
+ return fileDescriptor_api_763908915cb85439, []int{70, 4}
}
func (m *ExternalStorage_Azure) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3683,7 +3683,7 @@ func (m *ExternalStorage_Workload) Reset() { *m = ExternalStorage_Worklo
func (m *ExternalStorage_Workload) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_Workload) ProtoMessage() {}
func (*ExternalStorage_Workload) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{70, 5}
+ return fileDescriptor_api_763908915cb85439, []int{70, 5}
}
func (m *ExternalStorage_Workload) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3711,6 +3711,8 @@ var xxx_messageInfo_ExternalStorage_Workload proto.InternalMessageInfo
type ExternalStorage_FileTable struct {
// User interacting with the external storage. This is used to check access
// privileges of the requested user scoped tables.
+ // This field is really of type security.SQLUsername. We can't use
+ // the type directly however because it would create a circular dependency.
User string `protobuf:"bytes,1,opt,name=user,proto3" json:"user,omitempty"`
// QualifiedTableName specifies the database.schema.tablename which the
// FileTableSystem should interact with when servicing reads/writes.
@@ -3723,7 +3725,7 @@ func (m *ExternalStorage_FileTable) Reset() { *m = ExternalStorage_FileT
func (m *ExternalStorage_FileTable) String() string { return proto.CompactTextString(m) }
func (*ExternalStorage_FileTable) ProtoMessage() {}
func (*ExternalStorage_FileTable) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{70, 6}
+ return fileDescriptor_api_763908915cb85439, []int{70, 6}
}
func (m *ExternalStorage_FileTable) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3763,7 +3765,7 @@ func (m *WriteBatchRequest) Reset() { *m = WriteBatchRequest{} }
func (m *WriteBatchRequest) String() string { return proto.CompactTextString(m) }
func (*WriteBatchRequest) ProtoMessage() {}
func (*WriteBatchRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{71}
+ return fileDescriptor_api_763908915cb85439, []int{71}
}
func (m *WriteBatchRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3797,7 +3799,7 @@ func (m *WriteBatchResponse) Reset() { *m = WriteBatchResponse{} }
func (m *WriteBatchResponse) String() string { return proto.CompactTextString(m) }
func (*WriteBatchResponse) ProtoMessage() {}
func (*WriteBatchResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{72}
+ return fileDescriptor_api_763908915cb85439, []int{72}
}
func (m *WriteBatchResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3833,7 +3835,7 @@ func (m *FileEncryptionOptions) Reset() { *m = FileEncryptionOptions{} }
func (m *FileEncryptionOptions) String() string { return proto.CompactTextString(m) }
func (*FileEncryptionOptions) ProtoMessage() {}
func (*FileEncryptionOptions) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{73}
+ return fileDescriptor_api_763908915cb85439, []int{73}
}
func (m *FileEncryptionOptions) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3903,7 +3905,7 @@ func (m *ExportRequest) Reset() { *m = ExportRequest{} }
func (m *ExportRequest) String() string { return proto.CompactTextString(m) }
func (*ExportRequest) ProtoMessage() {}
func (*ExportRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{74}
+ return fileDescriptor_api_763908915cb85439, []int{74}
}
func (m *ExportRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3955,7 +3957,7 @@ func (m *BulkOpSummary) Reset() { *m = BulkOpSummary{} }
func (m *BulkOpSummary) String() string { return proto.CompactTextString(m) }
func (*BulkOpSummary) ProtoMessage() {}
func (*BulkOpSummary) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{75}
+ return fileDescriptor_api_763908915cb85439, []int{75}
}
func (m *BulkOpSummary) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3991,7 +3993,7 @@ func (m *ExportResponse) Reset() { *m = ExportResponse{} }
func (m *ExportResponse) String() string { return proto.CompactTextString(m) }
func (*ExportResponse) ProtoMessage() {}
func (*ExportResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{76}
+ return fileDescriptor_api_763908915cb85439, []int{76}
}
func (m *ExportResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4031,7 +4033,7 @@ func (m *ExportResponse_File) Reset() { *m = ExportResponse_File{} }
func (m *ExportResponse_File) String() string { return proto.CompactTextString(m) }
func (*ExportResponse_File) ProtoMessage() {}
func (*ExportResponse_File) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{76, 0}
+ return fileDescriptor_api_763908915cb85439, []int{76, 0}
}
func (m *ExportResponse_File) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4082,7 +4084,7 @@ func (m *ImportRequest) Reset() { *m = ImportRequest{} }
func (m *ImportRequest) String() string { return proto.CompactTextString(m) }
func (*ImportRequest) ProtoMessage() {}
func (*ImportRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{77}
+ return fileDescriptor_api_763908915cb85439, []int{77}
}
func (m *ImportRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4117,7 +4119,7 @@ func (m *ImportRequest_File) Reset() { *m = ImportRequest_File{} }
func (m *ImportRequest_File) String() string { return proto.CompactTextString(m) }
func (*ImportRequest_File) ProtoMessage() {}
func (*ImportRequest_File) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{77, 0}
+ return fileDescriptor_api_763908915cb85439, []int{77, 0}
}
func (m *ImportRequest_File) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4153,7 +4155,7 @@ func (m *ImportRequest_TableRekey) Reset() { *m = ImportRequest_TableRek
func (m *ImportRequest_TableRekey) String() string { return proto.CompactTextString(m) }
func (*ImportRequest_TableRekey) ProtoMessage() {}
func (*ImportRequest_TableRekey) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{77, 1}
+ return fileDescriptor_api_763908915cb85439, []int{77, 1}
}
func (m *ImportRequest_TableRekey) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4188,7 +4190,7 @@ func (m *ImportResponse) Reset() { *m = ImportResponse{} }
func (m *ImportResponse) String() string { return proto.CompactTextString(m) }
func (*ImportResponse) ProtoMessage() {}
func (*ImportResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{78}
+ return fileDescriptor_api_763908915cb85439, []int{78}
}
func (m *ImportResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4226,7 +4228,7 @@ func (m *AdminScatterRequest) Reset() { *m = AdminScatterRequest{} }
func (m *AdminScatterRequest) String() string { return proto.CompactTextString(m) }
func (*AdminScatterRequest) ProtoMessage() {}
func (*AdminScatterRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{79}
+ return fileDescriptor_api_763908915cb85439, []int{79}
}
func (m *AdminScatterRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4263,7 +4265,7 @@ func (m *AdminScatterResponse) Reset() { *m = AdminScatterResponse{} }
func (m *AdminScatterResponse) String() string { return proto.CompactTextString(m) }
func (*AdminScatterResponse) ProtoMessage() {}
func (*AdminScatterResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{80}
+ return fileDescriptor_api_763908915cb85439, []int{80}
}
func (m *AdminScatterResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4296,7 +4298,7 @@ func (m *AdminScatterResponse_Range) Reset() { *m = AdminScatterResponse
func (m *AdminScatterResponse_Range) String() string { return proto.CompactTextString(m) }
func (*AdminScatterResponse_Range) ProtoMessage() {}
func (*AdminScatterResponse_Range) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{80, 0}
+ return fileDescriptor_api_763908915cb85439, []int{80, 0}
}
func (m *AdminScatterResponse_Range) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4341,7 +4343,7 @@ func (m *AdminVerifyProtectedTimestampRequest) Reset() { *m = AdminVerif
func (m *AdminVerifyProtectedTimestampRequest) String() string { return proto.CompactTextString(m) }
func (*AdminVerifyProtectedTimestampRequest) ProtoMessage() {}
func (*AdminVerifyProtectedTimestampRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{81}
+ return fileDescriptor_api_763908915cb85439, []int{81}
}
func (m *AdminVerifyProtectedTimestampRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4379,7 +4381,7 @@ func (m *AdminVerifyProtectedTimestampResponse) Reset() { *m = AdminVeri
func (m *AdminVerifyProtectedTimestampResponse) String() string { return proto.CompactTextString(m) }
func (*AdminVerifyProtectedTimestampResponse) ProtoMessage() {}
func (*AdminVerifyProtectedTimestampResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{82}
+ return fileDescriptor_api_763908915cb85439, []int{82}
}
func (m *AdminVerifyProtectedTimestampResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4432,7 +4434,7 @@ func (m *AddSSTableRequest) Reset() { *m = AddSSTableRequest{} }
func (m *AddSSTableRequest) String() string { return proto.CompactTextString(m) }
func (*AddSSTableRequest) ProtoMessage() {}
func (*AddSSTableRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{83}
+ return fileDescriptor_api_763908915cb85439, []int{83}
}
func (m *AddSSTableRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4466,7 +4468,7 @@ func (m *AddSSTableResponse) Reset() { *m = AddSSTableResponse{} }
func (m *AddSSTableResponse) String() string { return proto.CompactTextString(m) }
func (*AddSSTableResponse) ProtoMessage() {}
func (*AddSSTableResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{84}
+ return fileDescriptor_api_763908915cb85439, []int{84}
}
func (m *AddSSTableResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4510,7 +4512,7 @@ func (m *RefreshRequest) Reset() { *m = RefreshRequest{} }
func (m *RefreshRequest) String() string { return proto.CompactTextString(m) }
func (*RefreshRequest) ProtoMessage() {}
func (*RefreshRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{85}
+ return fileDescriptor_api_763908915cb85439, []int{85}
}
func (m *RefreshRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4544,7 +4546,7 @@ func (m *RefreshResponse) Reset() { *m = RefreshResponse{} }
func (m *RefreshResponse) String() string { return proto.CompactTextString(m) }
func (*RefreshResponse) ProtoMessage() {}
func (*RefreshResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{86}
+ return fileDescriptor_api_763908915cb85439, []int{86}
}
func (m *RefreshResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4583,7 +4585,7 @@ func (m *RefreshRangeRequest) Reset() { *m = RefreshRangeRequest{} }
func (m *RefreshRangeRequest) String() string { return proto.CompactTextString(m) }
func (*RefreshRangeRequest) ProtoMessage() {}
func (*RefreshRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{87}
+ return fileDescriptor_api_763908915cb85439, []int{87}
}
func (m *RefreshRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4617,7 +4619,7 @@ func (m *RefreshRangeResponse) Reset() { *m = RefreshRangeResponse{} }
func (m *RefreshRangeResponse) String() string { return proto.CompactTextString(m) }
func (*RefreshRangeResponse) ProtoMessage() {}
func (*RefreshRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{88}
+ return fileDescriptor_api_763908915cb85439, []int{88}
}
func (m *RefreshRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4666,7 +4668,7 @@ func (m *SubsumeRequest) Reset() { *m = SubsumeRequest{} }
func (m *SubsumeRequest) String() string { return proto.CompactTextString(m) }
func (*SubsumeRequest) ProtoMessage() {}
func (*SubsumeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{89}
+ return fileDescriptor_api_763908915cb85439, []int{89}
}
func (m *SubsumeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4715,7 +4717,7 @@ func (m *SubsumeResponse) Reset() { *m = SubsumeResponse{} }
func (m *SubsumeResponse) String() string { return proto.CompactTextString(m) }
func (*SubsumeResponse) ProtoMessage() {}
func (*SubsumeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{90}
+ return fileDescriptor_api_763908915cb85439, []int{90}
}
func (m *SubsumeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4750,7 +4752,7 @@ func (m *RangeStatsRequest) Reset() { *m = RangeStatsRequest{} }
func (m *RangeStatsRequest) String() string { return proto.CompactTextString(m) }
func (*RangeStatsRequest) ProtoMessage() {}
func (*RangeStatsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{91}
+ return fileDescriptor_api_763908915cb85439, []int{91}
}
func (m *RangeStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4792,7 +4794,7 @@ func (m *RangeStatsResponse) Reset() { *m = RangeStatsResponse{} }
func (m *RangeStatsResponse) String() string { return proto.CompactTextString(m) }
func (*RangeStatsResponse) ProtoMessage() {}
func (*RangeStatsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{92}
+ return fileDescriptor_api_763908915cb85439, []int{92}
}
func (m *RangeStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -4875,7 +4877,7 @@ func (m *RequestUnion) Reset() { *m = RequestUnion{} }
func (m *RequestUnion) String() string { return proto.CompactTextString(m) }
func (*RequestUnion) ProtoMessage() {}
func (*RequestUnion) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{93}
+ return fileDescriptor_api_763908915cb85439, []int{93}
}
func (m *RequestUnion) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -6325,7 +6327,7 @@ func (m *ResponseUnion) Reset() { *m = ResponseUnion{} }
func (m *ResponseUnion) String() string { return proto.CompactTextString(m) }
func (*ResponseUnion) ProtoMessage() {}
func (*ResponseUnion) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{94}
+ return fileDescriptor_api_763908915cb85439, []int{94}
}
func (m *ResponseUnion) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -7860,7 +7862,7 @@ func (m *Header) Reset() { *m = Header{} }
func (m *Header) String() string { return proto.CompactTextString(m) }
func (*Header) ProtoMessage() {}
func (*Header) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{95}
+ return fileDescriptor_api_763908915cb85439, []int{95}
}
func (m *Header) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -7898,7 +7900,7 @@ func (m *ClientRangeInfo) Reset() { *m = ClientRangeInfo{} }
func (m *ClientRangeInfo) String() string { return proto.CompactTextString(m) }
func (*ClientRangeInfo) ProtoMessage() {}
func (*ClientRangeInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{96}
+ return fileDescriptor_api_763908915cb85439, []int{96}
}
func (m *ClientRangeInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -7934,7 +7936,7 @@ type BatchRequest struct {
func (m *BatchRequest) Reset() { *m = BatchRequest{} }
func (*BatchRequest) ProtoMessage() {}
func (*BatchRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{97}
+ return fileDescriptor_api_763908915cb85439, []int{97}
}
func (m *BatchRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -7971,7 +7973,7 @@ type BatchResponse struct {
func (m *BatchResponse) Reset() { *m = BatchResponse{} }
func (*BatchResponse) ProtoMessage() {}
func (*BatchResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{98}
+ return fileDescriptor_api_763908915cb85439, []int{98}
}
func (m *BatchResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8045,7 +8047,7 @@ func (m *BatchResponse_Header) Reset() { *m = BatchResponse_Header{} }
func (m *BatchResponse_Header) String() string { return proto.CompactTextString(m) }
func (*BatchResponse_Header) ProtoMessage() {}
func (*BatchResponse_Header) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{98, 0}
+ return fileDescriptor_api_763908915cb85439, []int{98, 0}
}
func (m *BatchResponse_Header) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8083,7 +8085,7 @@ func (m *RangeLookupRequest) Reset() { *m = RangeLookupRequest{} }
func (m *RangeLookupRequest) String() string { return proto.CompactTextString(m) }
func (*RangeLookupRequest) ProtoMessage() {}
func (*RangeLookupRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{99}
+ return fileDescriptor_api_763908915cb85439, []int{99}
}
func (m *RangeLookupRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8121,7 +8123,7 @@ func (m *RangeLookupResponse) Reset() { *m = RangeLookupResponse{} }
func (m *RangeLookupResponse) String() string { return proto.CompactTextString(m) }
func (*RangeLookupResponse) ProtoMessage() {}
func (*RangeLookupResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{100}
+ return fileDescriptor_api_763908915cb85439, []int{100}
}
func (m *RangeLookupResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8160,7 +8162,7 @@ func (m *RangeFeedRequest) Reset() { *m = RangeFeedRequest{} }
func (m *RangeFeedRequest) String() string { return proto.CompactTextString(m) }
func (*RangeFeedRequest) ProtoMessage() {}
func (*RangeFeedRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{101}
+ return fileDescriptor_api_763908915cb85439, []int{101}
}
func (m *RangeFeedRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8201,7 +8203,7 @@ func (m *RangeFeedValue) Reset() { *m = RangeFeedValue{} }
func (m *RangeFeedValue) String() string { return proto.CompactTextString(m) }
func (*RangeFeedValue) ProtoMessage() {}
func (*RangeFeedValue) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{102}
+ return fileDescriptor_api_763908915cb85439, []int{102}
}
func (m *RangeFeedValue) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8242,7 +8244,7 @@ func (m *RangeFeedCheckpoint) Reset() { *m = RangeFeedCheckpoint{} }
func (m *RangeFeedCheckpoint) String() string { return proto.CompactTextString(m) }
func (*RangeFeedCheckpoint) ProtoMessage() {}
func (*RangeFeedCheckpoint) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{103}
+ return fileDescriptor_api_763908915cb85439, []int{103}
}
func (m *RangeFeedCheckpoint) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8279,7 +8281,7 @@ func (m *RangeFeedError) Reset() { *m = RangeFeedError{} }
func (m *RangeFeedError) String() string { return proto.CompactTextString(m) }
func (*RangeFeedError) ProtoMessage() {}
func (*RangeFeedError) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{104}
+ return fileDescriptor_api_763908915cb85439, []int{104}
}
func (m *RangeFeedError) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8316,7 +8318,7 @@ func (m *RangeFeedEvent) Reset() { *m = RangeFeedEvent{} }
func (m *RangeFeedEvent) String() string { return proto.CompactTextString(m) }
func (*RangeFeedEvent) ProtoMessage() {}
func (*RangeFeedEvent) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{105}
+ return fileDescriptor_api_763908915cb85439, []int{105}
}
func (m *RangeFeedEvent) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8357,7 +8359,7 @@ func (m *GossipSubscriptionRequest) Reset() { *m = GossipSubscriptionReq
func (m *GossipSubscriptionRequest) String() string { return proto.CompactTextString(m) }
func (*GossipSubscriptionRequest) ProtoMessage() {}
func (*GossipSubscriptionRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{106}
+ return fileDescriptor_api_763908915cb85439, []int{106}
}
func (m *GossipSubscriptionRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8397,7 +8399,7 @@ func (m *GossipSubscriptionEvent) Reset() { *m = GossipSubscriptionEvent
func (m *GossipSubscriptionEvent) String() string { return proto.CompactTextString(m) }
func (*GossipSubscriptionEvent) ProtoMessage() {}
func (*GossipSubscriptionEvent) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{107}
+ return fileDescriptor_api_763908915cb85439, []int{107}
}
func (m *GossipSubscriptionEvent) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8433,7 +8435,7 @@ func (m *JoinNodeRequest) Reset() { *m = JoinNodeRequest{} }
func (m *JoinNodeRequest) String() string { return proto.CompactTextString(m) }
func (*JoinNodeRequest) ProtoMessage() {}
func (*JoinNodeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{108}
+ return fileDescriptor_api_763908915cb85439, []int{108}
}
func (m *JoinNodeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -8472,7 +8474,7 @@ func (m *JoinNodeResponse) Reset() { *m = JoinNodeResponse{} }
func (m *JoinNodeResponse) String() string { return proto.CompactTextString(m) }
func (*JoinNodeResponse) ProtoMessage() {}
func (*JoinNodeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_api_81c6bdcd9303d014, []int{109}
+ return fileDescriptor_api_763908915cb85439, []int{109}
}
func (m *JoinNodeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -39782,9 +39784,9 @@ var (
ErrIntOverflowApi = fmt.Errorf("proto: integer overflow")
)
-func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_81c6bdcd9303d014) }
+func init() { proto.RegisterFile("roachpb/api.proto", fileDescriptor_api_763908915cb85439) }
-var fileDescriptor_api_81c6bdcd9303d014 = []byte{
+var fileDescriptor_api_763908915cb85439 = []byte{
// 8018 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe4, 0x7d, 0x6f, 0x6c, 0x1b, 0x49,
0x96, 0x9f, 0x9a, 0xa4, 0x24, 0xf2, 0x91, 0xa2, 0xa8, 0x92, 0x6c, 0xd3, 0x9a, 0x19, 0xcb, 0xa6,
diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto
index 74608a38cdf9..6e87af464cc9 100644
--- a/pkg/roachpb/api.proto
+++ b/pkg/roachpb/api.proto
@@ -1367,6 +1367,8 @@ message ExternalStorage {
// User interacting with the external storage. This is used to check access
// privileges of the requested user scoped tables.
+ // This field is really of type security.SQLUsername. We can't use
+ // the type directly however because it would create a circular dependency.
string user = 1;
// QualifiedTableName specifies the database.schema.tablename which the
diff --git a/pkg/rpc/pg.go b/pkg/rpc/pg.go
index e16f0c9cbde2..64f8aec4b639 100644
--- a/pkg/rpc/pg.go
+++ b/pkg/rpc/pg.go
@@ -18,7 +18,9 @@ import (
// LoadSecurityOptions extends a url.Values with SSL settings suitable for
// the given server config.
-func (ctx *SecurityContext) LoadSecurityOptions(options url.Values, username string) error {
+func (ctx *SecurityContext) LoadSecurityOptions(
+ options url.Values, username security.SQLUsername,
+) error {
if ctx.config.Insecure {
options.Set("sslmode", "disable")
options.Del("sslrootcert")
@@ -81,7 +83,8 @@ func (ctx *SecurityContext) LoadSecurityOptions(options url.Values, username str
// config. There is no default database set.
func (ctx *SecurityContext) PGURL(user *url.Userinfo) (*url.URL, error) {
options := url.Values{}
- if err := ctx.LoadSecurityOptions(options, user.Username()); err != nil {
+ username, _ := security.MakeSQLUsernameFromUserInput(user.Username(), security.UsernameValidation)
+ if err := ctx.LoadSecurityOptions(options, username); err != nil {
return nil, err
}
return &url.URL{
diff --git a/pkg/rpc/tls.go b/pkg/rpc/tls.go
index 2f7353b241e8..c5211dcb43bf 100644
--- a/pkg/rpc/tls.go
+++ b/pkg/rpc/tls.go
@@ -235,8 +235,8 @@ func (ctx *SecurityContext) GetHTTPClient() (http.Client, error) {
// getClientCertPaths returns the paths to the client cert and key. This uses
// the node certs for the NodeUser, and the actual client certs for all others.
-func (ctx *SecurityContext) getClientCertPaths(user string) (string, string) {
- if user == security.NodeUser {
+func (ctx *SecurityContext) getClientCertPaths(user security.SQLUsername) (string, string) {
+ if user.IsNodeUser() {
return ctx.NodeCertPath(), ctx.NodeKeyPath()
}
return ctx.ClientCertPath(user), ctx.ClientKeyPath(user)
diff --git a/pkg/rpc/tls_test.go b/pkg/rpc/tls_test.go
index dc816692d734..3fe4f888a8e2 100644
--- a/pkg/rpc/tls_test.go
+++ b/pkg/rpc/tls_test.go
@@ -29,24 +29,26 @@ func TestClientSSLSettings(t *testing.T) {
const clientCertNotFound = "problem with client cert for user .*: not found"
const certDirNotFound = "no certificates found"
+ invalidUser := security.MakeSQLUsernameFromPreNormalizedString("not-a-user")
+ badUser := security.MakeSQLUsernameFromPreNormalizedString("bad-user")
testCases := []struct {
// args
insecure bool
hasCerts bool
- user string
+ user security.SQLUsername
// output
requestScheme string
configErr string
nilConfig bool
noCAs bool
}{
- {true, false, security.NodeUser, "http", "", true, false},
- {true, true, "not-a-user", "http", "", true, false},
- {false, true, "not-a-user", "https", clientCertNotFound, true, false},
- {false, false, security.NodeUser, "https", certDirNotFound, false, true},
- {false, true, security.NodeUser, "https", "", false, false},
- {false, true, "bad-user", "https", clientCertNotFound, false, false},
+ {true, false, security.NodeUserName(), "http", "", true, false},
+ {true, true, invalidUser, "http", "", true, false},
+ {false, true, invalidUser, "https", clientCertNotFound, true, false},
+ {false, false, security.NodeUserName(), "https", certDirNotFound, false, true},
+ {false, true, security.NodeUserName(), "https", "", false, false},
+ {false, true, badUser, "https", clientCertNotFound, false, false},
}
for _, tc := range testCases {
@@ -112,7 +114,7 @@ func TestServerSSLSettings(t *testing.T) {
for tcNum, tc := range testCases {
t.Run("", func(t *testing.T) {
- cfg := &base.Config{Insecure: tc.insecure, User: security.NodeUser}
+ cfg := &base.Config{Insecure: tc.insecure, User: security.NodeUserName()}
if tc.hasCerts {
testutils.FillCerts(cfg)
}
diff --git a/pkg/scheduledjobs/env.go b/pkg/scheduledjobs/env.go
index b1b9eabc6627..9643cec16f74 100644
--- a/pkg/scheduledjobs/env.go
+++ b/pkg/scheduledjobs/env.go
@@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/sqlutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
@@ -47,7 +48,7 @@ type JobExecutionConfig struct {
// be cast to that type in the sql package when it is used. Returns a cleanup
// function that must be called once the caller is done with the planner.
// This is the same mechanism used in jobs.Registry.
- PlanHookMaker func(opName string, tnx *kv.Txn, user string) (interface{}, func())
+ PlanHookMaker func(opName string, tnx *kv.Txn, user security.SQLUsername) (interface{}, func())
}
// production JobSchedulerEnv implementation.
diff --git a/pkg/security/auth.go b/pkg/security/auth.go
index 8bdfbdcf148d..2fcef2156292 100644
--- a/pkg/security/auth.go
+++ b/pkg/security/auth.go
@@ -19,22 +19,6 @@ import (
"github.com/cockroachdb/errors"
)
-const (
- // NodeUser is used by nodes for intra-cluster traffic.
- NodeUser = "node"
- // RootUser is the default cluster administrator.
- RootUser = "root"
-
- // AdminRole is the default (and non-droppable) role with superuser privileges.
- AdminRole = "admin"
-
- // PublicRole is the special "public" pseudo-role.
- // All users are implicit members of "public". The role cannot be created,
- // dropped, assigned to another role, and is generally not listed.
- // It can be granted privileges, implicitly granting them to all users (current and future).
- PublicRole = "public"
-)
-
var certPrincipalMap struct {
syncutil.RWMutex
m map[string]string
@@ -43,7 +27,7 @@ var certPrincipalMap struct {
// UserAuthHook authenticates a user based on their username and whether their
// connection originates from a client or another node in the cluster. It
// returns an optional func that is run at connection close.
-type UserAuthHook func(string, bool) (connClose func(), _ error)
+type UserAuthHook func(SQLUsername, bool) (connClose func(), _ error)
// SetCertPrincipalMap sets the global principal map. Each entry in the mapping
// list must either be empty or have the format :. The principal
@@ -124,13 +108,13 @@ func UserAuthCertHook(insecureMode bool, tlsState *tls.ConnectionState) (UserAut
}
}
- return func(requestedUser string, clientConnection bool) (func(), error) {
+ return func(requestedUser SQLUsername, clientConnection bool) (func(), error) {
// TODO(marc): we may eventually need stricter user syntax rules.
- if len(requestedUser) == 0 {
+ if requestedUser.Undefined() {
return nil, errors.New("user is missing")
}
- if !clientConnection && requestedUser != NodeUser {
+ if !clientConnection && !requestedUser.IsNodeUser() {
return nil, errors.Errorf("user %s is not allowed", requestedUser)
}
@@ -151,7 +135,7 @@ func UserAuthCertHook(insecureMode bool, tlsState *tls.ConnectionState) (UserAut
// The client certificate user must match the requested user,
// except if the certificate user is NodeUser, which is allowed to
// act on behalf of all other users.
- if !Contains(certUsers, requestedUser) && !Contains(certUsers, NodeUser) {
+ if !Contains(certUsers, requestedUser.Normalized()) && !Contains(certUsers, NodeUser) {
return nil, errors.Errorf("requested user is %s, but certificate is for %s", requestedUser, certUsers)
}
@@ -162,8 +146,8 @@ func UserAuthCertHook(insecureMode bool, tlsState *tls.ConnectionState) (UserAut
// UserAuthPasswordHook builds an authentication hook based on the security
// mode, password, and its potentially matching hash.
func UserAuthPasswordHook(insecureMode bool, password string, hashedPassword []byte) UserAuthHook {
- return func(requestedUser string, clientConnection bool) (func(), error) {
- if len(requestedUser) == 0 {
+ return func(requestedUser SQLUsername, clientConnection bool) (func(), error) {
+ if requestedUser.Undefined() {
return nil, errors.New("user is missing")
}
diff --git a/pkg/security/auth_test.go b/pkg/security/auth_test.go
index 8a4a54168adc..6ad9e90cc94b 100644
--- a/pkg/security/auth_test.go
+++ b/pkg/security/auth_test.go
@@ -169,35 +169,39 @@ func TestAuthenticationHook(t *testing.T) {
defer leaktest.AfterTest(t)()
defer func() { _ = security.SetCertPrincipalMap(nil) }()
+ fooUser := security.MakeSQLUsernameFromPreNormalizedString("foo")
+ barUser := security.MakeSQLUsernameFromPreNormalizedString("bar")
+ blahUser := security.MakeSQLUsernameFromPreNormalizedString("blah")
+
testCases := []struct {
insecure bool
tlsSpec string
- username string
+ username security.SQLUsername
principalMap string
buildHookSuccess bool
publicHookSuccess bool
privateHookSuccess bool
}{
// Insecure mode, empty username.
- {true, "", "", "", true, false, false},
+ {true, "", security.SQLUsername{}, "", true, false, false},
// Insecure mode, non-empty username.
- {true, "", "foo", "", true, true, false},
+ {true, "", fooUser, "", true, true, false},
// Secure mode, no TLS state.
- {false, "", "", "", false, false, false},
+ {false, "", security.SQLUsername{}, "", false, false, false},
// Secure mode, bad user.
- {false, "foo", "node", "", true, false, false},
+ {false, "foo", security.NodeUserName(), "", true, false, false},
// Secure mode, node user.
- {false, security.NodeUser, "node", "", true, true, true},
+ {false, security.NodeUser, security.NodeUserName(), "", true, true, true},
// Secure mode, root user.
- {false, security.RootUser, "node", "", true, false, false},
+ {false, security.RootUser, security.NodeUserName(), "", true, false, false},
// Secure mode, tenant cert, foo user.
- {false, "(Tenants)foo", "foo", "", true, false, false},
+ {false, "(Tenants)foo", fooUser, "", true, false, false},
// Secure mode, multiple cert principals.
- {false, "foo,bar", "foo", "", true, true, false},
- {false, "foo,bar", "bar", "", true, true, false},
+ {false, "foo,bar", fooUser, "", true, true, false},
+ {false, "foo,bar", barUser, "", true, true, false},
// Secure mode, principal map.
- {false, "foo,bar", "blah", "foo:blah", true, true, false},
- {false, "foo,bar", "blah", "bar:blah", true, true, false},
+ {false, "foo,bar", blahUser, "foo:blah", true, true, false},
+ {false, "foo,bar", blahUser, "bar:blah", true, true, false},
}
for _, tc := range testCases {
diff --git a/pkg/security/certificate_manager.go b/pkg/security/certificate_manager.go
index 36d63bd6faac..7bbead1e84db 100644
--- a/pkg/security/certificate_manager.go
+++ b/pkg/security/certificate_manager.go
@@ -121,7 +121,7 @@ type CertificateManager struct {
nodeCert *CertInfo // certificate for nodes (always server cert, sometimes client cert)
nodeClientCert *CertInfo // optional: client certificate for 'node' user. Also included in 'clientCerts'
uiCert *CertInfo // optional: server certificate for the admin UI.
- clientCerts map[string]*CertInfo
+ clientCerts map[SQLUsername]*CertInfo
// Certs only used with multi-tenancy.
tenantClientCACert, tenantClientCert *CertInfo
@@ -337,20 +337,24 @@ func TenantClientKeyFilename(tenantIdentifier string) string {
}
// ClientCertPath returns the expected file path for the user's certificate.
-func (cl CertsLocator) ClientCertPath(user string) string {
+func (cl CertsLocator) ClientCertPath(user SQLUsername) string {
return filepath.Join(cl.certsDir, ClientCertFilename(user))
}
// ClientCertFilename returns the expected file name for the user's certificate.
-func ClientCertFilename(user string) string { return "client." + user + certExtension }
+func ClientCertFilename(user SQLUsername) string {
+ return "client." + user.Normalized() + certExtension
+}
// ClientKeyPath returns the expected file path for the user's key.
-func (cl CertsLocator) ClientKeyPath(user string) string {
+func (cl CertsLocator) ClientKeyPath(user SQLUsername) string {
return filepath.Join(cl.certsDir, ClientKeyFilename(user))
}
// ClientKeyFilename returns the expected file name for the user's key.
-func ClientKeyFilename(user string) string { return "client." + user + keyExtension }
+func ClientKeyFilename(user SQLUsername) string {
+ return "client." + user.Normalized() + keyExtension
+}
// CACert returns the CA cert. May be nil.
// Callers should check for an internal Error field.
@@ -402,7 +406,7 @@ func (cm *CertificateManager) NodeCert() *CertInfo {
// ClientCerts returns the Client certs.
// Callers should check for internal Error fields.
-func (cm *CertificateManager) ClientCerts() map[string]*CertInfo {
+func (cm *CertificateManager) ClientCerts() map[SQLUsername]*CertInfo {
cm.mu.RLock()
defer cm.mu.RUnlock()
return cm.clientCerts
@@ -441,7 +445,7 @@ func (cm *CertificateManager) LoadCertificates() error {
var caCert, clientCACert, uiCACert, nodeCert, uiCert, nodeClientCert *CertInfo
var tenantClientCACert, tenantClientCert *CertInfo
- clientCerts := make(map[string]*CertInfo)
+ clientCerts := make(map[SQLUsername]*CertInfo)
for _, ci := range cl.Certificates() {
switch ci.FileUsage {
case CAPem:
@@ -468,8 +472,9 @@ func (cm *CertificateManager) LoadCertificates() error {
case UIPem:
uiCert = ci
case ClientPem:
- clientCerts[ci.Name] = ci
- if ci.Name == NodeUser {
+ username := MakeSQLUsernameFromPreNormalizedString(ci.Name)
+ clientCerts[username] = ci
+ if username.IsNodeUser() {
nodeClientCert = ci
}
default:
@@ -761,7 +766,7 @@ func (cm *CertificateManager) getUICertLocked() (*CertInfo, error) {
// getClientCertLocked returns the client cert/key for the specified user,
// or an error if not found.
// cm.mu must be held.
-func (cm *CertificateManager) getClientCertLocked(user string) (*CertInfo, error) {
+func (cm *CertificateManager) getClientCertLocked(user SQLUsername) (*CertInfo, error) {
ci := cm.clientCerts[user]
if err := checkCertIsValid(ci); err != nil {
return nil, makeErrorf(err, "problem with client cert for user %s", user)
@@ -846,7 +851,7 @@ func (cm *CertificateManager) GetTenantClientTLSConfig() (*tls.Config, error) {
// GetClientTLSConfig returns the most up-to-date client tls.Config.
// Returns the dual-purpose node certs if user == NodeUser and there is no
// separate client cert for 'node'.
-func (cm *CertificateManager) GetClientTLSConfig(user string) (*tls.Config, error) {
+func (cm *CertificateManager) GetClientTLSConfig(user SQLUsername) (*tls.Config, error) {
cm.mu.Lock()
defer cm.mu.Unlock()
@@ -856,7 +861,7 @@ func (cm *CertificateManager) GetClientTLSConfig(user string) (*tls.Config, erro
return nil, err
}
- if user != NodeUser {
+ if !user.IsNodeUser() {
clientCert, err := cm.getClientCertLocked(user)
if err != nil {
return nil, err
diff --git a/pkg/security/certificate_manager_test.go b/pkg/security/certificate_manager_test.go
index c52bff889732..612ac2ba861c 100644
--- a/pkg/security/certificate_manager_test.go
+++ b/pkg/security/certificate_manager_test.go
@@ -44,7 +44,7 @@ func TestManagerWithEmbedded(t *testing.T) {
t.Errorf("expected %d client certs, found %d", e, a)
}
- if _, ok := clientCerts[security.RootUser]; !ok {
+ if _, ok := clientCerts[security.RootUserName()]; !ok {
t.Error("no client cert for root user found")
}
@@ -52,19 +52,21 @@ func TestManagerWithEmbedded(t *testing.T) {
if _, err := cm.GetServerTLSConfig(); err != nil {
t.Error(err)
}
- if _, err := cm.GetClientTLSConfig(security.NodeUser); err != nil {
+ if _, err := cm.GetClientTLSConfig(security.NodeUserName()); err != nil {
t.Error(err)
}
- if _, err := cm.GetClientTLSConfig(security.RootUser); err != nil {
+ if _, err := cm.GetClientTLSConfig(security.RootUserName()); err != nil {
t.Error(err)
}
- if _, err := cm.GetClientTLSConfig("testuser"); err != nil {
+ if _, err := cm.GetClientTLSConfig(security.TestUserName()); err != nil {
t.Error(err)
}
- if _, err := cm.GetClientTLSConfig("testuser2"); err != nil {
+ if _, err := cm.GetClientTLSConfig(
+ security.MakeSQLUsernameFromPreNormalizedString("testuser2")); err != nil {
t.Error(err)
}
- if _, err := cm.GetClientTLSConfig("my-random-user"); err == nil {
+ if _, err := cm.GetClientTLSConfig(
+ security.MakeSQLUsernameFromPreNormalizedString("my-random-user")); err == nil {
t.Error("unexpected success")
}
}
@@ -94,7 +96,7 @@ func TestManagerWithPrincipalMap(t *testing.T) {
certsDir, caKey, testKeySize, time.Hour*96, true, true,
))
require.NoError(t, security.CreateClientPair(
- certsDir, caKey, testKeySize, time.Hour*48, true, "testuser", false,
+ certsDir, caKey, testKeySize, time.Hour*48, true, security.TestUserName(), false,
))
require.NoError(t, security.CreateNodePair(
certsDir, caKey, testKeySize, time.Hour*48, true, []string{"127.0.0.1", "foo"},
@@ -107,7 +109,7 @@ func TestManagerWithPrincipalMap(t *testing.T) {
_, err := security.NewCertificateManager(certsDir, security.CommandTLSSettings{})
return err
}
- loadUserCert := func(user string) error {
+ loadUserCert := func(user security.SQLUsername) error {
cm, err := security.NewCertificateManager(certsDir, security.CommandTLSSettings{})
if err != nil {
return err
@@ -133,7 +135,7 @@ func TestManagerWithPrincipalMap(t *testing.T) {
// Mapping the "testuser" principal to a different name should result in an
// error as it no longer matches the file name.
setCertPrincipalMap("testuser:foo,node.crdb.io:node")
- require.Regexp(t, `client certificate has principals \["foo"\], expected "testuser"`, loadUserCert("testuser"))
+ require.Regexp(t, `client certificate has principals \["foo"\], expected "testuser"`, loadUserCert(security.TestUserName()))
// Renaming "client.testuser.crt" to "client.foo.crt" allows us to load it
// under that name.
@@ -142,5 +144,5 @@ func TestManagerWithPrincipalMap(t *testing.T) {
require.NoError(t, os.Rename(filepath.Join(certsDir, "client.testuser.key"),
filepath.Join(certsDir, "client.foo.key")))
setCertPrincipalMap("testuser:foo,node.crdb.io:node")
- require.NoError(t, loadUserCert("foo"))
+ require.NoError(t, loadUserCert(security.MakeSQLUsernameFromPreNormalizedString("foo")))
}
diff --git a/pkg/security/certs.go b/pkg/security/certs.go
index 75aaad7f33b4..0f5acbf2f2b4 100644
--- a/pkg/security/certs.go
+++ b/pkg/security/certs.go
@@ -289,7 +289,10 @@ func CreateNodePair(
// Allow control of the principal to place in the cert via an env var. This
// is intended for testing purposes only.
- nodeUser := envutil.EnvOrDefaultString("COCKROACH_CERT_NODE_USER", NodeUser)
+ nodeUser, _ := MakeSQLUsernameFromUserInput(
+ envutil.EnvOrDefaultString("COCKROACH_CERT_NODE_USER", NodeUser),
+ UsernameValidation)
+
nodeCert, err := GenerateServerCert(caCert, caPrivateKey,
nodeKey.Public(), lifetime, nodeUser, hosts)
if err != nil {
@@ -376,7 +379,7 @@ func CreateClientPair(
keySize int,
lifetime time.Duration,
overwrite bool,
- user string,
+ user SQLUsername,
wantPKCS8Key bool,
) error {
if len(caKeyPath) == 0 {
diff --git a/pkg/security/certs_test.go b/pkg/security/certs_test.go
index 2b1e27159676..56c3db922736 100644
--- a/pkg/security/certs_test.go
+++ b/pkg/security/certs_test.go
@@ -237,7 +237,7 @@ func generateBaseCerts(certsDir string) error {
if err := security.CreateClientPair(
certsDir, caKey,
- testKeySize, time.Hour*48, true, security.RootUser, false,
+ testKeySize, time.Hour*48, true, security.RootUserName(), false,
); err != nil {
return err
}
@@ -287,14 +287,14 @@ func generateSplitCACerts(certsDir string) error {
if err := security.CreateClientPair(
certsDir, filepath.Join(certsDir, security.EmbeddedClientCAKey),
- testKeySize, time.Hour*48, true, security.NodeUser, false,
+ testKeySize, time.Hour*48, true, security.NodeUserName(), false,
); err != nil {
return errors.Errorf("could not generate Client pair: %v", err)
}
if err := security.CreateClientPair(
certsDir, filepath.Join(certsDir, security.EmbeddedClientCAKey),
- testKeySize, time.Hour*48, true, security.RootUser, false,
+ testKeySize, time.Hour*48, true, security.RootUserName(), false,
); err != nil {
return errors.Errorf("could not generate Client pair: %v", err)
}
diff --git a/pkg/security/username.go b/pkg/security/username.go
new file mode 100644
index 000000000000..4470909b9e0f
--- /dev/null
+++ b/pkg/security/username.go
@@ -0,0 +1,247 @@
+// 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 security
+
+import (
+ "bytes"
+ "fmt"
+ "regexp"
+
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
+ "github.com/cockroachdb/errors"
+ "github.com/cockroachdb/redact"
+)
+
+// SQLUsername represents a username valid inside SQL.
+//
+// Note that SQL usernames are not just ASCII names: they can start
+// with digits or contain only digits; they can contain certain
+// punctuation, and they can contain non-ASCII unicode letters.
+// For example, "123.-456" is a valid username.
+// Therefore, care must be taken when assembling a string from a
+// username for use in other contexts, e.g. to generate filenames:
+// some escaping and/or quoting is likely necessary.
+//
+// Additionally, beware that usernames as manipulated client-side (in
+// client drivers, in CLI commands) may not be the same as
+// server-side; this is because usernames can be substituted during
+// authentication. Additional care must be taken when deriving
+// server-side strings in client code. It is always better to add an
+// API server-side to assemble the string safely on the client's
+// behalf.
+//
+// This datatype is more complex to a simple string so as to force
+// usages to clarify when it is converted to/from strings.
+// This complexity is necessary because in CockroachDB SQL, unlike in
+// PostgreSQL, SQL usernames are case-folded and NFC-normalized when a
+// user logs in, or when used as input to certain CLI commands or SQL
+// statements. Then, "inside" CockroachDB, username strings are
+// considered pre-normalized and can be used directly for comparisons,
+// lookup etc.
+//
+// - The constructor MakeSQLUsernameFromUserInput() creates
+// a username from "external input".
+//
+// - The constructor MakeSQLUsernameFromPreNormalizedString()
+// creates a username when the caller can guarantee that
+// the input is already pre-normalized.
+//
+// For convenience, the SQLIdentifier() method also represents a
+// username in the form suitable for input back by the SQL parser.
+//
+type SQLUsername struct {
+ u string
+}
+
+// NodeUser is used by nodes for intra-cluster traffic.
+const NodeUser = "node"
+
+// NodeUserName is the SQLUsername for NodeUser.
+func NodeUserName() SQLUsername { return SQLUsername{NodeUser} }
+
+// IsNodeUser is true iff the username designates the node user.
+func (s SQLUsername) IsNodeUser() bool { return s.u == NodeUser }
+
+// RootUser is the default cluster administrator.
+const RootUser = "root"
+
+// RootUserName is the SQLUsername for RootUser.
+func RootUserName() SQLUsername { return SQLUsername{RootUser} }
+
+// IsRootUser is true iff the username designates the root user.
+func (s SQLUsername) IsRootUser() bool { return s.u == RootUser }
+
+// AdminRole is the default (and non-droppable) role with superuser privileges.
+const AdminRole = "admin"
+
+// AdminRoleName is the SQLUsername for AdminRole.
+func AdminRoleName() SQLUsername { return SQLUsername{AdminRole} }
+
+// IsAdminRole is true iff the username designates the admin role.
+func (s SQLUsername) IsAdminRole() bool { return s.u == AdminRole }
+
+// PublicRole is the special "public" pseudo-role.
+// All users are implicit members of "public". The role cannot be created,
+// dropped, assigned to another role, and is generally not listed.
+// It can be granted privileges, implicitly granting them to all users (current and future).
+const PublicRole = "public"
+
+// PublicRoleName is the SQLUsername for PublicRole.
+func PublicRoleName() SQLUsername { return SQLUsername{PublicRole} }
+
+// IsPublicRole is true iff the username designates the public role.
+func (s SQLUsername) IsPublicRole() bool { return s.u == PublicRole }
+
+// Undefined is true iff the username is an empty string.
+func (s SQLUsername) Undefined() bool { return len(s.u) == 0 }
+
+// TestUser is used in tests.
+const TestUser = "testuser"
+
+// TestUserName is the SQLUsername for testuser.
+func TestUserName() SQLUsername { return SQLUsername{TestUser} }
+
+// MakeSQLUsernameFromUserInput normalizes a username string as
+// entered in an ambiguous context into a SQL username (performs case
+// folding and unicode normalization form C - NFC).
+// If the purpose if UsernameCreation, the structure of the username
+// is also checked. An error is returned if the validation fails.
+// If the purpose is UsernameValidation, no error is returned.
+func MakeSQLUsernameFromUserInput(u string, purpose UsernamePurpose) (res SQLUsername, err error) {
+ // Perform case folding and NFC normalization.
+ res.u = lexbase.NormalizeName(u)
+ if purpose == UsernameCreation {
+ err = res.ValidateForCreation()
+ }
+ return res, err
+}
+
+// UsernamePurpose indicates the purpose of the resulting
+// SQLUsername in MakeSQLUsernameFromUserInput.
+type UsernamePurpose bool
+
+const (
+ // UsernameCreation indicates that the SQLUsername is being
+ // input for the purpose of creating a user account.
+ // This causes MakeSQLUsernameFromUserInput to also enforce
+ // structural restrictions on the username: which characters
+ // are allowed and a maximum length.
+ UsernameCreation UsernamePurpose = false
+
+ // UsernameValidation indicates that the SQLUsername is
+ // being input for the purpose of looking up an existing
+ // user, or to compare with an existing username.
+ // This skips the stuctural restrictions imposed
+ // for the purpose UsernameCreation.
+ UsernameValidation UsernamePurpose = true
+)
+
+const usernameHelp = "Usernames are case insensitive, must start with a letter, " +
+ "digit or underscore, may contain letters, digits, dashes, periods, or underscores, and must not exceed 63 characters."
+
+const maxUsernameLengthForCreation = 63
+
+var validUsernameCreationRE = regexp.MustCompile(`^[\p{Ll}0-9_][---\p{Ll}0-9_.]*$`)
+
+// ValidateForCreation checks that a username matches the structural
+// restrictions for creation of a user account with that name.
+func (s SQLUsername) ValidateForCreation() error {
+ if len(s.u) == 0 {
+ return ErrUsernameEmpty
+ }
+ if len(s.u) > maxUsernameLengthForCreation {
+ return ErrUsernameTooLong
+ }
+ if !validUsernameCreationRE.MatchString(s.u) {
+ return ErrUsernameInvalid
+ }
+ return nil
+}
+
+// ErrUsernameTooLong indicates that a username string was too
+// long. It is returned by ValidateForCreation() or
+// MakeSQLUserFromUserInput() with purpose UsernameCreation.
+var ErrUsernameTooLong = errors.WithHint(errors.New("username is too long"), usernameHelp)
+
+// ErrUsernameInvalid indicates that a username string contained
+// invalid characters. It is returned by ValidateForCreation() or
+// MakeSQLUserFromUserInput() with purpose UsernameCreation.
+var ErrUsernameInvalid = errors.WithHint(errors.New("username is invalid"), usernameHelp)
+
+// ErrUsernameEmpty indicates that an empty string was used as
+// username. It is returned by ValidateForCreation() or
+// MakeSQLUserFromUserInput() with purpose UsernameCreation.
+var ErrUsernameEmpty = errors.WithHint(errors.New("username is empty"), usernameHelp)
+
+// ErrUsernameNotNormalized indicates that a username
+// was not pre-normalized during a conversion.
+var ErrUsernameNotNormalized = errors.WithHint(errors.New("username is not normalized"),
+ "The username should be converted to lowercase and unicode characters normalized to NFC.")
+
+// MakeSQLUsernameFromPreNormalizedString takes a string containing a
+// canonical username and converts it to a SQLUsername. The caller of
+// this promises that the argument is pre-normalized. This conversion
+// is cheap.
+// Note: avoid using this function when processing strings
+// in requests from external APIs.
+// See also: MakeSQLUsernameFromPreNormalizedStringChecked().
+func MakeSQLUsernameFromPreNormalizedString(u string) SQLUsername {
+ return SQLUsername{u: u}
+}
+
+// MakeSQLUsernameFromPreNormalizedStringChecked takes a string,
+// validates it is a prenormalized username, then converts it to
+// a SQLUsername.
+// See also: MakeSQLUsernameFromPreNormalizedString().
+func MakeSQLUsernameFromPreNormalizedStringChecked(u string) (SQLUsername, error) {
+ res := SQLUsername{u: lexbase.NormalizeName(u)}
+ if res.u != u {
+ return res, ErrUsernameNotNormalized
+ }
+ return res, nil
+}
+
+// Normalized returns the normalized username, suitable for equality
+// comparison and lookups. The username is unquoted.
+func (s SQLUsername) Normalized() string { return s.u }
+
+// SQLIdentifier returns the normalized username in a form
+// suitable for parsing as a SQL identifier.
+// The identifier is quoted if it contains special characters
+// or it is a reserved keyword.
+func (s SQLUsername) SQLIdentifier() string {
+ var buf bytes.Buffer
+ lexbase.EncodeRestrictedSQLIdent(&buf, s.u, lexbase.EncNoFlags)
+ return buf.String()
+}
+
+// Format implements the fmt.Formatter interface. It renders
+// the username in normalized form.
+func (s SQLUsername) Format(fs fmt.State, verb rune) {
+ _, f := redact.MakeFormat(fs, verb)
+ fmt.Fprintf(fs, f, s.u)
+}
+
+// LessThan is true iff the receiver sorts strictly before
+// the given argument. This can be used e.g. in sort.Sort().
+func (s SQLUsername) LessThan(u SQLUsername) bool {
+ return s.u < u.u
+}
+
+// SQLUsernameProto is the wire representation of a SQLUsername.
+type SQLUsernameProto string
+
+// Decode turns the proto representation of a username back into its
+// legitimate form.
+func (s SQLUsernameProto) Decode() SQLUsername { return SQLUsername{u: string(s)} }
+
+// EncodeProto turns a username into its proto representation.
+func (s SQLUsername) EncodeProto() SQLUsernameProto { return SQLUsernameProto(s.u) }
diff --git a/pkg/security/username_test.go b/pkg/security/username_test.go
new file mode 100644
index 000000000000..5e80646db909
--- /dev/null
+++ b/pkg/security/username_test.go
@@ -0,0 +1,56 @@
+// 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 security_test
+
+import (
+ "testing"
+
+ "github.com/cockroachdb/cockroach/pkg/security"
+ "github.com/cockroachdb/cockroach/pkg/testutils"
+)
+
+func TestUserName(t *testing.T) {
+ testCases := []struct {
+ username string
+ normalized string
+ err string
+ }{
+ {"Abc123", "abc123", ""},
+ {"0123121132", "0123121132", ""},
+ {"HeLlO", "hello", ""},
+ {"Ομηρος", "ομηρος", ""},
+ {"_HeLlO", "_hello", ""},
+ {"a-BC-d", "a-bc-d", ""},
+ {"A.Bcd", "a.bcd", ""},
+ {"WWW.BIGSITE.NET", "www.bigsite.net", ""},
+ {"", "", `username is empty`},
+ {"-ABC", "", `username is invalid`},
+ {".ABC", "", `username is invalid`},
+ {"*.wildcard", "", `username is invalid`},
+ {"foofoofoofoofoofoofoofoofoofoofoofoofoofoofoofoofoofoofoofoofoof", "", `username is too long`},
+ {"M", "m", ""},
+ {".", "", `username is invalid`},
+ }
+
+ for _, tc := range testCases {
+ username, err := security.MakeSQLUsernameFromUserInput(tc.username, security.UsernameCreation)
+ if !testutils.IsError(err, tc.err) {
+ t.Errorf("%q: expected %q, got %v", tc.username, tc.err, err)
+ continue
+ }
+ if err != nil {
+ continue
+ }
+ if username.Normalized() != tc.normalized {
+ t.Errorf("%q: expected %q, got %q", tc.username, tc.normalized, username)
+ }
+ }
+}
diff --git a/pkg/security/x509.go b/pkg/security/x509.go
index 7cb875b73087..d91b07dc6b57 100644
--- a/pkg/security/x509.go
+++ b/pkg/security/x509.go
@@ -122,11 +122,11 @@ func GenerateServerCert(
caPrivateKey crypto.PrivateKey,
nodePublicKey crypto.PublicKey,
lifetime time.Duration,
- user string,
+ user SQLUsername,
hosts []string,
) ([]byte, error) {
// Create template for user.
- template, err := newTemplate(user, lifetime)
+ template, err := newTemplate(user.Normalized(), lifetime)
if err != nil {
return nil, err
}
@@ -243,16 +243,16 @@ func GenerateClientCert(
caPrivateKey crypto.PrivateKey,
clientPublicKey crypto.PublicKey,
lifetime time.Duration,
- user string,
+ user SQLUsername,
) ([]byte, error) {
// TODO(marc): should we add extra checks?
- if len(user) == 0 {
+ if user.Undefined() {
return nil, errors.Errorf("user cannot be empty")
}
// Create template for user.
- template, err := newTemplate(user, lifetime)
+ template, err := newTemplate(user.Normalized(), lifetime)
if err != nil {
return nil, err
}
diff --git a/pkg/security/x509_test.go b/pkg/security/x509_test.go
index 1490e5489e2c..a03a2abefc02 100644
--- a/pkg/security/x509_test.go
+++ b/pkg/security/x509_test.go
@@ -62,7 +62,7 @@ func TestGenerateCertLifetime(t *testing.T) {
// Create a Node certificate expiring in 4 days. Fails on shorter CA lifetime.
nodeDuration := time.Hour * 96
_, err = security.GenerateServerCert(caCert, testKey,
- testKey.Public(), nodeDuration, security.NodeUser, []string{"localhost"})
+ testKey.Public(), nodeDuration, security.NodeUserName(), []string{"localhost"})
if !testutils.IsError(err, "CA lifetime is .*, shorter than the requested .*") {
t.Fatal(err)
}
@@ -70,7 +70,7 @@ func TestGenerateCertLifetime(t *testing.T) {
// Try again, but expiring before the CA cert.
nodeDuration = time.Hour * 24
nodeBytes, err := security.GenerateServerCert(caCert, testKey,
- testKey.Public(), nodeDuration, security.NodeUser, []string{"localhost"})
+ testKey.Public(), nodeDuration, security.NodeUserName(), []string{"localhost"})
if err != nil {
t.Fatal(err)
}
@@ -86,14 +86,14 @@ func TestGenerateCertLifetime(t *testing.T) {
// Create a Client certificate expiring in 4 days. Should get reduced to the CA lifetime.
clientDuration := time.Hour * 96
- _, err = security.GenerateClientCert(caCert, testKey, testKey.Public(), clientDuration, "testuser")
+ _, err = security.GenerateClientCert(caCert, testKey, testKey.Public(), clientDuration, security.TestUserName())
if !testutils.IsError(err, "CA lifetime is .*, shorter than the requested .*") {
t.Fatal(err)
}
// Try again, but expiring before the CA cert.
clientDuration = time.Hour * 24
- clientBytes, err := security.GenerateClientCert(caCert, testKey, testKey.Public(), clientDuration, "testuser")
+ clientBytes, err := security.GenerateClientCert(caCert, testKey, testKey.Public(), clientDuration, security.TestUserName())
if err != nil {
t.Fatal(err)
}
diff --git a/pkg/server/admin.go b/pkg/server/admin.go
index 8a7f2716dc99..668724b7f0a4 100644
--- a/pkg/server/admin.go
+++ b/pkg/server/admin.go
@@ -1122,7 +1122,7 @@ func (s *adminServer) RangeLog(
// getUIData returns the values and timestamps for the given UI keys. Keys
// that are not found will not be returned.
func (s *adminServer) getUIData(
- ctx context.Context, userName string, keys []string,
+ ctx context.Context, userName security.SQLUsername, keys []string,
) (*serverpb.GetUIDataResponse, error) {
if len(keys) == 0 {
return &serverpb.GetUIDataResponse{}, nil
@@ -1143,7 +1143,7 @@ func (s *adminServer) getUIData(
}
rows, err := s.server.sqlServer.internalExecutor.QueryEx(
ctx, "admin-getUIData", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
query.String(), query.QueryArguments()...,
)
if err != nil {
@@ -1181,8 +1181,8 @@ func (s *adminServer) getUIData(
// system.ui.
// The username is combined to ensure that different users
// can use different customizations.
-func makeUIKey(username, key string) string {
- return username + "$" + key
+func makeUIKey(username security.SQLUsername, key string) string {
+ return username.Normalized() + "$" + key
}
// splitUIKey is the inverse of makeUIKey.
@@ -1215,7 +1215,7 @@ func (s *adminServer) SetUIData(
rowsAffected, err := s.server.sqlServer.internalExecutor.ExecEx(
ctx, "admin-set-ui-data", nil, /* txn */
sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
},
query, makeUIKey(userName, key), val)
if err != nil {
@@ -1528,7 +1528,7 @@ func (s *adminServer) Locations(
q.Append(`SELECT "localityKey", "localityValue", latitude, longitude FROM system.locations`)
rows, cols, err := s.server.sqlServer.internalExecutor.QueryWithCols(
ctx, "admin-locations", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
q.String(),
)
if err != nil {
@@ -2303,7 +2303,7 @@ func (rs resultScanner) Scan(row tree.Datums, colName string, dst interface{}) e
// queryZone retrieves the specific ZoneConfig associated with the supplied ID,
// if it exists.
func (s *adminServer) queryZone(
- ctx context.Context, userName string, id descpb.ID,
+ ctx context.Context, userName security.SQLUsername, id descpb.ID,
) (zonepb.ZoneConfig, bool, error) {
const query = `SELECT crdb_internal.get_zone_config($1)`
rows, cols, err := s.server.sqlServer.internalExecutor.QueryWithCols(
@@ -2346,7 +2346,7 @@ func (s *adminServer) queryZone(
// queryDescriptorIDPath(), for a ZoneConfig. It returns the most specific
// ZoneConfig specified for the object IDs in the path.
func (s *adminServer) queryZonePath(
- ctx context.Context, userName string, path []descpb.ID,
+ ctx context.Context, userName security.SQLUsername, path []descpb.ID,
) (descpb.ID, zonepb.ZoneConfig, bool, error) {
for i := len(path) - 1; i >= 0; i-- {
zone, zoneExists, err := s.queryZone(ctx, userName, path[i])
@@ -2359,7 +2359,7 @@ func (s *adminServer) queryZonePath(
// queryDatabaseID queries for the ID of the database with the given name.
func (s *adminServer) queryDatabaseID(
- ctx context.Context, userName string, name string,
+ ctx context.Context, userName security.SQLUsername, name string,
) (descpb.ID, error) {
const query = `SELECT crdb_internal.get_database_id($1)`
rows, cols, err := s.server.sqlServer.internalExecutor.QueryWithCols(
@@ -2394,7 +2394,7 @@ func (s *adminServer) queryDatabaseID(
// queryTableID queries for the ID of the table with the given name in the
// database with the given name. The table name may contain a schema qualifier.
func (s *adminServer) queryTableID(
- ctx context.Context, username string, database string, tableName string,
+ ctx context.Context, username security.SQLUsername, database string, tableName string,
) (descpb.ID, error) {
row, err := s.server.sqlServer.internalExecutor.QueryRowEx(
ctx, "admin-resolve-name", nil,
@@ -2428,31 +2428,33 @@ type adminPrivilegeChecker struct {
ie *sql.InternalExecutor
}
-func (c *adminPrivilegeChecker) requireAdminUser(ctx context.Context) (userName string, err error) {
+func (c *adminPrivilegeChecker) requireAdminUser(
+ ctx context.Context,
+) (userName security.SQLUsername, err error) {
userName, isAdmin, err := c.getUserAndRole(ctx)
if err != nil {
- return "", err
+ return userName, err
}
if !isAdmin {
- return "", errRequiresAdmin
+ return userName, errRequiresAdmin
}
return userName, nil
}
func (c *adminPrivilegeChecker) requireViewActivityPermission(
ctx context.Context,
-) (userName string, err error) {
+) (userName security.SQLUsername, err error) {
userName, isAdmin, err := c.getUserAndRole(ctx)
if err != nil {
- return "", err
+ return userName, err
}
if !isAdmin {
hasViewActivity, err := c.hasRoleOption(ctx, userName, roleoption.VIEWACTIVITY)
if err != nil {
- return "", err
+ return userName, err
}
if !hasViewActivity {
- return "", status.Errorf(
+ return userName, status.Errorf(
codes.PermissionDenied, "this operation requires the %s role option",
roleoption.VIEWACTIVITY)
}
@@ -2462,17 +2464,19 @@ func (c *adminPrivilegeChecker) requireViewActivityPermission(
func (c *adminPrivilegeChecker) getUserAndRole(
ctx context.Context,
-) (userName string, isAdmin bool, err error) {
+) (userName security.SQLUsername, isAdmin bool, err error) {
userName, err = userFromContext(ctx)
if err != nil {
- return "", false, err
+ return userName, false, err
}
isAdmin, err = c.hasAdminRole(ctx, userName)
return userName, isAdmin, err
}
-func (c *adminPrivilegeChecker) hasAdminRole(ctx context.Context, user string) (bool, error) {
- if user == security.RootUser {
+func (c *adminPrivilegeChecker) hasAdminRole(
+ ctx context.Context, user security.SQLUsername,
+) (bool, error) {
+ if user.IsRootUser() {
// Shortcut.
return true, nil
}
@@ -2497,9 +2501,9 @@ func (c *adminPrivilegeChecker) hasAdminRole(ctx context.Context, user string) (
}
func (c *adminPrivilegeChecker) hasRoleOption(
- ctx context.Context, user string, roleOption roleoption.Option,
+ ctx context.Context, user security.SQLUsername, roleOption roleoption.Option,
) (bool, error) {
- if user == security.RootUser {
+ if user.IsRootUser() {
// Shortcut.
return true, nil
}
diff --git a/pkg/server/admin_test.go b/pkg/server/admin_test.go
index 6fe67c3d4b42..b1f97245eef5 100644
--- a/pkg/server/admin_test.go
+++ b/pkg/server/admin_test.go
@@ -335,7 +335,7 @@ func TestAdminAPIDatabases(t *testing.T) {
"GRANT %s ON DATABASE %s TO %s",
strings.Join(privileges, ", "),
testdb,
- authenticatedUserNameNoAdmin,
+ authenticatedUserNameNoAdmin().SQLIdentifier(),
)
if _, err := db.Exec(query); err != nil {
t.Fatal(err)
@@ -389,7 +389,7 @@ func TestAdminAPIDatabases(t *testing.T) {
userGrants := make(map[string][]string)
for _, grant := range details.Grants {
switch grant.User {
- case security.AdminRole, security.RootUser, authenticatedUserNameNoAdmin:
+ case security.AdminRole, security.RootUser, authenticatedUserNoAdmin:
userGrants[grant.User] = append(userGrants[grant.User], grant.Privileges...)
default:
t.Fatalf("unknown grant to user %s", grant.User)
@@ -405,7 +405,7 @@ func TestAdminAPIDatabases(t *testing.T) {
if !reflect.DeepEqual(p, []string{"ALL"}) {
t.Fatalf("privileges %v != expected %v", p, privileges)
}
- case authenticatedUserNameNoAdmin:
+ case authenticatedUserNoAdmin:
sort.Strings(p)
if !reflect.DeepEqual(p, privileges) {
t.Fatalf("privileges %v != expected %v", p, privileges)
@@ -416,7 +416,7 @@ func TestAdminAPIDatabases(t *testing.T) {
}
// Verify Descriptor ID.
- databaseID, err := ts.admin.queryDatabaseID(ctx, security.RootUser, testdb)
+ databaseID, err := ts.admin.queryDatabaseID(ctx, security.RootUserName(), testdb)
if err != nil {
t.Fatal(err)
}
@@ -767,7 +767,7 @@ func TestAdminAPITableDetails(t *testing.T) {
}
// Verify Descriptor ID.
- tableID, err := ts.admin.queryTableID(ctx, security.RootUser, tc.dbName, tc.tblName)
+ tableID, err := ts.admin.queryTableID(ctx, security.RootUserName(), tc.dbName, tc.tblName)
if err != nil {
t.Fatal(err)
}
@@ -857,11 +857,11 @@ func TestAdminAPIZoneDetails(t *testing.T) {
verifyDbZone(s.(*TestServer).Cfg.DefaultZoneConfig, serverpb.ZoneConfigurationLevel_CLUSTER)
verifyTblZone(s.(*TestServer).Cfg.DefaultZoneConfig, serverpb.ZoneConfigurationLevel_CLUSTER)
- databaseID, err := ts.admin.queryDatabaseID(ctx, security.RootUser, "test")
+ databaseID, err := ts.admin.queryDatabaseID(ctx, security.RootUserName(), "test")
if err != nil {
t.Fatal(err)
}
- tableID, err := ts.admin.queryTableID(ctx, security.RootUser, "test", "tbl")
+ tableID, err := ts.admin.queryTableID(ctx, security.RootUserName(), "test", "tbl")
if err != nil {
t.Fatal(err)
}
@@ -1422,16 +1422,16 @@ func TestAdminAPIJobs(t *testing.T) {
status jobs.Status
details jobspb.Details
progress jobspb.ProgressDetails
- username string
+ username security.SQLUsername
}{
- {1, jobs.StatusRunning, jobspb.RestoreDetails{}, jobspb.RestoreProgress{}, security.RootUser},
- {2, jobs.StatusRunning, jobspb.BackupDetails{}, jobspb.BackupProgress{}, security.RootUser},
- {3, jobs.StatusSucceeded, jobspb.BackupDetails{}, jobspb.BackupProgress{}, security.RootUser},
- {4, jobs.StatusRunning, jobspb.ChangefeedDetails{}, jobspb.ChangefeedProgress{}, security.RootUser},
- {5, jobs.StatusSucceeded, jobspb.BackupDetails{}, jobspb.BackupProgress{}, authenticatedUserNameNoAdmin},
+ {1, jobs.StatusRunning, jobspb.RestoreDetails{}, jobspb.RestoreProgress{}, security.RootUserName()},
+ {2, jobs.StatusRunning, jobspb.BackupDetails{}, jobspb.BackupProgress{}, security.RootUserName()},
+ {3, jobs.StatusSucceeded, jobspb.BackupDetails{}, jobspb.BackupProgress{}, security.RootUserName()},
+ {4, jobs.StatusRunning, jobspb.ChangefeedDetails{}, jobspb.ChangefeedProgress{}, security.RootUserName()},
+ {5, jobs.StatusSucceeded, jobspb.BackupDetails{}, jobspb.BackupProgress{}, authenticatedUserNameNoAdmin()},
}
for _, job := range testJobs {
- payload := jobspb.Payload{Username: job.username, Details: jobspb.WrapPayloadDetails(job.details)}
+ payload := jobspb.Payload{UsernameProto: job.username.EncodeProto(), Details: jobspb.WrapPayloadDetails(job.details)}
payloadBytes, err := protoutil.Marshal(&payload)
if err != nil {
t.Fatal(err)
diff --git a/pkg/server/authentication.go b/pkg/server/authentication.go
index 471faee13e24..9d7053cb0eeb 100644
--- a/pkg/server/authentication.go
+++ b/pkg/server/authentication.go
@@ -132,8 +132,7 @@ func (s *authenticationServer) RegisterGateway(
func (s *authenticationServer) UserLogin(
ctx context.Context, req *serverpb.UserLoginRequest,
) (*serverpb.UserLoginResponse, error) {
- username := req.Username
- if username == "" {
+ if req.Username == "" {
return nil, status.Errorf(
codes.Unauthenticated,
"no username was provided",
@@ -145,7 +144,7 @@ func (s *authenticationServer) UserLogin(
// here, so that the normalized username is retained in the session
// table: the APIs extract the username from the session table
// without further normalization.
- username = tree.Name(username).Normalize()
+ username, _ := security.MakeSQLUsernameFromUserInput(req.Username, security.UsernameValidation)
// Verify the provided username/password pair.
verified, expired, err := s.verifyPassword(ctx, username, req.Password)
@@ -197,14 +196,14 @@ func (s *authenticationServer) ValidateOIDCState(
// The session's ID and secret are returned to the caller as an HTTP cookie,
// added via a "Set-Cookie" header.
func (s *authenticationServer) UserLoginFromSSO(
- ctx context.Context, username string,
+ ctx context.Context, reqUsername string,
) (*http.Cookie, error) {
// In CockroachDB SQL, unlike in PostgreSQL, usernames are
// case-insensitive. Therefore we need to normalize the username
// here, so that the normalized username is retained in the session
// table: the APIs extract the username from the session table
// without further normalization.
- username = tree.Name(username).Normalize()
+ username, _ := security.MakeSQLUsernameFromUserInput(reqUsername, security.UsernameValidation)
exists, canLogin, _, _, err := sql.GetUserHashedPassword(
ctx, s.server.sqlServer.execCfg.InternalExecutor, username,
@@ -225,7 +224,7 @@ func (s *authenticationServer) UserLoginFromSSO(
//
// The caller is responsible to ensure the username has been normalized already.
func (s *authenticationServer) createSessionFor(
- ctx context.Context, username string,
+ ctx context.Context, username security.SQLUsername,
) (*http.Cookie, error) {
// Create a new database session, generating an ID and secret key.
id, secret, err := s.newAuthSession(ctx, username)
@@ -268,7 +267,7 @@ func (s *authenticationServer) UserLogout(
ctx,
"revoke-auth-session",
nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
`UPDATE system.web_sessions SET "revokedAt" = now() WHERE id = $1`,
sessionID,
); err != nil {
@@ -318,7 +317,7 @@ WHERE id = $1`
ctx,
"lookup-auth-session",
nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
sessionQuery, cookie.ID)
if row == nil || err != nil {
return false, "", err
@@ -363,7 +362,7 @@ WHERE id = $1`
// The caller is responsible for ensuring that the username is normalized.
// (CockroachDB has case-insensitive usernames, unlike PostgreSQL.)
func (s *authenticationServer) verifyPassword(
- ctx context.Context, username string, password string,
+ ctx context.Context, username security.SQLUsername, password string,
) (valid bool, expired bool, err error) {
exists, canLogin, pwRetrieveFn, validUntilFn, err := sql.GetUserHashedPassword(
ctx, s.server.sqlServer.execCfg.InternalExecutor, username,
@@ -410,7 +409,7 @@ func CreateAuthSecret() (secret, hashedSecret []byte, err error) {
//
// The caller is responsible to ensure the username has been normalized already.
func (s *authenticationServer) newAuthSession(
- ctx context.Context, username string,
+ ctx context.Context, username security.SQLUsername,
) (int64, []byte, error) {
secret, hashedSecret, err := CreateAuthSecret()
if err != nil {
@@ -430,10 +429,10 @@ RETURNING id
ctx,
"create-auth-session",
nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
insertSessionStmt,
hashedSecret,
- username,
+ username.Normalized(),
expiration,
)
if err != nil {
diff --git a/pkg/server/authentication_test.go b/pkg/server/authentication_test.go
index bd3cb2676ede..00fd83f3de52 100644
--- a/pkg/server/authentication_test.go
+++ b/pkg/server/authentication_test.go
@@ -100,15 +100,15 @@ func TestSSLEnforcement(t *testing.T) {
}
// HTTPS with client certs for security.RootUser.
- rootCertsContext := newRPCContext(testutils.NewTestBaseContext(security.RootUser))
+ rootCertsContext := newRPCContext(testutils.NewTestBaseContext(security.RootUserName()))
// HTTPS with client certs for security.NodeUser.
nodeCertsContext := newRPCContext(testutils.NewNodeTestBaseContext())
// HTTPS with client certs for TestUser.
- testCertsContext := newRPCContext(testutils.NewTestBaseContext(TestUser))
+ testCertsContext := newRPCContext(testutils.NewTestBaseContext(security.TestUserName()))
// HTTPS without client certs. The user does not matter.
noCertsContext := insecureCtx{}
// Plain http.
- plainHTTPCfg := testutils.NewTestBaseContext(TestUser)
+ plainHTTPCfg := testutils.NewTestBaseContext(security.TestUserName())
plainHTTPCfg.Insecure = true
insecureContext := newRPCContext(plainHTTPCfg)
@@ -231,9 +231,10 @@ func TestVerifyPassword(t *testing.T) {
{"timelord", "12345", "", "VALID UNTIL $1",
[]interface{}{timeutil.Now().Add(59 * time.Minute).In(shanghaiLoc)}},
} {
+ username := security.MakeSQLUsernameFromPreNormalizedString(user.username)
cmd := fmt.Sprintf(
"CREATE USER %s WITH PASSWORD '%s' %s %s",
- user.username, user.password, user.loginFlag, user.validUntilClause)
+ username.SQLIdentifier(), user.password, user.loginFlag, user.validUntilClause)
if _, err := db.Exec(cmd, user.qargs...); err != nil {
t.Fatalf("failed to create user: %s", err)
@@ -270,7 +271,8 @@ func TestVerifyPassword(t *testing.T) {
{"cthon98", "12345", true, ""},
} {
t.Run("", func(t *testing.T) {
- valid, expired, err := ts.authentication.verifyPassword(context.Background(), tc.username, tc.password)
+ username := security.MakeSQLUsernameFromPreNormalizedString(tc.username)
+ valid, expired, err := ts.authentication.verifyPassword(context.Background(), username, tc.password)
if err != nil {
t.Errorf(
"credentials %s/%s failed with error %s, wanted no error",
@@ -299,7 +301,7 @@ func TestCreateSession(t *testing.T) {
defer s.Stopper().Stop(context.Background())
ts := s.(*TestServer)
- username := "testuser"
+ username := security.TestUserName()
// Create an authentication, noting the time before and after creation. This
// lets us ensure that the timestamps created are accurate.
@@ -347,7 +349,7 @@ WHERE id = $1`
}
// Username.
- if a, e := sessUsername, username; a != e {
+ if a, e := sessUsername, username.Normalized(); a != e {
t.Fatalf("session username got %s, wanted %s", a, e)
}
@@ -391,7 +393,7 @@ func TestVerifySession(t *testing.T) {
defer s.Stopper().Stop(context.Background())
ts := s.(*TestServer)
- sessionUsername := "testuser"
+ sessionUsername := security.TestUserName()
id, origSecret, err := ts.authentication.newAuthSession(context.Background(), sessionUsername)
if err != nil {
t.Fatal(err)
@@ -454,7 +456,7 @@ func TestVerifySession(t *testing.T) {
if a, e := valid, tc.shouldVerify; a != e {
t.Fatalf("cookie %v verification = %t, wanted %t", tc.cookie, a, e)
}
- if a, e := username, sessionUsername; tc.shouldVerify && a != e {
+ if a, e := username, sessionUsername.Normalized(); tc.shouldVerify && a != e {
t.Fatalf("cookie %v verification returned username %s, wanted %s", tc.cookie, a, e)
}
})
@@ -560,7 +562,7 @@ func TestLogout(t *testing.T) {
ts := s.(*TestServer)
// Log in.
- authHTTPClient, cookie, err := ts.getAuthenticatedHTTPClientAndCookie(authenticatedUserName, true)
+ authHTTPClient, cookie, err := ts.getAuthenticatedHTTPClientAndCookie(authenticatedUserName(), true)
if err != nil {
t.Fatal("error opening HTTP client", err)
}
@@ -803,7 +805,7 @@ func TestGRPCAuthentication(t *testing.T) {
if err != nil {
t.Fatal(err)
}
- tlsConfig, err := certManager.GetClientTLSConfig("testuser")
+ tlsConfig, err := certManager.GetClientTLSConfig(security.TestUserName())
if err != nil {
t.Fatal(err)
}
diff --git a/pkg/server/server.go b/pkg/server/server.go
index 436891d6f3ee..ad76a1b11da3 100644
--- a/pkg/server/server.go
+++ b/pkg/server/server.go
@@ -47,6 +47,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/debug"
"github.com/cockroachdb/cockroach/pkg/server/goroutinedumper"
"github.com/cockroachdb/cockroach/pkg/server/heapprofiler"
@@ -202,7 +203,7 @@ func (e *externalStorageBuilder) makeExternalStorage(
}
func (e *externalStorageBuilder) makeExternalStorageFromURI(
- ctx context.Context, uri string, user string,
+ ctx context.Context, uri string, user security.SQLUsername,
) (cloud.ExternalStorage, error) {
if !e.initCalled {
return nil, errors.New("cannot create external storage before init")
@@ -467,7 +468,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) {
return externalStorageBuilder.makeExternalStorage(ctx, dest)
}
externalStorageFromURI := func(ctx context.Context, uri string,
- user string) (cloud.ExternalStorage, error) {
+ user security.SQLUsername) (cloud.ExternalStorage, error) {
return externalStorageBuilder.makeExternalStorageFromURI(ctx, uri, user)
}
diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go
index 08c7954a59f4..6a9d39378cab 100644
--- a/pkg/server/server_sql.go
+++ b/pkg/server/server_sql.go
@@ -36,6 +36,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/rpc"
"github.com/cockroachdb/cockroach/pkg/rpc/nodedialer"
"github.com/cockroachdb/cockroach/pkg/scheduledjobs"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/status"
"github.com/cockroachdb/cockroach/pkg/sql"
@@ -244,7 +245,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
cfg.sqlLivenessProvider,
cfg.Settings,
cfg.HistogramWindowInterval(),
- func(opName, user string) (interface{}, func()) {
+ func(opName string, user security.SQLUsername) (interface{}, func()) {
// This is a hack to get around a Go package dependency cycle. See comment
// in sql/jobs/registry.go on planHookMaker.
return sql.NewInternalPlanner(opName, nil, user, &sql.MemoryMetrics{}, execCfg)
@@ -805,7 +806,7 @@ func (s *sqlServer) preStart(
InternalExecutor: s.internalExecutor,
DB: s.execCfg.DB,
TestingKnobs: knobs.JobsTestingKnobs,
- PlanHookMaker: func(opName string, txn *kv.Txn, user string) (interface{}, func()) {
+ PlanHookMaker: func(opName string, txn *kv.Txn, user security.SQLUsername) (interface{}, func()) {
// This is a hack to get around a Go package dependency cycle. See comment
// in sql/jobs/registry.go on planHookMaker.
return sql.NewInternalPlanner(opName, txn, user, &sql.MemoryMetrics{}, s.execCfg)
diff --git a/pkg/server/server_systemlog_gc.go b/pkg/server/server_systemlog_gc.go
index 5eee701a9415..19b9f3301e60 100644
--- a/pkg/server/server_systemlog_gc.go
+++ b/pkg/server/server_systemlog_gc.go
@@ -96,7 +96,7 @@ func (s *Server) gcSystemLog(
ctx,
table+"-gc",
txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
deleteStmt,
timestampLowerBound,
timestampUpperBound,
diff --git a/pkg/server/server_test.go b/pkg/server/server_test.go
index 9ccd623a3472..2ac8b5a654ba 100644
--- a/pkg/server/server_test.go
+++ b/pkg/server/server_test.go
@@ -566,7 +566,7 @@ func TestSystemConfigGossip(t *testing.T) {
key := catalogkeys.MakeDescMetadataKey(keys.SystemSQLCodec, keys.MaxReservedDescID)
valAt := func(i int) *descpb.Descriptor {
return dbdesc.NewInitial(
- descpb.ID(i), "foo", security.AdminRole,
+ descpb.ID(i), "foo", security.AdminRoleName(),
).DescriptorProto()
}
diff --git a/pkg/server/server_update.go b/pkg/server/server_update.go
index 963a84204ccb..17a6c297554f 100644
--- a/pkg/server/server_update.go
+++ b/pkg/server/server_update.go
@@ -69,7 +69,7 @@ func (s *Server) startAttemptUpgrade(ctx context.Context) {
for ur := retry.StartWithCtx(ctx, upgradeRetryOpts); ur.Next(); {
if _, err := s.sqlServer.internalExecutor.ExecEx(
ctx, "set-version", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SET CLUSTER SETTING version = crdb_internal.node_executable_version();",
); err != nil {
log.Infof(ctx, "error when finalizing cluster version upgrade: %s", err)
@@ -142,7 +142,7 @@ func (s *Server) upgradeStatus(ctx context.Context) (bool, error) {
// SET CLUSTER SETTING.
datums, err := s.sqlServer.internalExecutor.QueryEx(
ctx, "read-downgrade", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SELECT value FROM system.settings WHERE name = 'cluster.preserve_downgrade_option';",
)
if err != nil {
@@ -167,7 +167,7 @@ func (s *Server) upgradeStatus(ctx context.Context) (bool, error) {
func (s *Server) clusterVersion(ctx context.Context) (string, error) {
datums, err := s.sqlServer.internalExecutor.QueryEx(
ctx, "show-version", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SHOW CLUSTER SETTING version;",
)
if err != nil {
diff --git a/pkg/server/serverpb/status.pb.go b/pkg/server/serverpb/status.pb.go
index 4288d6ed6215..b0855ac92e14 100644
--- a/pkg/server/serverpb/status.pb.go
+++ b/pkg/server/serverpb/status.pb.go
@@ -65,7 +65,7 @@ func (x StacksType) String() string {
return proto.EnumName(StacksType_name, int32(x))
}
func (StacksType) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{0}
}
// Represents the type of file.
@@ -92,7 +92,7 @@ func (x FileType) String() string {
return proto.EnumName(FileType_name, int32(x))
}
func (FileType) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{1}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{1}
}
// We use an enum to allow reporting of client certs and potential others (eg:
@@ -129,7 +129,7 @@ func (x CertificateDetails_CertificateType) String() string {
return proto.EnumName(CertificateDetails_CertificateType_name, int32(x))
}
func (CertificateDetails_CertificateType) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{1, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{1, 0}
}
type ProfileRequest_Type int32
@@ -152,7 +152,7 @@ func (x ProfileRequest_Type) String() string {
return proto.EnumName(ProfileRequest_Type_name, int32(x))
}
func (ProfileRequest_Type) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{37, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{37, 0}
}
// Enum for phase of execution.
@@ -176,7 +176,7 @@ func (x ActiveQuery_Phase) String() string {
return proto.EnumName(ActiveQuery_Phase_name, int32(x))
}
func (ActiveQuery_Phase) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{45, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{45, 0}
}
type CertificatesRequest struct {
@@ -189,7 +189,7 @@ func (m *CertificatesRequest) Reset() { *m = CertificatesRequest{} }
func (m *CertificatesRequest) String() string { return proto.CompactTextString(m) }
func (*CertificatesRequest) ProtoMessage() {}
func (*CertificatesRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{0}
}
func (m *CertificatesRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -228,7 +228,7 @@ func (m *CertificateDetails) Reset() { *m = CertificateDetails{} }
func (m *CertificateDetails) String() string { return proto.CompactTextString(m) }
func (*CertificateDetails) ProtoMessage() {}
func (*CertificateDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{1}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{1}
}
func (m *CertificateDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -269,7 +269,7 @@ func (m *CertificateDetails_Fields) Reset() { *m = CertificateDetails_Fi
func (m *CertificateDetails_Fields) String() string { return proto.CompactTextString(m) }
func (*CertificateDetails_Fields) ProtoMessage() {}
func (*CertificateDetails_Fields) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{1, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{1, 0}
}
func (m *CertificateDetails_Fields) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -302,7 +302,7 @@ func (m *CertificatesResponse) Reset() { *m = CertificatesResponse{} }
func (m *CertificatesResponse) String() string { return proto.CompactTextString(m) }
func (*CertificatesResponse) ProtoMessage() {}
func (*CertificatesResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{2}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{2}
}
func (m *CertificatesResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -339,7 +339,7 @@ func (m *DetailsRequest) Reset() { *m = DetailsRequest{} }
func (m *DetailsRequest) String() string { return proto.CompactTextString(m) }
func (*DetailsRequest) ProtoMessage() {}
func (*DetailsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{3}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{3}
}
func (m *DetailsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -376,7 +376,7 @@ func (m *SystemInfo) Reset() { *m = SystemInfo{} }
func (m *SystemInfo) String() string { return proto.CompactTextString(m) }
func (*SystemInfo) ProtoMessage() {}
func (*SystemInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{4}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{4}
}
func (m *SystemInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -413,7 +413,7 @@ func (m *DetailsResponse) Reset() { *m = DetailsResponse{} }
func (m *DetailsResponse) String() string { return proto.CompactTextString(m) }
func (*DetailsResponse) ProtoMessage() {}
func (*DetailsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{5}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{5}
}
func (m *DetailsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -445,7 +445,7 @@ func (m *NodesRequest) Reset() { *m = NodesRequest{} }
func (m *NodesRequest) String() string { return proto.CompactTextString(m) }
func (*NodesRequest) ProtoMessage() {}
func (*NodesRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{6}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{6}
}
func (m *NodesRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -479,7 +479,7 @@ func (m *NodesResponse) Reset() { *m = NodesResponse{} }
func (m *NodesResponse) String() string { return proto.CompactTextString(m) }
func (*NodesResponse) ProtoMessage() {}
func (*NodesResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{7}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{7}
}
func (m *NodesResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -514,7 +514,7 @@ func (m *NodeRequest) Reset() { *m = NodeRequest{} }
func (m *NodeRequest) String() string { return proto.CompactTextString(m) }
func (*NodeRequest) ProtoMessage() {}
func (*NodeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{8}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{8}
}
func (m *NodeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -559,7 +559,7 @@ func (m *RaftState) Reset() { *m = RaftState{} }
func (m *RaftState) String() string { return proto.CompactTextString(m) }
func (*RaftState) ProtoMessage() {}
func (*RaftState) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{9}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{9}
}
func (m *RaftState) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -596,7 +596,7 @@ func (m *RaftState_Progress) Reset() { *m = RaftState_Progress{} }
func (m *RaftState_Progress) String() string { return proto.CompactTextString(m) }
func (*RaftState_Progress) ProtoMessage() {}
func (*RaftState_Progress) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{9, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{9, 0}
}
func (m *RaftState_Progress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -641,7 +641,7 @@ func (m *RangeProblems) Reset() { *m = RangeProblems{} }
func (m *RangeProblems) String() string { return proto.CompactTextString(m) }
func (*RangeProblems) ProtoMessage() {}
func (*RangeProblems) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{10}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{10}
}
func (m *RangeProblems) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -677,7 +677,7 @@ func (m *RangeStatistics) Reset() { *m = RangeStatistics{} }
func (m *RangeStatistics) String() string { return proto.CompactTextString(m) }
func (*RangeStatistics) ProtoMessage() {}
func (*RangeStatistics) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{11}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{11}
}
func (m *RangeStatistics) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -711,7 +711,7 @@ func (m *PrettySpan) Reset() { *m = PrettySpan{} }
func (m *PrettySpan) String() string { return proto.CompactTextString(m) }
func (*PrettySpan) ProtoMessage() {}
func (*PrettySpan) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{12}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{12}
}
func (m *PrettySpan) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -757,7 +757,7 @@ func (m *RangeInfo) Reset() { *m = RangeInfo{} }
func (m *RangeInfo) String() string { return proto.CompactTextString(m) }
func (*RangeInfo) ProtoMessage() {}
func (*RangeInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{13}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{13}
}
func (m *RangeInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -793,7 +793,7 @@ func (m *RangesRequest) Reset() { *m = RangesRequest{} }
func (m *RangesRequest) String() string { return proto.CompactTextString(m) }
func (*RangesRequest) ProtoMessage() {}
func (*RangesRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{14}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{14}
}
func (m *RangesRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -826,7 +826,7 @@ func (m *RangesResponse) Reset() { *m = RangesResponse{} }
func (m *RangesResponse) String() string { return proto.CompactTextString(m) }
func (*RangesResponse) ProtoMessage() {}
func (*RangesResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{15}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{15}
}
func (m *RangesResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -861,7 +861,7 @@ func (m *GossipRequest) Reset() { *m = GossipRequest{} }
func (m *GossipRequest) String() string { return proto.CompactTextString(m) }
func (*GossipRequest) ProtoMessage() {}
func (*GossipRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{16}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{16}
}
func (m *GossipRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -896,7 +896,7 @@ func (m *EngineStatsInfo) Reset() { *m = EngineStatsInfo{} }
func (m *EngineStatsInfo) String() string { return proto.CompactTextString(m) }
func (*EngineStatsInfo) ProtoMessage() {}
func (*EngineStatsInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{17}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{17}
}
func (m *EngineStatsInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -931,7 +931,7 @@ func (m *EngineStatsRequest) Reset() { *m = EngineStatsRequest{} }
func (m *EngineStatsRequest) String() string { return proto.CompactTextString(m) }
func (*EngineStatsRequest) ProtoMessage() {}
func (*EngineStatsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{18}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{18}
}
func (m *EngineStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -964,7 +964,7 @@ func (m *EngineStatsResponse) Reset() { *m = EngineStatsResponse{} }
func (m *EngineStatsResponse) String() string { return proto.CompactTextString(m) }
func (*EngineStatsResponse) ProtoMessage() {}
func (*EngineStatsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{19}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{19}
}
func (m *EngineStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -998,7 +998,7 @@ func (m *TraceEvent) Reset() { *m = TraceEvent{} }
func (m *TraceEvent) String() string { return proto.CompactTextString(m) }
func (*TraceEvent) ProtoMessage() {}
func (*TraceEvent) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{20}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{20}
}
func (m *TraceEvent) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1032,7 +1032,7 @@ func (m *AllocatorDryRun) Reset() { *m = AllocatorDryRun{} }
func (m *AllocatorDryRun) String() string { return proto.CompactTextString(m) }
func (*AllocatorDryRun) ProtoMessage() {}
func (*AllocatorDryRun) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{21}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{21}
}
func (m *AllocatorDryRun) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1065,7 +1065,7 @@ func (m *AllocatorRangeRequest) Reset() { *m = AllocatorRangeRequest{} }
func (m *AllocatorRangeRequest) String() string { return proto.CompactTextString(m) }
func (*AllocatorRangeRequest) ProtoMessage() {}
func (*AllocatorRangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{22}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{22}
}
func (m *AllocatorRangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1101,7 +1101,7 @@ func (m *AllocatorRangeResponse) Reset() { *m = AllocatorRangeResponse{}
func (m *AllocatorRangeResponse) String() string { return proto.CompactTextString(m) }
func (*AllocatorRangeResponse) ProtoMessage() {}
func (*AllocatorRangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{23}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{23}
}
func (m *AllocatorRangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1135,7 +1135,7 @@ func (m *AllocatorRequest) Reset() { *m = AllocatorRequest{} }
func (m *AllocatorRequest) String() string { return proto.CompactTextString(m) }
func (*AllocatorRequest) ProtoMessage() {}
func (*AllocatorRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{24}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{24}
}
func (m *AllocatorRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1168,7 +1168,7 @@ func (m *AllocatorResponse) Reset() { *m = AllocatorResponse{} }
func (m *AllocatorResponse) String() string { return proto.CompactTextString(m) }
func (*AllocatorResponse) ProtoMessage() {}
func (*AllocatorResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{25}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{25}
}
func (m *AllocatorResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1201,7 +1201,7 @@ func (m *JSONResponse) Reset() { *m = JSONResponse{} }
func (m *JSONResponse) String() string { return proto.CompactTextString(m) }
func (*JSONResponse) ProtoMessage() {}
func (*JSONResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{26}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{26}
}
func (m *JSONResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1237,7 +1237,7 @@ func (m *ResponseError) Reset() { *m = ResponseError{} }
func (m *ResponseError) String() string { return proto.CompactTextString(m) }
func (*ResponseError) ProtoMessage() {}
func (*ResponseError) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{27}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{27}
}
func (m *ResponseError) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1289,7 +1289,7 @@ func (m *LogsRequest) Reset() { *m = LogsRequest{} }
func (m *LogsRequest) String() string { return proto.CompactTextString(m) }
func (*LogsRequest) ProtoMessage() {}
func (*LogsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{28}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{28}
}
func (m *LogsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1322,7 +1322,7 @@ func (m *LogEntriesResponse) Reset() { *m = LogEntriesResponse{} }
func (m *LogEntriesResponse) String() string { return proto.CompactTextString(m) }
func (*LogEntriesResponse) ProtoMessage() {}
func (*LogEntriesResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{29}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{29}
}
func (m *LogEntriesResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1357,7 +1357,7 @@ func (m *LogFilesListRequest) Reset() { *m = LogFilesListRequest{} }
func (m *LogFilesListRequest) String() string { return proto.CompactTextString(m) }
func (*LogFilesListRequest) ProtoMessage() {}
func (*LogFilesListRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{30}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{30}
}
func (m *LogFilesListRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1390,7 +1390,7 @@ func (m *LogFilesListResponse) Reset() { *m = LogFilesListResponse{} }
func (m *LogFilesListResponse) String() string { return proto.CompactTextString(m) }
func (*LogFilesListResponse) ProtoMessage() {}
func (*LogFilesListResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{31}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{31}
}
func (m *LogFilesListResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1438,7 +1438,7 @@ func (m *LogFileRequest) Reset() { *m = LogFileRequest{} }
func (m *LogFileRequest) String() string { return proto.CompactTextString(m) }
func (*LogFileRequest) ProtoMessage() {}
func (*LogFileRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{32}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{32}
}
func (m *LogFileRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1474,7 +1474,7 @@ func (m *StacksRequest) Reset() { *m = StacksRequest{} }
func (m *StacksRequest) String() string { return proto.CompactTextString(m) }
func (*StacksRequest) ProtoMessage() {}
func (*StacksRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{33}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{33}
}
func (m *StacksRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1510,7 +1510,7 @@ func (m *File) Reset() { *m = File{} }
func (m *File) String() string { return proto.CompactTextString(m) }
func (*File) ProtoMessage() {}
func (*File) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{34}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{34}
}
func (m *File) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1554,7 +1554,7 @@ func (m *GetFilesRequest) Reset() { *m = GetFilesRequest{} }
func (m *GetFilesRequest) String() string { return proto.CompactTextString(m) }
func (*GetFilesRequest) ProtoMessage() {}
func (*GetFilesRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{35}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{35}
}
func (m *GetFilesRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1587,7 +1587,7 @@ func (m *GetFilesResponse) Reset() { *m = GetFilesResponse{} }
func (m *GetFilesResponse) String() string { return proto.CompactTextString(m) }
func (*GetFilesResponse) ProtoMessage() {}
func (*GetFilesResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{36}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{36}
}
func (m *GetFilesResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1625,7 +1625,7 @@ func (m *ProfileRequest) Reset() { *m = ProfileRequest{} }
func (m *ProfileRequest) String() string { return proto.CompactTextString(m) }
func (*ProfileRequest) ProtoMessage() {}
func (*ProfileRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{37}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{37}
}
func (m *ProfileRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1660,7 +1660,7 @@ func (m *MetricsRequest) Reset() { *m = MetricsRequest{} }
func (m *MetricsRequest) String() string { return proto.CompactTextString(m) }
func (*MetricsRequest) ProtoMessage() {}
func (*MetricsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{38}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{38}
}
func (m *MetricsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1694,7 +1694,7 @@ func (m *RaftRangeNode) Reset() { *m = RaftRangeNode{} }
func (m *RaftRangeNode) String() string { return proto.CompactTextString(m) }
func (*RaftRangeNode) ProtoMessage() {}
func (*RaftRangeNode) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{39}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{39}
}
func (m *RaftRangeNode) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1727,7 +1727,7 @@ func (m *RaftRangeError) Reset() { *m = RaftRangeError{} }
func (m *RaftRangeError) String() string { return proto.CompactTextString(m) }
func (*RaftRangeError) ProtoMessage() {}
func (*RaftRangeError) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{40}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{40}
}
func (m *RaftRangeError) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1762,7 +1762,7 @@ func (m *RaftRangeStatus) Reset() { *m = RaftRangeStatus{} }
func (m *RaftRangeStatus) String() string { return proto.CompactTextString(m) }
func (*RaftRangeStatus) ProtoMessage() {}
func (*RaftRangeStatus) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{41}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{41}
}
func (m *RaftRangeStatus) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1795,7 +1795,7 @@ func (m *RaftDebugRequest) Reset() { *m = RaftDebugRequest{} }
func (m *RaftDebugRequest) String() string { return proto.CompactTextString(m) }
func (*RaftDebugRequest) ProtoMessage() {}
func (*RaftDebugRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{42}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{42}
}
func (m *RaftDebugRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1829,7 +1829,7 @@ func (m *RaftDebugResponse) Reset() { *m = RaftDebugResponse{} }
func (m *RaftDebugResponse) String() string { return proto.CompactTextString(m) }
func (*RaftDebugResponse) ProtoMessage() {}
func (*RaftDebugResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{43}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{43}
}
func (m *RaftDebugResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1887,7 +1887,7 @@ func (m *TxnInfo) Reset() { *m = TxnInfo{} }
func (m *TxnInfo) String() string { return proto.CompactTextString(m) }
func (*TxnInfo) ProtoMessage() {}
func (*TxnInfo) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{44}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{44}
}
func (m *TxnInfo) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1935,7 +1935,7 @@ func (m *ActiveQuery) Reset() { *m = ActiveQuery{} }
func (m *ActiveQuery) String() string { return proto.CompactTextString(m) }
func (*ActiveQuery) ProtoMessage() {}
func (*ActiveQuery) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{45}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{45}
}
func (m *ActiveQuery) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -1963,6 +1963,8 @@ var xxx_messageInfo_ActiveQuery proto.InternalMessageInfo
// Request object for ListSessions and ListLocalSessions.
type ListSessionsRequest struct {
// Username of the user making this request.
+ // The caller is responsible to normalize the username
+ // (= case fold and perform unicode NFC normalization).
Username string `protobuf:"bytes,1,opt,name=username,proto3" json:"username,omitempty"`
}
@@ -1970,7 +1972,7 @@ func (m *ListSessionsRequest) Reset() { *m = ListSessionsRequest{} }
func (m *ListSessionsRequest) String() string { return proto.CompactTextString(m) }
func (*ListSessionsRequest) ProtoMessage() {}
func (*ListSessionsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{46}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{46}
}
func (m *ListSessionsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2029,7 +2031,7 @@ func (m *Session) Reset() { *m = Session{} }
func (m *Session) String() string { return proto.CompactTextString(m) }
func (*Session) ProtoMessage() {}
func (*Session) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{47}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{47}
}
func (m *Session) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2066,7 +2068,7 @@ func (m *ListSessionsError) Reset() { *m = ListSessionsError{} }
func (m *ListSessionsError) String() string { return proto.CompactTextString(m) }
func (*ListSessionsError) ProtoMessage() {}
func (*ListSessionsError) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{48}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{48}
}
func (m *ListSessionsError) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2103,7 +2105,7 @@ func (m *ListSessionsResponse) Reset() { *m = ListSessionsResponse{} }
func (m *ListSessionsResponse) String() string { return proto.CompactTextString(m) }
func (*ListSessionsResponse) ProtoMessage() {}
func (*ListSessionsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{49}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{49}
}
func (m *ListSessionsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2142,6 +2144,7 @@ type CancelQueryRequest struct {
QueryID string `protobuf:"bytes,2,opt,name=query_id,json=queryId,proto3" json:"query_id,omitempty"`
// Username of the user making this cancellation request. This may be omitted
// if the user is the same as the one issuing the CancelQueryRequest.
+ // The caller is responsible for case-folding and NFC normalization.
Username string `protobuf:"bytes,3,opt,name=username,proto3" json:"username,omitempty"`
}
@@ -2149,7 +2152,7 @@ func (m *CancelQueryRequest) Reset() { *m = CancelQueryRequest{} }
func (m *CancelQueryRequest) String() string { return proto.CompactTextString(m) }
func (*CancelQueryRequest) ProtoMessage() {}
func (*CancelQueryRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{50}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{50}
}
func (m *CancelQueryRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2186,7 +2189,7 @@ func (m *CancelQueryResponse) Reset() { *m = CancelQueryResponse{} }
func (m *CancelQueryResponse) String() string { return proto.CompactTextString(m) }
func (*CancelQueryResponse) ProtoMessage() {}
func (*CancelQueryResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{51}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{51}
}
func (m *CancelQueryResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2221,6 +2224,7 @@ type CancelSessionRequest struct {
SessionID []byte `protobuf:"bytes,2,opt,name=session_id,json=sessionId,proto3" json:"session_id,omitempty"`
// Username of the user making this cancellation request. This may be omitted
// if the user is the same as the one issuing the CancelSessionRequest.
+ // The caller is responsiblef or case-folding and NFC normalization.
Username string `protobuf:"bytes,3,opt,name=username,proto3" json:"username,omitempty"`
}
@@ -2228,7 +2232,7 @@ func (m *CancelSessionRequest) Reset() { *m = CancelSessionRequest{} }
func (m *CancelSessionRequest) String() string { return proto.CompactTextString(m) }
func (*CancelSessionRequest) ProtoMessage() {}
func (*CancelSessionRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{52}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{52}
}
func (m *CancelSessionRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2262,7 +2266,7 @@ func (m *CancelSessionResponse) Reset() { *m = CancelSessionResponse{} }
func (m *CancelSessionResponse) String() string { return proto.CompactTextString(m) }
func (*CancelSessionResponse) ProtoMessage() {}
func (*CancelSessionResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{53}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{53}
}
func (m *CancelSessionResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2297,7 +2301,7 @@ func (m *SpanStatsRequest) Reset() { *m = SpanStatsRequest{} }
func (m *SpanStatsRequest) String() string { return proto.CompactTextString(m) }
func (*SpanStatsRequest) ProtoMessage() {}
func (*SpanStatsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{54}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{54}
}
func (m *SpanStatsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2332,7 +2336,7 @@ func (m *SpanStatsResponse) Reset() { *m = SpanStatsResponse{} }
func (m *SpanStatsResponse) String() string { return proto.CompactTextString(m) }
func (*SpanStatsResponse) ProtoMessage() {}
func (*SpanStatsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{55}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{55}
}
func (m *SpanStatsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2366,7 +2370,7 @@ func (m *ProblemRangesRequest) Reset() { *m = ProblemRangesRequest{} }
func (m *ProblemRangesRequest) String() string { return proto.CompactTextString(m) }
func (*ProblemRangesRequest) ProtoMessage() {}
func (*ProblemRangesRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{56}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{56}
}
func (m *ProblemRangesRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2401,7 +2405,7 @@ func (m *ProblemRangesResponse) Reset() { *m = ProblemRangesResponse{} }
func (m *ProblemRangesResponse) String() string { return proto.CompactTextString(m) }
func (*ProblemRangesResponse) ProtoMessage() {}
func (*ProblemRangesResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{57}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{57}
}
func (m *ProblemRangesResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2442,7 +2446,7 @@ func (m *ProblemRangesResponse_NodeProblems) Reset() { *m = ProblemRange
func (m *ProblemRangesResponse_NodeProblems) String() string { return proto.CompactTextString(m) }
func (*ProblemRangesResponse_NodeProblems) ProtoMessage() {}
func (*ProblemRangesResponse_NodeProblems) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{57, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{57, 0}
}
func (m *ProblemRangesResponse_NodeProblems) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2486,7 +2490,7 @@ func (m *HotRangesRequest) Reset() { *m = HotRangesRequest{} }
func (m *HotRangesRequest) String() string { return proto.CompactTextString(m) }
func (*HotRangesRequest) ProtoMessage() {}
func (*HotRangesRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{58}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{58}
}
func (m *HotRangesRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2526,7 +2530,7 @@ func (m *HotRangesResponse) Reset() { *m = HotRangesResponse{} }
func (m *HotRangesResponse) String() string { return proto.CompactTextString(m) }
func (*HotRangesResponse) ProtoMessage() {}
func (*HotRangesResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{59}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{59}
}
func (m *HotRangesResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2570,7 +2574,7 @@ func (m *HotRangesResponse_HotRange) Reset() { *m = HotRangesResponse_Ho
func (m *HotRangesResponse_HotRange) String() string { return proto.CompactTextString(m) }
func (*HotRangesResponse_HotRange) ProtoMessage() {}
func (*HotRangesResponse_HotRange) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{59, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{59, 0}
}
func (m *HotRangesResponse_HotRange) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2610,7 +2614,7 @@ func (m *HotRangesResponse_StoreResponse) Reset() { *m = HotRangesRespon
func (m *HotRangesResponse_StoreResponse) String() string { return proto.CompactTextString(m) }
func (*HotRangesResponse_StoreResponse) ProtoMessage() {}
func (*HotRangesResponse_StoreResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{59, 1}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{59, 1}
}
func (m *HotRangesResponse_StoreResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2652,7 +2656,7 @@ func (m *HotRangesResponse_NodeResponse) Reset() { *m = HotRangesRespons
func (m *HotRangesResponse_NodeResponse) String() string { return proto.CompactTextString(m) }
func (*HotRangesResponse_NodeResponse) ProtoMessage() {}
func (*HotRangesResponse_NodeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{59, 2}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{59, 2}
}
func (m *HotRangesResponse_NodeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2685,7 +2689,7 @@ func (m *RangeRequest) Reset() { *m = RangeRequest{} }
func (m *RangeRequest) String() string { return proto.CompactTextString(m) }
func (*RangeRequest) ProtoMessage() {}
func (*RangeRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{60}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{60}
}
func (m *RangeRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2721,7 +2725,7 @@ func (m *RangeResponse) Reset() { *m = RangeResponse{} }
func (m *RangeResponse) String() string { return proto.CompactTextString(m) }
func (*RangeResponse) ProtoMessage() {}
func (*RangeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{61}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{61}
}
func (m *RangeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2756,7 +2760,7 @@ func (m *RangeResponse_NodeResponse) Reset() { *m = RangeResponse_NodeRe
func (m *RangeResponse_NodeResponse) String() string { return proto.CompactTextString(m) }
func (*RangeResponse_NodeResponse) ProtoMessage() {}
func (*RangeResponse_NodeResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{61, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{61, 0}
}
func (m *RangeResponse_NodeResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2792,7 +2796,7 @@ func (m *DiagnosticsRequest) Reset() { *m = DiagnosticsRequest{} }
func (m *DiagnosticsRequest) String() string { return proto.CompactTextString(m) }
func (*DiagnosticsRequest) ProtoMessage() {}
func (*DiagnosticsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{62}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{62}
}
func (m *DiagnosticsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2827,7 +2831,7 @@ func (m *StoresRequest) Reset() { *m = StoresRequest{} }
func (m *StoresRequest) String() string { return proto.CompactTextString(m) }
func (*StoresRequest) ProtoMessage() {}
func (*StoresRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{63}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{63}
}
func (m *StoresRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2870,7 +2874,7 @@ func (m *StoreDetails) Reset() { *m = StoreDetails{} }
func (m *StoreDetails) String() string { return proto.CompactTextString(m) }
func (*StoreDetails) ProtoMessage() {}
func (*StoreDetails) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{64}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{64}
}
func (m *StoreDetails) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2903,7 +2907,7 @@ func (m *StoresResponse) Reset() { *m = StoresResponse{} }
func (m *StoresResponse) String() string { return proto.CompactTextString(m) }
func (*StoresResponse) ProtoMessage() {}
func (*StoresResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{65}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{65}
}
func (m *StoresResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2936,7 +2940,7 @@ func (m *StatementsRequest) Reset() { *m = StatementsRequest{} }
func (m *StatementsRequest) String() string { return proto.CompactTextString(m) }
func (*StatementsRequest) ProtoMessage() {}
func (*StatementsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{66}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{66}
}
func (m *StatementsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -2977,7 +2981,7 @@ func (m *StatementsResponse) Reset() { *m = StatementsResponse{} }
func (m *StatementsResponse) String() string { return proto.CompactTextString(m) }
func (*StatementsResponse) ProtoMessage() {}
func (*StatementsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{67}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{67}
}
func (m *StatementsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3015,7 +3019,7 @@ func (m *StatementsResponse_ExtendedStatementStatisticsKey) String() string {
}
func (*StatementsResponse_ExtendedStatementStatisticsKey) ProtoMessage() {}
func (*StatementsResponse_ExtendedStatementStatisticsKey) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{67, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{67, 0}
}
func (m *StatementsResponse_ExtendedStatementStatisticsKey) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3054,7 +3058,7 @@ func (m *StatementsResponse_CollectedStatementStatistics) String() string {
}
func (*StatementsResponse_CollectedStatementStatistics) ProtoMessage() {}
func (*StatementsResponse_CollectedStatementStatistics) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{67, 1}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{67, 1}
}
func (m *StatementsResponse_CollectedStatementStatistics) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3092,7 +3096,7 @@ func (m *StatementsResponse_ExtendedCollectedTransactionStatistics) String() str
}
func (*StatementsResponse_ExtendedCollectedTransactionStatistics) ProtoMessage() {}
func (*StatementsResponse_ExtendedCollectedTransactionStatistics) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{67, 2}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{67, 2}
}
func (m *StatementsResponse_ExtendedCollectedTransactionStatistics) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3129,7 +3133,7 @@ func (m *StatementDiagnosticsReport) Reset() { *m = StatementDiagnostics
func (m *StatementDiagnosticsReport) String() string { return proto.CompactTextString(m) }
func (*StatementDiagnosticsReport) ProtoMessage() {}
func (*StatementDiagnosticsReport) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{68}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{68}
}
func (m *StatementDiagnosticsReport) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3164,7 +3168,7 @@ func (m *CreateStatementDiagnosticsReportRequest) Reset() {
func (m *CreateStatementDiagnosticsReportRequest) String() string { return proto.CompactTextString(m) }
func (*CreateStatementDiagnosticsReportRequest) ProtoMessage() {}
func (*CreateStatementDiagnosticsReportRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{69}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{69}
}
func (m *CreateStatementDiagnosticsReportRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3199,7 +3203,7 @@ func (m *CreateStatementDiagnosticsReportResponse) Reset() {
func (m *CreateStatementDiagnosticsReportResponse) String() string { return proto.CompactTextString(m) }
func (*CreateStatementDiagnosticsReportResponse) ProtoMessage() {}
func (*CreateStatementDiagnosticsReportResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{70}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{70}
}
func (m *CreateStatementDiagnosticsReportResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3231,7 +3235,7 @@ func (m *StatementDiagnosticsReportsRequest) Reset() { *m = StatementDia
func (m *StatementDiagnosticsReportsRequest) String() string { return proto.CompactTextString(m) }
func (*StatementDiagnosticsReportsRequest) ProtoMessage() {}
func (*StatementDiagnosticsReportsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{71}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{71}
}
func (m *StatementDiagnosticsReportsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3264,7 +3268,7 @@ func (m *StatementDiagnosticsReportsResponse) Reset() { *m = StatementDi
func (m *StatementDiagnosticsReportsResponse) String() string { return proto.CompactTextString(m) }
func (*StatementDiagnosticsReportsResponse) ProtoMessage() {}
func (*StatementDiagnosticsReportsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{72}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{72}
}
func (m *StatementDiagnosticsReportsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3300,7 +3304,7 @@ func (m *StatementDiagnostics) Reset() { *m = StatementDiagnostics{} }
func (m *StatementDiagnostics) String() string { return proto.CompactTextString(m) }
func (*StatementDiagnostics) ProtoMessage() {}
func (*StatementDiagnostics) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{73}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{73}
}
func (m *StatementDiagnostics) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3333,7 +3337,7 @@ func (m *StatementDiagnosticsRequest) Reset() { *m = StatementDiagnostic
func (m *StatementDiagnosticsRequest) String() string { return proto.CompactTextString(m) }
func (*StatementDiagnosticsRequest) ProtoMessage() {}
func (*StatementDiagnosticsRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{74}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{74}
}
func (m *StatementDiagnosticsRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3366,7 +3370,7 @@ func (m *StatementDiagnosticsResponse) Reset() { *m = StatementDiagnosti
func (m *StatementDiagnosticsResponse) String() string { return proto.CompactTextString(m) }
func (*StatementDiagnosticsResponse) ProtoMessage() {}
func (*StatementDiagnosticsResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{75}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{75}
}
func (m *StatementDiagnosticsResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3399,7 +3403,7 @@ func (m *JobRegistryStatusRequest) Reset() { *m = JobRegistryStatusReque
func (m *JobRegistryStatusRequest) String() string { return proto.CompactTextString(m) }
func (*JobRegistryStatusRequest) ProtoMessage() {}
func (*JobRegistryStatusRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{76}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{76}
}
func (m *JobRegistryStatusRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3433,7 +3437,7 @@ func (m *JobRegistryStatusResponse) Reset() { *m = JobRegistryStatusResp
func (m *JobRegistryStatusResponse) String() string { return proto.CompactTextString(m) }
func (*JobRegistryStatusResponse) ProtoMessage() {}
func (*JobRegistryStatusResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{77}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{77}
}
func (m *JobRegistryStatusResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3466,7 +3470,7 @@ func (m *JobRegistryStatusResponse_Job) Reset() { *m = JobRegistryStatus
func (m *JobRegistryStatusResponse_Job) String() string { return proto.CompactTextString(m) }
func (*JobRegistryStatusResponse_Job) ProtoMessage() {}
func (*JobRegistryStatusResponse_Job) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{77, 0}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{77, 0}
}
func (m *JobRegistryStatusResponse_Job) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3499,7 +3503,7 @@ func (m *JobStatusRequest) Reset() { *m = JobStatusRequest{} }
func (m *JobStatusRequest) String() string { return proto.CompactTextString(m) }
func (*JobStatusRequest) ProtoMessage() {}
func (*JobStatusRequest) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{78}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{78}
}
func (m *JobStatusRequest) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -3532,7 +3536,7 @@ func (m *JobStatusResponse) Reset() { *m = JobStatusResponse{} }
func (m *JobStatusResponse) String() string { return proto.CompactTextString(m) }
func (*JobStatusResponse) ProtoMessage() {}
func (*JobStatusResponse) Descriptor() ([]byte, []int) {
- return fileDescriptor_status_fdd87e929f38276a, []int{79}
+ return fileDescriptor_status_25b7a53ea004fb24, []int{79}
}
func (m *JobStatusResponse) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -22442,10 +22446,10 @@ var (
)
func init() {
- proto.RegisterFile("server/serverpb/status.proto", fileDescriptor_status_fdd87e929f38276a)
+ proto.RegisterFile("server/serverpb/status.proto", fileDescriptor_status_25b7a53ea004fb24)
}
-var fileDescriptor_status_fdd87e929f38276a = []byte{
+var fileDescriptor_status_25b7a53ea004fb24 = []byte{
// 6184 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xcc, 0x7c, 0x5d, 0x6c, 0x5b, 0x47,
0x76, 0xbf, 0x2f, 0x49, 0x51, 0xe4, 0xa1, 0x3e, 0xa8, 0xd1, 0x87, 0x69, 0xda, 0x91, 0x9c, 0xeb,
diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto
index 66a9967abe52..c9bce0e97c48 100644
--- a/pkg/server/serverpb/status.proto
+++ b/pkg/server/serverpb/status.proto
@@ -543,6 +543,8 @@ message ActiveQuery {
// Request object for ListSessions and ListLocalSessions.
message ListSessionsRequest {
// Username of the user making this request.
+ // The caller is responsible to normalize the username
+ // (= case fold and perform unicode NFC normalization).
string username = 1;
}
@@ -614,9 +616,10 @@ message CancelQueryRequest {
// forwarding is necessary.
string node_id = 1;
// ID of query to be canceled (converted to string).
- string query_id = 2 [ (gogoproto.customname) = "QueryID" ];
+ string query_id = 2 [(gogoproto.customname) = "QueryID" ];
// Username of the user making this cancellation request. This may be omitted
// if the user is the same as the one issuing the CancelQueryRequest.
+ // The caller is responsible for case-folding and NFC normalization.
string username = 3;
}
@@ -635,9 +638,10 @@ message CancelSessionRequest {
// node_id is a string so that "local" can be used to specify that no
// forwarding is necessary.
string node_id = 1;
- bytes session_id = 2 [ (gogoproto.customname) = "SessionID" ];
+ bytes session_id = 2 [(gogoproto.customname) = "SessionID"];
// Username of the user making this cancellation request. This may be omitted
// if the user is the same as the one issuing the CancelSessionRequest.
+ // The caller is responsiblef or case-folding and NFC normalization.
string username = 3;
}
diff --git a/pkg/server/statement_diagnostics_requests.go b/pkg/server/statement_diagnostics_requests.go
index 04b2a0f2dc17..7bda0a4e4d52 100644
--- a/pkg/server/statement_diagnostics_requests.go
+++ b/pkg/server/statement_diagnostics_requests.go
@@ -102,7 +102,7 @@ func (s *statusServer) StatementDiagnosticsRequests(
// TODO(davidh): Add pagination to this request.
rows, err := s.internalExecutor.QueryEx(ctx, "stmt-diag-get-all", nil, /* txn */
sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
},
`SELECT
id,
@@ -169,7 +169,7 @@ func (s *statusServer) StatementDiagnostics(
var err error
row, err := s.internalExecutor.QueryRowEx(ctx, "stmt-diag-get-one", nil, /* txn */
sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
},
`SELECT
id,
diff --git a/pkg/server/status.go b/pkg/server/status.go
index 9ff67f5949de..ef956dd09f89 100644
--- a/pkg/server/status.go
+++ b/pkg/server/status.go
@@ -151,31 +151,36 @@ func (b *baseStatusServer) getLocalSessions(
return nil, err
}
+ reqUsername, err := security.MakeSQLUsernameFromPreNormalizedStringChecked(req.Username)
+ if err != nil {
+ return nil, err
+ }
+
if !isAdmin && !hasViewActivity {
// For non-superusers, requests with an empty username is
// implicitly a request for the client's own sessions.
- if req.Username == "" {
- req.Username = sessionUser
+ if reqUsername.Undefined() {
+ reqUsername = sessionUser
}
// Non-superusers are not allowed to query sessions others than their own.
- if sessionUser != req.Username {
+ if sessionUser != reqUsername {
return nil, grpcstatus.Errorf(
codes.PermissionDenied,
"client user %q does not have permission to view sessions from user %q",
- sessionUser, req.Username)
+ sessionUser, reqUsername)
}
}
// The empty username means "all sessions".
- showAll := req.Username == ""
+ showAll := reqUsername.Undefined()
registry := b.sessionRegistry
sessions := registry.SerializeAll()
userSessions := make([]serverpb.Session, 0, len(sessions))
for _, session := range sessions {
- if req.Username != session.Username && !showAll {
+ if reqUsername.Normalized() != session.Username && !showAll {
continue
}
@@ -221,18 +226,18 @@ func findSessionByQueryID(queryID string) sessionFinder {
}
func (b *baseStatusServer) checkCancelPrivilege(
- ctx context.Context, username string, findSession sessionFinder,
+ ctx context.Context, username security.SQLUsername, findSession sessionFinder,
) error {
ctx = propagateGatewayMetadata(ctx)
ctx = b.AnnotateCtx(ctx)
// reqUser is the user who made the cancellation request.
- var reqUser string
+ var reqUser security.SQLUsername
{
sessionUser, isAdmin, err := b.privilegeChecker.getUserAndRole(ctx)
if err != nil {
return err
}
- if username == "" || username == sessionUser {
+ if username.Undefined() || username == sessionUser {
reqUser = sessionUser
} else {
// When CANCEL QUERY is run as a SQL statement, sessionUser is always root
@@ -256,7 +261,8 @@ func (b *baseStatusServer) checkCancelPrivilege(
return err
}
- if session.Username != reqUser {
+ sessionUser := security.MakeSQLUsernameFromPreNormalizedString(session.Username)
+ if sessionUser != reqUser {
// Must have CANCELQUERY privilege to cancel other users'
// sessions/queries.
ok, err := b.privilegeChecker.hasRoleOption(ctx, reqUser, roleoption.CANCELQUERY)
@@ -267,7 +273,7 @@ func (b *baseStatusServer) checkCancelPrivilege(
return errRequiresRoleOption(roleoption.CANCELQUERY)
}
// Non-admins cannot cancel admins' sessions/queries.
- isAdminSession, err := b.privilegeChecker.hasAdminRole(ctx, session.Username)
+ isAdminSession, err := b.privilegeChecker.hasAdminRole(ctx, sessionUser)
if err != nil {
return err
}
@@ -1929,7 +1935,12 @@ func (s *statusServer) CancelSession(
return status.CancelSession(ctx, req)
}
- if err := s.checkCancelPrivilege(ctx, req.Username, findSessionBySessionID(req.SessionID)); err != nil {
+ reqUsername, err := security.MakeSQLUsernameFromPreNormalizedStringChecked(req.Username)
+ if err != nil {
+ return nil, err
+ }
+
+ if err := s.checkCancelPrivilege(ctx, reqUsername, findSessionBySessionID(req.SessionID)); err != nil {
return nil, err
}
@@ -1956,7 +1967,12 @@ func (s *statusServer) CancelQuery(
return status.CancelQuery(ctx, req)
}
- if err := s.checkCancelPrivilege(ctx, req.Username, findSessionByQueryID(req.QueryID)); err != nil {
+ reqUsername, err := security.MakeSQLUsernameFromPreNormalizedStringChecked(req.Username)
+ if err != nil {
+ return nil, err
+ }
+
+ if err := s.checkCancelPrivilege(ctx, reqUsername, findSessionByQueryID(req.QueryID)); err != nil {
return nil, err
}
@@ -2121,25 +2137,28 @@ func marshalJSONResponse(value interface{}) (*serverpb.JSONResponse, error) {
return &serverpb.JSONResponse{Data: data}, nil
}
-func userFromContext(ctx context.Context) (string, error) {
+func userFromContext(ctx context.Context) (res security.SQLUsername, err error) {
md, ok := metadata.FromIncomingContext(ctx)
if !ok {
// If the incoming context has metadata but no attached web session user,
// it's a gRPC / internal SQL connection which has root on the cluster.
- return security.RootUser, nil
+ return security.RootUserName(), nil
}
usernames, ok := md[webSessionUserKeyStr]
if !ok {
// If the incoming context has metadata but no attached web session user,
// it's a gRPC / internal SQL connection which has root on the cluster.
- return security.RootUser, nil
+ return security.RootUserName(), nil
}
if len(usernames) != 1 {
log.Warningf(ctx, "context's incoming metadata contains unexpected number of usernames: %+v ", md)
- return "", fmt.Errorf(
+ return res, fmt.Errorf(
"context's incoming metadata contains unexpected number of usernames: %+v ", md)
}
- return usernames[0], nil
+ // At this point the user is already logged in, so we can assume
+ // the username has been normalized already.
+ username := security.MakeSQLUsernameFromPreNormalizedString(usernames[0])
+ return username, nil
}
type systemInfoOnce struct {
diff --git a/pkg/server/status_test.go b/pkg/server/status_test.go
index 470440dfaa6c..40399d2bb3dc 100644
--- a/pkg/server/status_test.go
+++ b/pkg/server/status_test.go
@@ -398,7 +398,7 @@ func TestStatusGetFiles(t *testing.T) {
ts := tsI.(*TestServer)
defer ts.Stopper().Stop(context.Background())
- rootConfig := testutils.NewTestBaseContext(security.RootUser)
+ rootConfig := testutils.NewTestBaseContext(security.RootUserName())
rpcContext := newRPCTestContext(ts, rootConfig)
url := ts.ServingRPCAddr()
@@ -1248,7 +1248,7 @@ func TestNodesGRPCResponse(t *testing.T) {
ts := startServer(t)
defer ts.Stopper().Stop(context.Background())
- rootConfig := testutils.NewTestBaseContext(security.RootUser)
+ rootConfig := testutils.NewTestBaseContext(security.RootUserName())
rpcContext := newRPCTestContext(ts, rootConfig)
var request serverpb.NodesRequest
@@ -1452,7 +1452,7 @@ func TestRemoteDebugModeSetting(t *testing.T) {
// don't indicate that the grpc gateway is correctly adding the necessary
// metadata for differentiating between the two (and that we're correctly
// interpreting said metadata).
- rootConfig := testutils.NewTestBaseContext(security.RootUser)
+ rootConfig := testutils.NewTestBaseContext(security.RootUserName())
rpcContext := newRPCTestContext(ts, rootConfig)
url := ts.ServingRPCAddr()
nodeID := ts.NodeID()
@@ -1777,7 +1777,7 @@ func TestStatusAPIStatements(t *testing.T) {
}
// Grant VIEWACTIVITY.
- thirdServerSQL.Exec(t, "ALTER USER $1 VIEWACTIVITY", authenticatedUserNameNoAdmin)
+ thirdServerSQL.Exec(t, "ALTER USER $1 VIEWACTIVITY", authenticatedUserNameNoAdmin().Normalized())
// Hit query endpoint.
if err := getStatusJSONProtoWithAdminOption(firstServerProto, "statements", &resp, false); err != nil {
@@ -1833,10 +1833,10 @@ func TestListSessionsSecurity(t *testing.T) {
for _, requestWithAdmin := range []bool{true, false} {
t.Run(fmt.Sprintf("admin=%v", requestWithAdmin), func(t *testing.T) {
- myUser := authenticatedUserNameNoAdmin
+ myUser := authenticatedUserNameNoAdmin()
expectedErrOnListingRootSessions := "does not have permission to view sessions from user"
if requestWithAdmin {
- myUser = authenticatedUserName
+ myUser = authenticatedUserName()
expectedErrOnListingRootSessions = ""
}
@@ -1847,10 +1847,10 @@ func TestListSessionsSecurity(t *testing.T) {
}{
{"local_sessions", ""},
{"sessions", ""},
- {fmt.Sprintf("local_sessions?username=%s", myUser), ""},
- {fmt.Sprintf("sessions?username=%s", myUser), ""},
- {"local_sessions?username=root", expectedErrOnListingRootSessions},
- {"sessions?username=root", expectedErrOnListingRootSessions},
+ {fmt.Sprintf("local_sessions?username=%s", myUser.Normalized()), ""},
+ {fmt.Sprintf("sessions?username=%s", myUser.Normalized()), ""},
+ {"local_sessions?username=" + security.RootUser, expectedErrOnListingRootSessions},
+ {"sessions?username=" + security.RootUser, expectedErrOnListingRootSessions},
}
for _, tc := range testCases {
var response serverpb.ListSessionsResponse
@@ -1876,7 +1876,7 @@ func TestListSessionsSecurity(t *testing.T) {
}
// gRPC requests behave as root and thus are always allowed.
- rootConfig := testutils.NewTestBaseContext(security.RootUser)
+ rootConfig := testutils.NewTestBaseContext(security.RootUserName())
rpcContext := newRPCTestContext(ts, rootConfig)
url := ts.ServingRPCAddr()
nodeID := ts.NodeID()
@@ -1886,7 +1886,7 @@ func TestListSessionsSecurity(t *testing.T) {
}
client := serverpb.NewStatusClient(conn)
- for _, user := range []string{"", authenticatedUserName, "root"} {
+ for _, user := range []string{"", authenticatedUser, security.RootUser} {
request := &serverpb.ListSessionsRequest{Username: user}
if resp, err := client.ListLocalSessions(ctx, request); err != nil || len(resp.Errors) > 0 {
t.Errorf("unexpected failure listing local sessions for %q; error: %v; response errors: %v",
@@ -1978,7 +1978,7 @@ func TestJobStatusResponse(t *testing.T) {
ts := startServer(t)
defer ts.Stopper().Stop(context.Background())
- rootConfig := testutils.NewTestBaseContext(security.RootUser)
+ rootConfig := testutils.NewTestBaseContext(security.RootUserName())
rpcContext := newRPCTestContext(ts, rootConfig)
url := ts.ServingRPCAddr()
@@ -2000,7 +2000,7 @@ func TestJobStatusResponse(t *testing.T) {
jobs.Record{
Description: "testing",
Statement: "SELECT 1",
- Username: "root",
+ Username: security.RootUserName(),
Details: jobspb.ImportDetails{
Tables: []jobspb.ImportDetails_Table{
{
diff --git a/pkg/server/tenant_status.go b/pkg/server/tenant_status.go
index ff725ee3d7ce..98c3e619c978 100644
--- a/pkg/server/tenant_status.go
+++ b/pkg/server/tenant_status.go
@@ -13,6 +13,7 @@ package server
import (
"context"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
@@ -65,7 +66,8 @@ func (t *tenantStatusServer) ListLocalSessions(
func (t *tenantStatusServer) CancelQuery(
ctx context.Context, request *serverpb.CancelQueryRequest,
) (*serverpb.CancelQueryResponse, error) {
- if err := t.checkCancelPrivilege(ctx, request.Username, findSessionByQueryID(request.QueryID)); err != nil {
+ reqUsername := security.MakeSQLUsernameFromPreNormalizedString(request.Username)
+ if err := t.checkCancelPrivilege(ctx, reqUsername, findSessionByQueryID(request.QueryID)); err != nil {
return nil, err
}
var (
@@ -82,7 +84,8 @@ func (t *tenantStatusServer) CancelQuery(
func (t *tenantStatusServer) CancelSession(
ctx context.Context, request *serverpb.CancelSessionRequest,
) (*serverpb.CancelSessionResponse, error) {
- if err := t.checkCancelPrivilege(ctx, request.Username, findSessionBySessionID(request.SessionID)); err != nil {
+ reqUsername := security.MakeSQLUsernameFromPreNormalizedString(request.Username)
+ if err := t.checkCancelPrivilege(ctx, reqUsername, findSessionBySessionID(request.SessionID)); err != nil {
return nil, err
}
return t.sessionRegistry.CancelSession(request.SessionID)
diff --git a/pkg/server/testserver.go b/pkg/server/testserver.go
index eb27895fccae..691d2177ef44 100644
--- a/pkg/server/testserver.go
+++ b/pkg/server/testserver.go
@@ -65,12 +65,6 @@ import (
"google.golang.org/grpc"
)
-const (
- // TestUser is a fixed user used in unittests.
- // It has valid embedded client certs.
- TestUser = "testuser"
-)
-
// makeTestConfig returns a config for testing. It overrides the
// Certs with the test certs directory.
// We need to override the certs loader.
@@ -105,7 +99,7 @@ func makeTestBaseConfig(st *cluster.Settings) BaseConfig {
baseCfg.SplitListenSQL = true
baseCfg.HTTPAddr = util.TestAddr.String()
// Set standard user for intra-cluster traffic.
- baseCfg.User = security.NodeUser
+ baseCfg.User = security.NodeUserName()
return baseCfg
}
@@ -550,7 +544,7 @@ func makeSQLServerArgs(
return nil, errors.New("external storage is not available to secondary tenants")
},
externalStorageFromURI: func(ctx context.Context,
- uri, user string) (cloud.ExternalStorage, error) {
+ uri string, user security.SQLUsername) (cloud.ExternalStorage, error) {
return nil, errors.New("external uri storage is not available to secondary tenants")
},
},
@@ -838,27 +832,37 @@ func (ts *TestServer) GetHTTPClient() (http.Client, error) {
return ts.Server.rpcContext.GetHTTPClient()
}
-const authenticatedUserName = "authentic_user"
-const authenticatedUserNameNoAdmin = "authentic_user_noadmin"
+const authenticatedUser = "authentic_user"
+
+func authenticatedUserName() security.SQLUsername {
+ return security.MakeSQLUsernameFromPreNormalizedString(authenticatedUser)
+}
+
+const authenticatedUserNoAdmin = "authentic_user_noadmin"
+
+func authenticatedUserNameNoAdmin() security.SQLUsername {
+ return security.MakeSQLUsernameFromPreNormalizedString(authenticatedUserNoAdmin)
+}
// GetAdminAuthenticatedHTTPClient implements the TestServerInterface.
func (ts *TestServer) GetAdminAuthenticatedHTTPClient() (http.Client, error) {
- httpClient, _, err := ts.getAuthenticatedHTTPClientAndCookie(authenticatedUserName, true)
+ httpClient, _, err := ts.getAuthenticatedHTTPClientAndCookie(
+ authenticatedUserName(), true)
return httpClient, err
}
// GetAuthenticatedHTTPClient implements the TestServerInterface.
func (ts *TestServer) GetAuthenticatedHTTPClient(isAdmin bool) (http.Client, error) {
- authUser := authenticatedUserName
+ authUser := authenticatedUserName()
if !isAdmin {
- authUser = authenticatedUserNameNoAdmin
+ authUser = authenticatedUserNameNoAdmin()
}
httpClient, _, err := ts.getAuthenticatedHTTPClientAndCookie(authUser, isAdmin)
return httpClient, err
}
func (ts *TestServer) getAuthenticatedHTTPClientAndCookie(
- authUser string, isAdmin bool,
+ authUser security.SQLUsername, isAdmin bool,
) (http.Client, *serverpb.SessionCookie, error) {
authIdx := 0
if isAdmin {
@@ -909,11 +913,11 @@ func (ts *TestServer) getAuthenticatedHTTPClientAndCookie(
return authClient.httpClient, authClient.cookie, authClient.err
}
-func (ts *TestServer) createAuthUser(userName string, isAdmin bool) error {
+func (ts *TestServer) createAuthUser(userName security.SQLUsername, isAdmin bool) error {
if _, err := ts.Server.sqlServer.internalExecutor.ExecEx(context.TODO(),
"create-auth-user", nil,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
- "CREATE USER $1", userName,
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
+ "CREATE USER $1", userName.Normalized(),
); err != nil {
return err
}
@@ -922,8 +926,8 @@ func (ts *TestServer) createAuthUser(userName string, isAdmin bool) error {
// to rely on CCL code.
if _, err := ts.Server.sqlServer.internalExecutor.ExecEx(context.TODO(),
"grant-admin", nil,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
- "INSERT INTO system.role_members (role, member, \"isAdmin\") VALUES ('admin', $1, true)", userName,
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
+ "INSERT INTO system.role_members (role, member, \"isAdmin\") VALUES ('admin', $1, true)", userName.Normalized(),
); err != nil {
return err
}
@@ -1220,7 +1224,7 @@ func (ts *TestServer) ForceTableGC(
`
row, err := ts.sqlServer.internalExecutor.QueryRowEx(
ctx, "resolve-table-id", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
tableIDQuery, database, table)
if err != nil {
return err
diff --git a/pkg/server/updates.go b/pkg/server/updates.go
index a27c60ac3027..bb054a054a69 100644
--- a/pkg/server/updates.go
+++ b/pkg/server/updates.go
@@ -323,7 +323,7 @@ func (s *Server) getReportingInfo(
// flattened for quick reads, but we'd rather only report the non-defaults.
if datums, err := s.sqlServer.internalExecutor.QueryEx(
ctx, "read-setting", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SELECT name FROM system.settings",
); err != nil {
log.Warningf(ctx, "failed to read settings: %s", err)
@@ -339,7 +339,7 @@ func (s *Server) getReportingInfo(
ctx,
"read-zone-configs",
nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SELECT id, config FROM system.zones",
); err != nil {
log.Warningf(ctx, "%v", err)
@@ -490,7 +490,7 @@ type stringRedactor struct{}
func (stringRedactor) Primitive(v reflect.Value) error {
if v.Kind() == reflect.String && v.String() != "" {
- v.Set(reflect.ValueOf("_"))
+ v.Set(reflect.ValueOf("_").Convert(v.Type()))
}
return nil
}
diff --git a/pkg/sql/alter_database.go b/pkg/sql/alter_database.go
index 808797d7c493..72f9843c0caa 100644
--- a/pkg/sql/alter_database.go
+++ b/pkg/sql/alter_database.go
@@ -13,6 +13,7 @@ package sql
import (
"context"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
@@ -41,8 +42,8 @@ func (n *alterDatabaseOwnerNode) startExec(params runParams) error {
privs := n.desc.GetPrivileges()
// If the owner we want to set to is the current owner, do a no-op.
- newOwner := string(n.n.Owner)
- if newOwner == privs.Owner {
+ newOwner := n.n.Owner
+ if newOwner == privs.Owner() {
return nil
}
if err := params.p.checkCanAlterDatabaseAndSetNewOwner(params.ctx, n.desc, newOwner); err != nil {
@@ -67,16 +68,16 @@ func (n *alterDatabaseOwnerNode) startExec(params runParams) error {
int32(params.extendedEvalCtx.NodeID.SQLInstanceID()),
struct {
DatabaseName string
- Owner string
- User string
- }{n.n.Name.String(), newOwner, params.p.SessionData().User},
+ Owner security.SQLUsername
+ User security.SQLUsername
+ }{n.n.Name.String(), newOwner, params.p.SessionData().User()},
)
}
// checkCanAlterDatabaseAndSetNewOwner handles privilege checking and setting new owner.
// Called in ALTER DATABASE and REASSIGN OWNED BY.
func (p *planner) checkCanAlterDatabaseAndSetNewOwner(
- ctx context.Context, desc catalog.MutableDescriptor, newOwner string,
+ ctx context.Context, desc catalog.MutableDescriptor, newOwner security.SQLUsername,
) error {
if err := p.checkCanAlterToNewOwner(ctx, desc, newOwner); err != nil {
return err
diff --git a/pkg/sql/alter_index.go b/pkg/sql/alter_index.go
index 5ce87fc2fa41..f39070192cac 100644
--- a/pkg/sql/alter_index.go
+++ b/pkg/sql/alter_index.go
@@ -13,6 +13,7 @@ package sql
import (
"context"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
@@ -122,11 +123,11 @@ func (n *alterIndexNode) startExec(params runParams) error {
TableName string
IndexName string
Statement string
- User string
+ User security.SQLUsername
MutationID uint32
}{
n.n.Index.Table.FQString(), n.indexDesc.Name, n.n.String(),
- params.SessionData().User, uint32(mutationID),
+ params.SessionData().User(), uint32(mutationID),
},
)
}
diff --git a/pkg/sql/alter_role.go b/pkg/sql/alter_role.go
index 8d3133c22b25..49c3b62d5956 100644
--- a/pkg/sql/alter_role.go
+++ b/pkg/sql/alter_role.go
@@ -146,7 +146,7 @@ func (n *alterRoleNode) startExec(params runParams) error {
params.ctx,
opName,
params.p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
fmt.Sprintf("SELECT 1 FROM %s WHERE username = $1", userTableName),
normalizedUsername,
)
@@ -236,7 +236,7 @@ func (n *alterRoleNode) startExec(params runParams) error {
params.ctx,
opName,
params.p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
stmt,
qargs...,
)
diff --git a/pkg/sql/alter_schema.go b/pkg/sql/alter_schema.go
index ba9e22a608e6..940255d2f5f5 100644
--- a/pkg/sql/alter_schema.go
+++ b/pkg/sql/alter_schema.go
@@ -14,6 +14,7 @@ import (
"context"
"fmt"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
@@ -94,11 +95,11 @@ func (n *alterSchemaNode) startExec(params runParams) error {
struct {
SchemaName string
NewSchemaName string
- User string
- }{oldName, newName, params.p.SessionData().User},
+ User security.SQLUsername
+ }{oldName, newName, params.p.SessionData().User()},
)
case *tree.AlterSchemaOwner:
- newOwner := string(t.Owner)
+ newOwner := t.Owner
if err := params.p.alterSchemaOwner(
params.ctx, n.desc, newOwner, tree.AsStringWithFQNames(n.n, params.Ann()),
); err != nil {
@@ -112,9 +113,9 @@ func (n *alterSchemaNode) startExec(params runParams) error {
int32(params.extendedEvalCtx.NodeID.SQLInstanceID()),
struct {
SchemaName string
- Owner string
- User string
- }{n.desc.Name, newOwner, params.p.SessionData().User},
+ Owner security.SQLUsername
+ User security.SQLUsername
+ }{n.desc.Name, newOwner, params.p.SessionData().User()},
)
default:
return errors.AssertionFailedf("unknown schema cmd %T", t)
@@ -122,12 +123,15 @@ func (n *alterSchemaNode) startExec(params runParams) error {
}
func (p *planner) alterSchemaOwner(
- ctx context.Context, scDesc *schemadesc.Mutable, newOwner string, jobDescription string,
+ ctx context.Context,
+ scDesc *schemadesc.Mutable,
+ newOwner security.SQLUsername,
+ jobDescription string,
) error {
privs := scDesc.GetPrivileges()
// If the owner we want to set to is the current owner, do a no-op.
- if newOwner == privs.Owner {
+ if newOwner == privs.Owner() {
return nil
}
@@ -141,7 +145,7 @@ func (p *planner) alterSchemaOwner(
// checkCanAlterSchemaAndSetNewOwner handles privilege checking and setting new owner.
// Called in ALTER SCHEMA and REASSIGN OWNED BY.
func (p *planner) checkCanAlterSchemaAndSetNewOwner(
- ctx context.Context, scDesc *schemadesc.Mutable, newOwner string,
+ ctx context.Context, scDesc *schemadesc.Mutable, newOwner security.SQLUsername,
) error {
if err := p.checkCanAlterToNewOwner(ctx, scDesc, newOwner); err != nil {
return err
diff --git a/pkg/sql/alter_sequence.go b/pkg/sql/alter_sequence.go
index c37b859be6f4..8c589406af87 100644
--- a/pkg/sql/alter_sequence.go
+++ b/pkg/sql/alter_sequence.go
@@ -13,6 +13,7 @@ package sql
import (
"context"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
@@ -79,8 +80,8 @@ func (n *alterSequenceNode) startExec(params runParams) error {
struct {
SequenceName string
Statement string
- User string
- }{params.p.ResolvedName(n.n.Name).FQString(), n.n.String(), params.SessionData().User},
+ User security.SQLUsername
+ }{params.p.ResolvedName(n.n.Name).FQString(), n.n.String(), params.SessionData().User()},
)
}
diff --git a/pkg/sql/alter_table.go b/pkg/sql/alter_table.go
index 5f384b5115d8..ca3a48cd7c73 100644
--- a/pkg/sql/alter_table.go
+++ b/pkg/sql/alter_table.go
@@ -782,7 +782,7 @@ func (n *alterTableNode) startExec(params runParams) error {
}
descriptorChanged = true
case *tree.AlterTableOwner:
- changed, err := params.p.alterTableOwner(params.p.EvalContext().Context, n, string(t.Owner))
+ changed, err := params.p.alterTableOwner(params.p.EvalContext().Context, n, t.Owner)
if err != nil {
return err
}
@@ -835,11 +835,11 @@ func (n *alterTableNode) startExec(params runParams) error {
struct {
TableName string
Statement string
- User string
+ User security.SQLUsername
MutationID uint32
CascadeDroppedViews []string
}{params.p.ResolvedName(n.n.Table).FQString(), n.n.String(),
- params.SessionData().User, uint32(mutationID), droppedViews},
+ params.SessionData().User(), uint32(mutationID), droppedViews},
)
}
@@ -1146,7 +1146,7 @@ func (p *planner) removeColumnComment(
ctx,
"delete-column-comment",
p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"DELETE FROM system.comments WHERE type=$1 AND object_id=$2 AND sub_id=$3",
keys.ColumnCommentType,
tableID,
@@ -1199,12 +1199,12 @@ func (p *planner) updateFKBackReferenceName(
// alterTableOwner sets the owner of the table to newOwner and returns true if the descriptor
// was updated.
func (p *planner) alterTableOwner(
- ctx context.Context, n *alterTableNode, newOwner string,
+ ctx context.Context, n *alterTableNode, newOwner security.SQLUsername,
) (bool, error) {
privs := n.tableDesc.GetPrivileges()
// If the owner we want to set to is the current owner, do a no-op.
- if newOwner == privs.Owner {
+ if newOwner == privs.Owner() {
return false, nil
}
@@ -1218,7 +1218,7 @@ func (p *planner) alterTableOwner(
// checkCanAlterTableAndSetNewOwner handles privilege checking and setting new owner.
// Called in ALTER TABLE and REASSIGN OWNED BY.
func (p *planner) checkCanAlterTableAndSetNewOwner(
- ctx context.Context, desc *tabledesc.Mutable, newOwner string,
+ ctx context.Context, desc *tabledesc.Mutable, newOwner security.SQLUsername,
) error {
if err := p.checkCanAlterToNewOwner(ctx, desc, newOwner); err != nil {
return err
diff --git a/pkg/sql/alter_type.go b/pkg/sql/alter_type.go
index 2eefc961accb..25192638b3aa 100644
--- a/pkg/sql/alter_type.go
+++ b/pkg/sql/alter_type.go
@@ -13,6 +13,7 @@ package sql
import (
"context"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -77,7 +78,7 @@ func (n *alterTypeNode) startExec(params runParams) error {
case *tree.AlterTypeSetSchema:
err = params.p.setTypeSchema(params.ctx, n, string(t.Schema))
case *tree.AlterTypeOwner:
- err = params.p.alterTypeOwner(params.ctx, n, string(t.Owner))
+ err = params.p.alterTypeOwner(params.ctx, n, t.Owner)
default:
err = errors.AssertionFailedf("unknown alter type cmd %s", t)
}
@@ -101,7 +102,7 @@ func (n *alterTypeNode) startExec(params runParams) error {
struct {
TypeName string
Statement string
- User string
+ User security.SQLUsername
}{n.desc.Name, tree.AsStringWithFQNames(n.n, params.Ann()), params.p.User()},
)
}
@@ -297,13 +298,15 @@ func (p *planner) setTypeSchema(ctx context.Context, n *alterTypeNode, schema st
)
}
-func (p *planner) alterTypeOwner(ctx context.Context, n *alterTypeNode, newOwner string) error {
+func (p *planner) alterTypeOwner(
+ ctx context.Context, n *alterTypeNode, newOwner security.SQLUsername,
+) error {
typeDesc := n.desc
privs := typeDesc.GetPrivileges()
// If the owner we want to set to is the current owner, do a no-op.
- if newOwner == privs.Owner {
+ if newOwner == privs.Owner() {
return nil
}
@@ -330,9 +333,11 @@ func (p *planner) alterTypeOwner(ctx context.Context, n *alterTypeNode, newOwner
// checkCanAlterTypeAndSetNewOwner handles privilege checking and setting new owner.
// Called in ALTER TYPE and REASSIGN OWNED BY.
func (p *planner) checkCanAlterTypeAndSetNewOwner(
- ctx context.Context, typeDesc *typedesc.Mutable, arrayTypeDesc *typedesc.Mutable, newOwner string,
+ ctx context.Context,
+ typeDesc *typedesc.Mutable,
+ arrayTypeDesc *typedesc.Mutable,
+ newOwner security.SQLUsername,
) error {
-
if err := p.checkCanAlterToNewOwner(ctx, typeDesc, newOwner); err != nil {
return err
}
diff --git a/pkg/sql/authorization.go b/pkg/sql/authorization.go
index f9eba6f8e1f3..6bcbac9ccda4 100644
--- a/pkg/sql/authorization.go
+++ b/pkg/sql/authorization.go
@@ -38,17 +38,17 @@ type MembershipCache struct {
syncutil.Mutex
tableVersion descpb.DescriptorVersion
// userCache is a mapping from username to userRoleMembership.
- userCache map[string]userRoleMembership
+ userCache map[security.SQLUsername]userRoleMembership
}
// userRoleMembership is a mapping of "rolename" -> "with admin option".
-type userRoleMembership map[string]bool
+type userRoleMembership map[security.SQLUsername]bool
// AuthorizationAccessor for checking authorization (e.g. desc privileges).
type AuthorizationAccessor interface {
// CheckPrivilege verifies that the user has `privilege` on `descriptor`.
CheckPrivilegeForUser(
- ctx context.Context, descriptor catalog.Descriptor, privilege privilege.Kind, user string,
+ ctx context.Context, descriptor catalog.Descriptor, privilege privilege.Kind, user security.SQLUsername,
) error
// CheckPrivilege verifies that the current user has `privilege` on `descriptor`.
@@ -64,7 +64,7 @@ type AuthorizationAccessor interface {
// (false, nil) means that the user has NO admin role
// (false, err) means that there was an error running the query on
// the `system.users` table
- UserHasAdminRole(ctx context.Context, user string) (bool, error)
+ UserHasAdminRole(ctx context.Context, user security.SQLUsername) (bool, error)
// HasAdminRole checks if the current session's user has admin role.
HasAdminRole(ctx context.Context) (bool, error)
@@ -76,7 +76,7 @@ type AuthorizationAccessor interface {
// MemberOfWithAdminOption looks up all the roles (direct and indirect) that 'member' is a member
// of and returns a map of role -> isAdmin.
- MemberOfWithAdminOption(ctx context.Context, member string) (map[string]bool, error)
+ MemberOfWithAdminOption(ctx context.Context, member security.SQLUsername) (map[security.SQLUsername]bool, error)
// HasRoleOption converts the roleoption to its SQL column name and checks if
// the user belongs to a role where the option has value true. Requires a
@@ -93,7 +93,10 @@ var _ AuthorizationAccessor = &planner{}
// CheckPrivilegeForUser implements the AuthorizationAccessor interface.
// Requires a valid transaction to be open.
func (p *planner) CheckPrivilegeForUser(
- ctx context.Context, descriptor catalog.Descriptor, privilege privilege.Kind, user string,
+ ctx context.Context,
+ descriptor catalog.Descriptor,
+ privilege privilege.Kind,
+ user security.SQLUsername,
) error {
// Verify that the txn is valid in any case, so that
// we don't get the risk to say "OK" to root requests
@@ -112,11 +115,11 @@ func (p *planner) CheckPrivilegeForUser(
privs := descriptor.GetPrivileges()
// Check if the 'public' pseudo-role has privileges.
- if privs.CheckPrivilege(security.PublicRole, privilege) {
+ if privs.CheckPrivilege(security.PublicRoleName(), privilege) {
return nil
}
- hasPriv, err := p.checkRolePredicate(ctx, user, func(role string) bool {
+ hasPriv, err := p.checkRolePredicate(ctx, user, func(role security.SQLUsername) bool {
return IsOwner(descriptor, role) || privs.CheckPrivilege(role, privilege)
})
if err != nil {
@@ -138,25 +141,25 @@ func (p *planner) CheckPrivilege(
return p.CheckPrivilegeForUser(ctx, descriptor, privilege, p.User())
}
-func getOwnerOfDesc(desc catalog.Descriptor) string {
+func getOwnerOfDesc(desc catalog.Descriptor) security.SQLUsername {
// Descriptors created prior to 20.2 do not have owners set.
- owner := desc.GetPrivileges().Owner
- if owner == "" {
+ owner := desc.GetPrivileges().Owner()
+ if owner.Undefined() {
// If the descriptor is ownerless and the descriptor is part of the system db,
// node is the owner.
if desc.GetID() == keys.SystemDatabaseID || desc.GetParentID() == keys.SystemDatabaseID {
- owner = security.NodeUser
+ owner = security.NodeUserName()
} else {
// This check is redundant in this case since admin already has privilege
// on all non-system objects.
- owner = security.AdminRole
+ owner = security.AdminRoleName()
}
}
return owner
}
// IsOwner returns if the role has ownership on the descriptor.
-func IsOwner(desc catalog.Descriptor, role string) bool {
+func IsOwner(desc catalog.Descriptor, role security.SQLUsername) bool {
return role == getOwnerOfDesc(desc)
}
@@ -165,9 +168,9 @@ func IsOwner(desc catalog.Descriptor, role string) bool {
// TODO(richardjcai): SUPERUSER has implicit ownership.
// We do not have SUPERUSER privilege yet but should we consider root a superuser?
func (p *planner) HasOwnership(ctx context.Context, descriptor catalog.Descriptor) (bool, error) {
- user := p.SessionData().User
+ user := p.SessionData().User()
- return p.checkRolePredicate(ctx, user, func(role string) bool {
+ return p.checkRolePredicate(ctx, user, func(role security.SQLUsername) bool {
return IsOwner(descriptor, role)
})
}
@@ -175,7 +178,7 @@ func (p *planner) HasOwnership(ctx context.Context, descriptor catalog.Descripto
// checkRolePredicate checks if the predicate is true for the user or
// any roles the user is a member of.
func (p *planner) checkRolePredicate(
- ctx context.Context, user string, predicate func(role string) bool,
+ ctx context.Context, user security.SQLUsername, predicate func(role security.SQLUsername) bool,
) (bool, error) {
if ok := predicate(user); ok {
return ok, nil
@@ -202,7 +205,7 @@ func (p *planner) CheckAnyPrivilege(ctx context.Context, descriptor catalog.Desc
return errors.AssertionFailedf("cannot use CheckAnyPrivilege without a txn")
}
- user := p.SessionData().User
+ user := p.SessionData().User()
privs := descriptor.GetPrivileges()
// Check if 'user' itself has privileges.
@@ -211,7 +214,7 @@ func (p *planner) CheckAnyPrivilege(ctx context.Context, descriptor catalog.Desc
}
// Check if 'public' has privileges.
- if privs.AnyPrivilege(security.PublicRole) {
+ if privs.AnyPrivilege(security.PublicRoleName()) {
return nil
}
@@ -230,13 +233,13 @@ func (p *planner) CheckAnyPrivilege(ctx context.Context, descriptor catalog.Desc
return pgerror.Newf(pgcode.InsufficientPrivilege,
"user %s has no privileges on %s %s",
- p.SessionData().User, descriptor.TypeName(), descriptor.GetName())
+ p.SessionData().User(), descriptor.TypeName(), descriptor.GetName())
}
// UserHasAdminRole implements the AuthorizationAccessor interface.
// Requires a valid transaction to be open.
-func (p *planner) UserHasAdminRole(ctx context.Context, user string) (bool, error) {
- if user == "" {
+func (p *planner) UserHasAdminRole(ctx context.Context, user security.SQLUsername) (bool, error) {
+ if user.Undefined() {
return false, errors.AssertionFailedf("empty user")
}
// Verify that the txn is valid in any case, so that
@@ -249,7 +252,7 @@ func (p *planner) UserHasAdminRole(ctx context.Context, user string) (bool, erro
// Check if user is 'root' or 'node'.
// TODO(knz): planner HasAdminRole has no business authorizing
// the "node" principal - node should not be issuing SQL queries.
- if user == security.RootUser || user == security.NodeUser {
+ if user.IsRootUser() || user.IsNodeUser() {
return true, nil
}
@@ -260,7 +263,7 @@ func (p *planner) UserHasAdminRole(ctx context.Context, user string) (bool, erro
}
// Check is 'user' is a member of role 'admin'.
- if _, ok := memberOf[security.AdminRole]; ok {
+ if _, ok := memberOf[security.AdminRoleName()]; ok {
return true, nil
}
@@ -294,8 +297,8 @@ func (p *planner) RequireAdminRole(ctx context.Context, action string) error {
// The "isAdmin" flag applies to both direct and indirect members.
// Requires a valid transaction to be open.
func (p *planner) MemberOfWithAdminOption(
- ctx context.Context, member string,
-) (map[string]bool, error) {
+ ctx context.Context, member security.SQLUsername,
+) (map[security.SQLUsername]bool, error) {
if p.txn == nil || !p.txn.IsOpen() {
return nil, errors.AssertionFailedf("cannot use MemberOfWithAdminoption without a txn")
}
@@ -331,7 +334,7 @@ func (p *planner) MemberOfWithAdminOption(
if roleMembersCache.tableVersion != tableVersion {
// Update version and drop the map.
roleMembersCache.tableVersion = tableVersion
- roleMembersCache.userCache = make(map[string]userRoleMembership)
+ roleMembersCache.userCache = make(map[security.SQLUsername]userRoleMembership)
}
userMapping, ok := roleMembersCache.userCache[member]
@@ -369,13 +372,13 @@ func (p *planner) MemberOfWithAdminOption(
// we could save detailed memberships (as opposed to fully expanded) and reuse them
// across users. We may then want to lookup more than just this user.
func (p *planner) resolveMemberOfWithAdminOption(
- ctx context.Context, member string, txn *kv.Txn,
-) (map[string]bool, error) {
- ret := map[string]bool{}
+ ctx context.Context, member security.SQLUsername, txn *kv.Txn,
+) (map[security.SQLUsername]bool, error) {
+ ret := map[security.SQLUsername]bool{}
// Keep track of members we looked up.
- visited := map[string]struct{}{}
- toVisit := []string{member}
+ visited := map[security.SQLUsername]struct{}{}
+ toVisit := []security.SQLUsername{member}
lookupRolesStmt := `SELECT "role", "isAdmin" FROM system.role_members WHERE "member" = $1`
for len(toVisit) > 0 {
@@ -388,7 +391,7 @@ func (p *planner) resolveMemberOfWithAdminOption(
visited[m] = struct{}{}
rows, err := p.ExecCfg().InternalExecutor.Query(
- ctx, "expand-roles", txn, lookupRolesStmt, m,
+ ctx, "expand-roles", txn, lookupRolesStmt, m.Normalized(),
)
if err != nil {
return nil, err
@@ -398,10 +401,12 @@ func (p *planner) resolveMemberOfWithAdminOption(
roleName := tree.MustBeDString(row[0])
isAdmin := row[1].(*tree.DBool)
- ret[string(roleName)] = bool(*isAdmin)
+ // system.role_members stores pre-normalized usernames.
+ role := security.MakeSQLUsernameFromPreNormalizedString(string(roleName))
+ ret[role] = bool(*isAdmin)
// We need to expand this role. Let the "pop" worry about already-visited elements.
- toVisit = append(toVisit, string(roleName))
+ toVisit = append(toVisit, role)
}
}
@@ -417,8 +422,8 @@ func (p *planner) HasRoleOption(ctx context.Context, roleOption roleoption.Optio
return false, errors.AssertionFailedf("cannot use HasRoleOption without a txn")
}
- user := p.SessionData().User
- if user == security.RootUser || user == security.NodeUser {
+ user := p.SessionData().User()
+ if user.IsRootUser() || user.IsNodeUser() {
return true, nil
}
@@ -431,16 +436,12 @@ func (p *planner) HasRoleOption(ctx context.Context, roleOption roleoption.Optio
return true, nil
}
- normalizedName, err := NormalizeAndValidateUsername(user)
- if err != nil {
- return false, err
- }
hasRolePrivilege, err := p.ExecCfg().InternalExecutor.QueryEx(
ctx, "has-role-option", p.Txn(),
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
fmt.Sprintf(
`SELECT 1 from %s WHERE option = '%s' AND username = $1 LIMIT 1`,
- RoleOptionsTableName, roleOption.String()), normalizedName)
+ RoleOptionsTableName, roleOption.String()), user.Normalized())
if err != nil {
return false, err
}
@@ -506,7 +507,7 @@ func (p *planner) canCreateOnSchema(
ctx context.Context,
schemaID descpb.ID,
dbID descpb.ID,
- user string,
+ user security.SQLUsername,
checkPublicSchema shouldCheckPublicSchema,
) error {
resolvedSchema, err := p.Descriptors().ResolveSchemaByID(ctx, p.Txn(), schemaID)
@@ -569,7 +570,7 @@ func (p *planner) canResolveDescUnderSchema(
// has privileges to alter the owner of the object. If the current user is not
// a superuser, it also checks that they are a member of the new owner role.
func (p *planner) checkCanAlterToNewOwner(
- ctx context.Context, desc catalog.MutableDescriptor, newOwner string,
+ ctx context.Context, desc catalog.MutableDescriptor, newOwner security.SQLUsername,
) error {
// Make sure the newOwner exists.
roleExists, err := p.RoleExists(ctx, newOwner)
diff --git a/pkg/sql/cancel_queries.go b/pkg/sql/cancel_queries.go
index a5727fd34e7e..4b6c46e99d21 100644
--- a/pkg/sql/cancel_queries.go
+++ b/pkg/sql/cancel_queries.go
@@ -60,7 +60,7 @@ func (n *cancelQueriesNode) Next(params runParams) (bool, error) {
request := &serverpb.CancelQueryRequest{
NodeId: fmt.Sprintf("%d", nodeID),
QueryID: string(queryIDString),
- Username: params.SessionData().User,
+ Username: params.SessionData().User().Normalized(),
}
response, err := params.extendedEvalCtx.SQLStatusServer.CancelQuery(params.ctx, request)
diff --git a/pkg/sql/cancel_sessions.go b/pkg/sql/cancel_sessions.go
index 26ed17db36af..3b7e20203ce4 100644
--- a/pkg/sql/cancel_sessions.go
+++ b/pkg/sql/cancel_sessions.go
@@ -60,7 +60,7 @@ func (n *cancelSessionsNode) Next(params runParams) (bool, error) {
request := &serverpb.CancelSessionRequest{
NodeId: fmt.Sprintf("%d", nodeID),
SessionID: sessionID.GetBytes(),
- Username: params.SessionData().User,
+ Username: params.SessionData().User().Normalized(),
}
response, err := params.extendedEvalCtx.SQLStatusServer.CancelSession(params.ctx, request)
diff --git a/pkg/sql/catalog/dbdesc/database_desc.go b/pkg/sql/catalog/dbdesc/database_desc.go
index af646e106a13..e6ca8a840737 100644
--- a/pkg/sql/catalog/dbdesc/database_desc.go
+++ b/pkg/sql/catalog/dbdesc/database_desc.go
@@ -16,6 +16,7 @@ import (
"fmt"
"github.com/cockroachdb/cockroach/pkg/keys"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
@@ -48,7 +49,7 @@ type Mutable struct {
// NewInitial constructs a new Mutable for an initial version from an id and
// name with default privileges.
-func NewInitial(id descpb.ID, name string, owner string) *Mutable {
+func NewInitial(id descpb.ID, name string, owner security.SQLUsername) *Mutable {
return NewInitialWithPrivileges(id, name,
descpb.NewDefaultPrivilegeDescriptor(owner))
}
diff --git a/pkg/sql/catalog/dbdesc/database_test.go b/pkg/sql/catalog/dbdesc/database_test.go
index d559884cee59..4db2d2c13650 100644
--- a/pkg/sql/catalog/dbdesc/database_test.go
+++ b/pkg/sql/catalog/dbdesc/database_test.go
@@ -67,7 +67,7 @@ func TestMakeDatabaseDesc(t *testing.T) {
t.Fatal(err)
}
const id = 17
- desc := NewInitial(id, string(stmt.AST.(*tree.CreateDatabase).Name), security.AdminRole)
+ desc := NewInitial(id, string(stmt.AST.(*tree.CreateDatabase).Name), security.AdminRoleName())
if desc.GetName() != "test" {
t.Fatalf("expected Name == test, got %s", desc.GetName())
}
diff --git a/pkg/sql/catalog/descpb/privilege.go b/pkg/sql/catalog/descpb/privilege.go
index 1cfd8d6f1970..f59406574fca 100644
--- a/pkg/sql/catalog/descpb/privilege.go
+++ b/pkg/sql/catalog/descpb/privilege.go
@@ -39,13 +39,23 @@ func isPrivilegeSet(bits uint32, priv privilege.Kind) bool {
return bits&priv.Mask() != 0
}
+// Owner accesses the owner field.
+func (p PrivilegeDescriptor) Owner() security.SQLUsername {
+ return p.OwnerProto.Decode()
+}
+
+// User accesses the owner field.
+func (u UserPrivileges) User() security.SQLUsername {
+ return u.UserProto.Decode()
+}
+
// findUserIndex looks for a given user and returns its
// index in the User array if found. Returns -1 otherwise.
-func (p PrivilegeDescriptor) findUserIndex(user string) int {
+func (p PrivilegeDescriptor) findUserIndex(user security.SQLUsername) int {
idx := sort.Search(len(p.Users), func(i int) bool {
- return p.Users[i].User >= user
+ return !p.Users[i].User().LessThan(user)
})
- if idx < len(p.Users) && p.Users[idx].User == user {
+ if idx < len(p.Users) && p.Users[idx].User() == user {
return idx
}
return -1
@@ -53,7 +63,7 @@ func (p PrivilegeDescriptor) findUserIndex(user string) int {
// findUser looks for a specific user in the list.
// Returns (nil, false) if not found, or (obj, true) if found.
-func (p PrivilegeDescriptor) findUser(user string) (*UserPrivileges, bool) {
+func (p PrivilegeDescriptor) findUser(user security.SQLUsername) (*UserPrivileges, bool) {
idx := p.findUserIndex(user)
if idx == -1 {
return nil, false
@@ -62,26 +72,26 @@ func (p PrivilegeDescriptor) findUser(user string) (*UserPrivileges, bool) {
}
// findOrCreateUser looks for a specific user in the list, creating it if needed.
-func (p *PrivilegeDescriptor) findOrCreateUser(user string) *UserPrivileges {
+func (p *PrivilegeDescriptor) findOrCreateUser(user security.SQLUsername) *UserPrivileges {
idx := sort.Search(len(p.Users), func(i int) bool {
- return p.Users[i].User >= user
+ return !p.Users[i].User().LessThan(user)
})
if idx == len(p.Users) {
// Not found but should be inserted at the end.
- p.Users = append(p.Users, UserPrivileges{User: user})
- } else if p.Users[idx].User == user {
+ p.Users = append(p.Users, UserPrivileges{UserProto: user.EncodeProto()})
+ } else if p.Users[idx].User() == user {
// Found.
} else {
// New element to be inserted at idx.
p.Users = append(p.Users, UserPrivileges{})
copy(p.Users[idx+1:], p.Users[idx:])
- p.Users[idx] = UserPrivileges{User: user}
+ p.Users[idx] = UserPrivileges{UserProto: user.EncodeProto()}
}
return &p.Users[idx]
}
// removeUser looks for a given user in the list and removes it if present.
-func (p *PrivilegeDescriptor) removeUser(user string) {
+func (p *PrivilegeDescriptor) removeUser(user security.SQLUsername) {
idx := p.findUserIndex(user)
if idx == -1 {
// Not found.
@@ -92,16 +102,18 @@ func (p *PrivilegeDescriptor) removeUser(user string) {
// NewCustomSuperuserPrivilegeDescriptor returns a privilege descriptor for the root user
// and the admin role with specified privileges.
-func NewCustomSuperuserPrivilegeDescriptor(priv privilege.List, owner string) *PrivilegeDescriptor {
+func NewCustomSuperuserPrivilegeDescriptor(
+ priv privilege.List, owner security.SQLUsername,
+) *PrivilegeDescriptor {
return &PrivilegeDescriptor{
- Owner: owner,
+ OwnerProto: owner.EncodeProto(),
Users: []UserPrivileges{
{
- User: security.AdminRole,
+ UserProto: security.AdminRoleName().EncodeProto(),
Privileges: priv.ToBitField(),
},
{
- User: security.RootUser,
+ UserProto: security.RootUserName().EncodeProto(),
Privileges: priv.ToBitField(),
},
},
@@ -111,12 +123,14 @@ func NewCustomSuperuserPrivilegeDescriptor(priv privilege.List, owner string) *P
// NewPrivilegeDescriptor returns a privilege descriptor for the given
// user with the specified list of privileges.
-func NewPrivilegeDescriptor(user string, priv privilege.List, owner string) *PrivilegeDescriptor {
+func NewPrivilegeDescriptor(
+ user security.SQLUsername, priv privilege.List, owner security.SQLUsername,
+) *PrivilegeDescriptor {
return &PrivilegeDescriptor{
- Owner: owner,
+ OwnerProto: owner.EncodeProto(),
Users: []UserPrivileges{
{
- User: user,
+ UserProto: user.EncodeProto(),
Privileges: priv.ToBitField(),
},
},
@@ -130,14 +144,14 @@ var DefaultSuperuserPrivileges = privilege.List{privilege.ALL}
// NewDefaultPrivilegeDescriptor returns a privilege descriptor
// with ALL privileges for the root user and admin role.
-func NewDefaultPrivilegeDescriptor(owner string) *PrivilegeDescriptor {
+func NewDefaultPrivilegeDescriptor(owner security.SQLUsername) *PrivilegeDescriptor {
return NewCustomSuperuserPrivilegeDescriptor(DefaultSuperuserPrivileges, owner)
}
// Grant adds new privileges to this descriptor for a given list of users.
// TODO(marc): if all privileges other than ALL are set, should we collapse
// them into ALL?
-func (p *PrivilegeDescriptor) Grant(user string, privList privilege.List) {
+func (p *PrivilegeDescriptor) Grant(user security.SQLUsername, privList privilege.List) {
userPriv := p.findOrCreateUser(user)
if isPrivilegeSet(userPriv.Privileges, privilege.ALL) {
// User already has 'ALL' privilege: no-op.
@@ -157,7 +171,7 @@ func (p *PrivilegeDescriptor) Grant(user string, privList privilege.List) {
// Revoke removes privileges from this descriptor for a given list of users.
func (p *PrivilegeDescriptor) Revoke(
- user string, privList privilege.List, objectType privilege.ObjectType,
+ user security.SQLUsername, privList privilege.List, objectType privilege.ObjectType,
) {
userPriv, ok := p.findUser(user)
if !ok || userPriv.Privileges == 0 {
@@ -213,7 +227,7 @@ func MaybeFixPrivileges(id ID, p *PrivilegeDescriptor) bool {
var modified bool
- fixSuperUser := func(user string) {
+ fixSuperUser := func(user security.SQLUsername) {
privs := p.findOrCreateUser(user)
if privs.Privileges != allowedPrivilegesBits {
privs.Privileges = allowedPrivilegesBits
@@ -222,8 +236,8 @@ func MaybeFixPrivileges(id ID, p *PrivilegeDescriptor) bool {
}
// Check "root" user and "admin" role.
- fixSuperUser(security.RootUser)
- fixSuperUser(security.AdminRole)
+ fixSuperUser(security.RootUserName())
+ fixSuperUser(security.AdminRoleName())
if isPrivilegeSet(allowedPrivilegesBits, privilege.ALL) {
// ALL privileges allowed, we can skip regular users.
@@ -233,7 +247,7 @@ func MaybeFixPrivileges(id ID, p *PrivilegeDescriptor) bool {
for i := range p.Users {
// Users is a slice of values, we need pointers to make them mutable.
u := &p.Users[i]
- if u.User == security.RootUser || u.User == security.AdminRole {
+ if u.User().IsRootUser() || u.User().IsAdminRole() {
// we've already checked super users.
continue
}
@@ -265,17 +279,17 @@ func (p PrivilegeDescriptor) Validate(id ID, objectType privilege.ObjectType) er
}
// Check "root" user.
- if err := p.validateRequiredSuperuser(id, allowedPrivileges, security.RootUser, objectType); err != nil {
+ if err := p.validateRequiredSuperuser(id, allowedPrivileges, security.RootUserName(), objectType); err != nil {
return err
}
// We expect an "admin" role. Check that it has desired superuser permissions.
- if err := p.validateRequiredSuperuser(id, allowedPrivileges, security.AdminRole, objectType); err != nil {
+ if err := p.validateRequiredSuperuser(id, allowedPrivileges, security.AdminRoleName(), objectType); err != nil {
return err
}
if p.Version >= OwnerVersion {
- if p.Owner == "" {
+ if p.Owner().Undefined() {
return errors.AssertionFailedf("found no owner for system %s with ID=%d",
objectType, id)
}
@@ -286,14 +300,14 @@ func (p PrivilegeDescriptor) Validate(id ID, objectType privilege.ObjectType) er
// For all non-super users, privileges must not exceed the allowed privileges.
// Also the privileges must be valid on the object type.
for _, u := range p.Users {
- if u.User == security.RootUser || u.User == security.AdminRole {
+ if u.User().IsRootUser() || u.User().IsAdminRole() {
// We've already checked super users.
continue
}
if remaining := u.Privileges &^ allowedPrivilegesBits; remaining != 0 {
return fmt.Errorf("user %s must not have %s privileges on system %s with ID=%d",
- u.User, privilege.ListFromBitField(remaining, privilege.Any), objectType, id)
+ u.User(), privilege.ListFromBitField(remaining, privilege.Any), objectType, id)
}
// Get all the privilege bits set on the descriptor even if they're not valid.
privs := privilege.ListFromBitField(u.Privileges, privilege.Any)
@@ -308,7 +322,10 @@ func (p PrivilegeDescriptor) Validate(id ID, objectType privilege.ObjectType) er
}
func (p PrivilegeDescriptor) validateRequiredSuperuser(
- id ID, allowedPrivileges privilege.List, user string, objectType privilege.ObjectType,
+ id ID,
+ allowedPrivileges privilege.List,
+ user security.SQLUsername,
+ objectType privilege.ObjectType,
) error {
superPriv, ok := p.findUser(user)
if !ok {
@@ -328,7 +345,7 @@ func (p PrivilegeDescriptor) validateRequiredSuperuser(
// UserPrivilegeString is a pair of strings describing the
// privileges for a given user.
type UserPrivilegeString struct {
- User string
+ User security.SQLUsername
Privileges []string
}
@@ -343,7 +360,7 @@ func (p PrivilegeDescriptor) Show(objectType privilege.ObjectType) []UserPrivile
ret := make([]UserPrivilegeString, 0, len(p.Users))
for _, userPriv := range p.Users {
ret = append(ret, UserPrivilegeString{
- User: userPriv.User,
+ User: userPriv.User(),
Privileges: privilege.ListFromBitField(userPriv.Privileges, objectType).SortedNames(),
})
}
@@ -351,11 +368,11 @@ func (p PrivilegeDescriptor) Show(objectType privilege.ObjectType) []UserPrivile
}
// CheckPrivilege returns true if 'user' has 'privilege' on this descriptor.
-func (p PrivilegeDescriptor) CheckPrivilege(user string, priv privilege.Kind) bool {
+func (p PrivilegeDescriptor) CheckPrivilege(user security.SQLUsername, priv privilege.Kind) bool {
userPriv, ok := p.findUser(user)
if !ok {
// User "node" has all privileges.
- return user == security.NodeUser
+ return user.IsNodeUser()
}
if isPrivilegeSet(userPriv.Privileges, privilege.ALL) {
@@ -365,8 +382,8 @@ func (p PrivilegeDescriptor) CheckPrivilege(user string, priv privilege.Kind) bo
}
// AnyPrivilege returns true if 'user' has any privilege on this descriptor.
-func (p PrivilegeDescriptor) AnyPrivilege(user string) bool {
- if p.Owner == user {
+func (p PrivilegeDescriptor) AnyPrivilege(user security.SQLUsername) bool {
+ if p.Owner() == user {
return true
}
userPriv, ok := p.findUser(user)
@@ -421,6 +438,6 @@ var SystemAllowedPrivileges = map[ID]privilege.List{
}
// SetOwner sets the owner of the privilege descriptor to the provided string.
-func (p *PrivilegeDescriptor) SetOwner(owner string) {
- p.Owner = owner
+func (p *PrivilegeDescriptor) SetOwner(owner security.SQLUsername) {
+ p.OwnerProto = owner.EncodeProto()
}
diff --git a/pkg/sql/catalog/descpb/privilege.pb.go b/pkg/sql/catalog/descpb/privilege.pb.go
index 6a5cb982497b..86e4873774bf 100644
--- a/pkg/sql/catalog/descpb/privilege.pb.go
+++ b/pkg/sql/catalog/descpb/privilege.pb.go
@@ -7,6 +7,8 @@ import proto "github.com/gogo/protobuf/proto"
import fmt "fmt"
import math "math"
+import github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security"
+
import io "io"
// Reference imports to suppress errors if they are not otherwise used.
@@ -22,7 +24,7 @@ const _ = proto.GoGoProtoPackageIsVersion2 // please upgrade the proto package
// UserPrivileges describes the list of privileges available for a given user.
type UserPrivileges struct {
- User string `protobuf:"bytes,1,opt,name=user" json:"user"`
+ UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,1,opt,name=user_proto,json=userProto,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto"`
// privileges is a bitfield of 1< l {
return io.ErrUnexpectedEOF
}
- m.User = string(dAtA[iNdEx:postIndex])
+ m.UserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 2:
if wireType != 0 {
@@ -433,7 +435,7 @@ func (m *PrivilegeDescriptor) Unmarshal(dAtA []byte) error {
iNdEx = postIndex
case 2:
if wireType != 2 {
- return fmt.Errorf("proto: wrong wireType = %d for field Owner", wireType)
+ return fmt.Errorf("proto: wrong wireType = %d for field OwnerProto", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
@@ -458,7 +460,7 @@ func (m *PrivilegeDescriptor) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
- m.Owner = string(dAtA[iNdEx:postIndex])
+ m.OwnerProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 3:
if wireType != 0 {
@@ -606,27 +608,31 @@ var (
)
func init() {
- proto.RegisterFile("sql/catalog/descpb/privilege.proto", fileDescriptor_privilege_c3e6b8f383a64b66)
+ proto.RegisterFile("sql/catalog/descpb/privilege.proto", fileDescriptor_privilege_f48077236b0cbe6a)
}
-var fileDescriptor_privilege_c3e6b8f383a64b66 = []byte{
- // 284 bytes of a gzipped FileDescriptorProto
+var fileDescriptor_privilege_f48077236b0cbe6a = []byte{
+ // 342 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xe2, 0x52, 0x2a, 0x2e, 0xcc, 0xd1,
0x4f, 0x4e, 0x2c, 0x49, 0xcc, 0xc9, 0x4f, 0xd7, 0x4f, 0x49, 0x2d, 0x4e, 0x2e, 0x48, 0xd2, 0x2f,
0x28, 0xca, 0x2c, 0xcb, 0xcc, 0x49, 0x4d, 0x4f, 0xd5, 0x2b, 0x28, 0xca, 0x2f, 0xc9, 0x17, 0x12,
0x4d, 0xce, 0x4f, 0xce, 0x2e, 0xca, 0x4f, 0x4c, 0xce, 0xd0, 0x2b, 0x2e, 0xcc, 0x01, 0xe1, 0xa4,
0xc4, 0xe2, 0x54, 0x29, 0x91, 0xf4, 0xfc, 0xf4, 0x7c, 0xb0, 0x0a, 0x7d, 0x10, 0x0b, 0xa2, 0x58,
- 0x29, 0x8c, 0x8b, 0x2f, 0xb4, 0x38, 0xb5, 0x28, 0x00, 0x66, 0x46, 0xb1, 0x90, 0x04, 0x17, 0x4b,
- 0x69, 0x71, 0x6a, 0x91, 0x04, 0xa3, 0x02, 0xa3, 0x06, 0xa7, 0x13, 0xcb, 0x89, 0x7b, 0xf2, 0x0c,
- 0x41, 0x60, 0x11, 0x21, 0x15, 0x2e, 0x2e, 0xb8, 0x5d, 0xc5, 0x12, 0x4c, 0x0a, 0x8c, 0x1a, 0xbc,
- 0x50, 0x79, 0x24, 0x71, 0x2b, 0x96, 0x17, 0x0b, 0xe4, 0x19, 0x95, 0x36, 0x31, 0x72, 0x09, 0xc3,
- 0x0d, 0x75, 0x49, 0x2d, 0x4e, 0x2e, 0xca, 0x2c, 0x28, 0xc9, 0x2f, 0x12, 0x72, 0xe4, 0x62, 0x05,
- 0x99, 0x55, 0x2c, 0xc1, 0xa8, 0xc0, 0xac, 0xc1, 0x6d, 0xa4, 0xaa, 0x87, 0xd5, 0xb1, 0x7a, 0xa8,
- 0x6e, 0x82, 0xda, 0x02, 0xd1, 0x29, 0x24, 0xc5, 0xc5, 0x9a, 0x5f, 0x9e, 0x97, 0x5a, 0x04, 0x76,
- 0x01, 0xcc, 0x85, 0x10, 0x21, 0x21, 0x33, 0x2e, 0xf6, 0xb2, 0xd4, 0xa2, 0xe2, 0xcc, 0xfc, 0x3c,
- 0x09, 0x66, 0xb0, 0xfb, 0x64, 0x40, 0xb2, 0xbf, 0xee, 0xc9, 0x8b, 0xa0, 0x38, 0x26, 0x0c, 0xa2,
- 0x26, 0x08, 0xa6, 0x18, 0xe2, 0x68, 0x27, 0x8d, 0x13, 0x0f, 0xe5, 0x18, 0x4e, 0x3c, 0x92, 0x63,
- 0xbc, 0xf0, 0x48, 0x8e, 0xf1, 0xc6, 0x23, 0x39, 0xc6, 0x07, 0x8f, 0xe4, 0x18, 0x27, 0x3c, 0x96,
- 0x63, 0xb8, 0xf0, 0x58, 0x8e, 0xe1, 0xc6, 0x63, 0x39, 0x86, 0x28, 0x36, 0x48, 0x98, 0x03, 0x02,
- 0x00, 0x00, 0xff, 0xff, 0xe6, 0x32, 0x58, 0x8d, 0x88, 0x01, 0x00, 0x00,
+ 0x69, 0x31, 0x23, 0x17, 0x5f, 0x68, 0x71, 0x6a, 0x51, 0x00, 0xcc, 0x90, 0x62, 0xa1, 0x54, 0x2e,
+ 0xae, 0xd2, 0xe2, 0xd4, 0xa2, 0x78, 0xb0, 0x02, 0x09, 0x46, 0x05, 0x46, 0x0d, 0x4e, 0x27, 0xb7,
+ 0x13, 0xf7, 0xe4, 0x19, 0x7e, 0xdd, 0x93, 0xb7, 0x4b, 0xcf, 0x2c, 0xc9, 0x28, 0x4d, 0xd2, 0x4b,
+ 0xce, 0xcf, 0xd5, 0x87, 0x5b, 0x93, 0x92, 0x84, 0x60, 0xeb, 0x17, 0x64, 0xa7, 0xeb, 0x17, 0xa7,
+ 0x26, 0x97, 0x16, 0x65, 0x96, 0x54, 0xea, 0x05, 0x07, 0xfa, 0x80, 0x0c, 0xcf, 0x4b, 0xcc, 0x4d,
+ 0x0d, 0x00, 0x99, 0x16, 0xc4, 0x59, 0x0a, 0xb6, 0x0b, 0xe4, 0x4c, 0x15, 0x2e, 0x2e, 0xb8, 0xcb,
+ 0x8b, 0x25, 0x98, 0x14, 0x18, 0x35, 0x78, 0x9d, 0x58, 0x40, 0xd6, 0x04, 0x21, 0x89, 0x5b, 0xb1,
+ 0xbc, 0x58, 0x20, 0xcf, 0xa8, 0xf4, 0x9f, 0x91, 0x4b, 0x18, 0xee, 0x42, 0x97, 0xd4, 0xe2, 0xe4,
+ 0xa2, 0xcc, 0x82, 0x92, 0xfc, 0x22, 0x21, 0x47, 0x2e, 0x56, 0x90, 0x81, 0xc5, 0x12, 0x8c, 0x0a,
+ 0xcc, 0x1a, 0xdc, 0x46, 0xaa, 0x7a, 0x58, 0xbd, 0xae, 0x87, 0xea, 0x41, 0xa8, 0x2d, 0x10, 0x9d,
+ 0x42, 0xe9, 0x5c, 0xdc, 0xf9, 0xe5, 0x79, 0x70, 0xef, 0x32, 0x51, 0xd5, 0xbb, 0x5c, 0x60, 0xa3,
+ 0x21, 0xfe, 0x35, 0xe3, 0x62, 0x2f, 0x4b, 0x2d, 0x2a, 0xce, 0xcc, 0xcf, 0x93, 0x60, 0x06, 0x7b,
+ 0x56, 0x06, 0x6a, 0x89, 0x08, 0x8a, 0xcf, 0xc2, 0x20, 0x6a, 0x82, 0x60, 0x8a, 0x21, 0x21, 0xe0,
+ 0xa4, 0x71, 0xe2, 0xa1, 0x1c, 0xc3, 0x89, 0x47, 0x72, 0x8c, 0x17, 0x1e, 0xc9, 0x31, 0xde, 0x78,
+ 0x24, 0xc7, 0xf8, 0xe0, 0x91, 0x1c, 0xe3, 0x84, 0xc7, 0x72, 0x0c, 0x17, 0x1e, 0xcb, 0x31, 0xdc,
+ 0x78, 0x2c, 0xc7, 0x10, 0xc5, 0x06, 0x49, 0x0e, 0x80, 0x00, 0x00, 0x00, 0xff, 0xff, 0xcc, 0x67,
+ 0x63, 0xa3, 0x23, 0x02, 0x00, 0x00,
}
diff --git a/pkg/sql/catalog/descpb/privilege.proto b/pkg/sql/catalog/descpb/privilege.proto
index 3a6e18572ce3..bd1a1c02951f 100644
--- a/pkg/sql/catalog/descpb/privilege.proto
+++ b/pkg/sql/catalog/descpb/privilege.proto
@@ -17,7 +17,8 @@ import "gogoproto/gogo.proto";
// UserPrivileges describes the list of privileges available for a given user.
message UserPrivileges {
option (gogoproto.equal) = true;
- optional string user = 1 [(gogoproto.nullable) = false];
+ optional string user_proto = 1 [(gogoproto.nullable) = false,
+ (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"];
// privileges is a bitfield of 1< 63 {
- return "", errors.WithHint(pgerror.Newf(pgcode.NameTooLong, "username %q is too long", username), usernameHelp)
- }
- return username, nil
+ return username, errors.Wrapf(err, "%q", input)
}
var errNoUserNameSpecified = errors.New("no username specified")
@@ -295,17 +288,17 @@ func (p *planner) getUserAuthInfo(
}
// resolveUsername returns the actual user name.
-func (ua *userNameInfo) resolveUsername() (string, error) {
+func (ua *userNameInfo) resolveUsername() (res security.SQLUsername, err error) {
name, err := ua.name()
if err != nil {
- return "", err
+ return res, err
}
if name == "" {
- return "", errNoUserNameSpecified
+ return res, errNoUserNameSpecified
}
normalizedUsername, err := NormalizeAndValidateUsername(name)
if err != nil {
- return "", err
+ return res, err
}
return normalizedUsername, nil
diff --git a/pkg/sql/create_schema.go b/pkg/sql/create_schema.go
index 98352e6edeb5..3e8b6a18abca 100644
--- a/pkg/sql/create_schema.go
+++ b/pkg/sql/create_schema.go
@@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -61,7 +62,7 @@ func (p *planner) createUserDefinedSchema(params runParams, n *tree.CreateSchema
schemaName := string(n.Schema)
if n.Schema == "" {
- schemaName = n.AuthRole
+ schemaName = n.AuthRole.Normalized()
}
// Ensure there aren't any name collisions.
@@ -102,7 +103,7 @@ func (p *planner) createUserDefinedSchema(params runParams, n *tree.CreateSchema
privs.Users[i].Privileges &= privilege.SchemaPrivileges.ToBitField()
}
- if n.AuthRole != "" {
+ if !n.AuthRole.Undefined() {
exists, err := p.RoleExists(params.ctx, n.AuthRole)
if err != nil {
return err
@@ -112,7 +113,7 @@ func (p *planner) createUserDefinedSchema(params runParams, n *tree.CreateSchema
}
privs.SetOwner(n.AuthRole)
} else {
- privs.SetOwner(params.SessionData().User)
+ privs.SetOwner(params.SessionData().User())
}
// Create the SchemaDescriptor.
@@ -159,9 +160,9 @@ func (p *planner) createUserDefinedSchema(params runParams, n *tree.CreateSchema
int32(params.extendedEvalCtx.NodeID.SQLInstanceID()),
struct {
SchemaName string
- Owner string
- User string
- }{schemaName, privs.Owner, params.SessionData().User},
+ Owner security.SQLUsername
+ User security.SQLUsername
+ }{schemaName, privs.Owner(), params.SessionData().User()},
)
}
diff --git a/pkg/sql/create_sequence.go b/pkg/sql/create_sequence.go
index a574da2d94c0..ef827e79cde8 100644
--- a/pkg/sql/create_sequence.go
+++ b/pkg/sql/create_sequence.go
@@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
@@ -90,7 +91,7 @@ func doCreateSequence(
return err
}
- privs := CreateInheritedPrivilegesFromDBDesc(dbDesc, params.SessionData().User)
+ privs := CreateInheritedPrivilegesFromDBDesc(dbDesc, params.SessionData().User())
if persistence.IsTemporary() {
telemetry.Inc(sqltelemetry.CreateTempSequenceCounter)
@@ -158,8 +159,8 @@ func doCreateSequence(
struct {
SequenceName string
Statement string
- User string
- }{name.FQString(), context, params.SessionData().User},
+ User security.SQLUsername
+ }{name.FQString(), context, params.SessionData().User()},
)
}
diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go
index e2ca71d2560e..6c6d3b0b98f0 100644
--- a/pkg/sql/create_table.go
+++ b/pkg/sql/create_table.go
@@ -264,7 +264,7 @@ func (n *createTableNode) startExec(params runParams) error {
return err
}
- privs := CreateInheritedPrivilegesFromDBDesc(n.dbDesc, params.SessionData().User)
+ privs := CreateInheritedPrivilegesFromDBDesc(n.dbDesc, params.SessionData().User())
var asCols colinfo.ResultColumns
var desc *tabledesc.Mutable
@@ -371,8 +371,8 @@ func (n *createTableNode) startExec(params runParams) error {
struct {
TableName string
Statement string
- User string
- }{n.n.Table.FQString(), n.n.String(), params.SessionData().User},
+ User security.SQLUsername
+ }{n.n.Table.FQString(), n.n.String(), params.SessionData().User()},
); err != nil {
return err
}
@@ -2093,12 +2093,12 @@ func incTelemetryForNewColumn(def *tree.ColumnTableDef, desc *descpb.ColumnDescr
// CreateInheritedPrivilegesFromDBDesc creates privileges with the appropriate
// owner (node for system, the restoring user otherwise.)
func CreateInheritedPrivilegesFromDBDesc(
- dbDesc catalog.DatabaseDescriptor, user string,
+ dbDesc catalog.DatabaseDescriptor, user security.SQLUsername,
) *descpb.PrivilegeDescriptor {
// If a new system table is being created (which should only be doable by
// an internal user account), make sure it gets the correct privileges.
if dbDesc.GetID() == keys.SystemDatabaseID {
- return descpb.NewDefaultPrivilegeDescriptor(security.NodeUser)
+ return descpb.NewDefaultPrivilegeDescriptor(security.NodeUserName())
}
privs := dbDesc.GetPrivileges()
diff --git a/pkg/sql/create_type.go b/pkg/sql/create_type.go
index 790ea16a74e0..566630c895bf 100644
--- a/pkg/sql/create_type.go
+++ b/pkg/sql/create_type.go
@@ -338,7 +338,7 @@ func (p *planner) createEnum(params runParams, n *tree.CreateType) error {
struct {
TypeName string
Statement string
- User string
+ User security.SQLUsername
}{typeName.FQString(), tree.AsStringWithFQNames(n, params.Ann()), p.User()},
)
}
@@ -359,7 +359,7 @@ func inheritUsagePrivilegeFromSchema(
switch resolvedSchema.Kind {
case catalog.SchemaPublic:
// If the type is in the public schema, the public role has USAGE on it.
- privs.Grant(security.PublicRole, privilege.List{privilege.USAGE})
+ privs.Grant(security.PublicRoleName(), privilege.List{privilege.USAGE})
case catalog.SchemaTemporary, catalog.SchemaVirtual:
// No types should be created in a temporary schema or a virtual schema.
panic(errors.AssertionFailedf(
@@ -373,7 +373,7 @@ func inheritUsagePrivilegeFromSchema(
// privilege descriptor.
for _, u := range schemaPrivs.Users {
if u.Privileges&privilege.USAGE.Mask() == 1 {
- privs.Grant(u.User, privilege.List{privilege.USAGE})
+ privs.Grant(u.User(), privilege.List{privilege.USAGE})
}
}
default:
diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go
index fccf68df522c..9c6ac2c6105b 100644
--- a/pkg/sql/create_view.go
+++ b/pkg/sql/create_view.go
@@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
@@ -136,7 +137,7 @@ func (n *createViewNode) startExec(params runParams) error {
telemetry.Inc(sqltelemetry.CreateTempViewCounter)
}
- privs := CreateInheritedPrivilegesFromDBDesc(n.dbDesc, params.SessionData().User)
+ privs := CreateInheritedPrivilegesFromDBDesc(n.dbDesc, params.SessionData().User())
var newDesc *tabledesc.Mutable
@@ -266,11 +267,11 @@ func (n *createViewNode) startExec(params runParams) error {
struct {
ViewName string
ViewQuery string
- User string
+ User security.SQLUsername
}{
ViewName: n.viewName.FQString(),
ViewQuery: n.viewQuery,
- User: params.SessionData().User,
+ User: params.SessionData().User(),
},
)
}
diff --git a/pkg/sql/descriptor.go b/pkg/sql/descriptor.go
index 66a069eccbff..de693ef73153 100644
--- a/pkg/sql/descriptor.go
+++ b/pkg/sql/descriptor.go
@@ -86,7 +86,7 @@ func (p *planner) createDatabase(
return nil, false, err
}
- desc := dbdesc.NewInitial(id, string(database.Name), p.SessionData().User)
+ desc := dbdesc.NewInitial(id, string(database.Name), p.SessionData().User())
if err := p.createDescriptorWithID(ctx, dKey.Key(p.ExecCfg().Codec), id, desc, nil, jobDesc); err != nil {
return nil, true, err
}
diff --git a/pkg/sql/distsql_plan_csv.go b/pkg/sql/distsql_plan_csv.go
index f5fa65b0cc46..ed2a7e43a92d 100644
--- a/pkg/sql/distsql_plan_csv.go
+++ b/pkg/sql/distsql_plan_csv.go
@@ -20,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
@@ -110,7 +111,7 @@ func makeImportReaderSpecs(
format roachpb.IOFileFormat,
nodes []roachpb.NodeID,
walltime int64,
- user string,
+ user security.SQLUsername,
) []*execinfrapb.ReadImportDataSpec {
// For each input file, assign it to a node.
@@ -131,7 +132,7 @@ func makeImportReaderSpecs(
WalltimeNanos: walltime,
Uri: make(map[int32]string),
ResumePos: make(map[int32]int64),
- User: user,
+ UserProto: user.EncodeProto(),
}
inputSpecs = append(inputSpecs, spec)
}
diff --git a/pkg/sql/distsql_running_test.go b/pkg/sql/distsql_running_test.go
index 3acf43c14c62..d7559debcbd9 100644
--- a/pkg/sql/distsql_running_test.go
+++ b/pkg/sql/distsql_running_test.go
@@ -62,7 +62,7 @@ func TestDistSQLRunningInAbortedTxn(t *testing.T) {
internalPlanner, cleanup := NewInternalPlanner(
"test",
kv.NewTxn(ctx, db, s.NodeID()),
- security.RootUser,
+ security.RootUserName(),
&MemoryMetrics{},
&execCfg,
)
diff --git a/pkg/sql/doctor/doctor_test.go b/pkg/sql/doctor/doctor_test.go
index 800cdc584d89..09843cd68348 100644
--- a/pkg/sql/doctor/doctor_test.go
+++ b/pkg/sql/doctor/doctor_test.go
@@ -53,7 +53,7 @@ var validTableDesc = &descpb.Descriptor{
},
NextIndexID: 2,
Privileges: descpb.NewCustomSuperuserPrivilegeDescriptor(
- descpb.SystemAllowedPrivileges[keys.SqllivenessID], security.NodeUser),
+ descpb.SystemAllowedPrivileges[keys.SqllivenessID], security.NodeUserName()),
FormatVersion: descpb.InterleavedFormatVersion,
NextMutationID: 1,
},
diff --git a/pkg/sql/drop_database.go b/pkg/sql/drop_database.go
index 6f5383c9aa5c..8885e7d65e90 100644
--- a/pkg/sql/drop_database.go
+++ b/pkg/sql/drop_database.go
@@ -195,9 +195,9 @@ func (n *dropDatabaseNode) startExec(params runParams) error {
struct {
DatabaseName string
Statement string
- User string
+ User security.SQLUsername
DroppedSchemaObjects []string
- }{n.n.Name.String(), n.n.String(), p.SessionData().User, n.d.droppedNames},
+ }{n.n.Name.String(), n.n.String(), p.SessionData().User(), n.d.droppedNames},
)
}
@@ -308,7 +308,7 @@ func (p *planner) removeDbComment(ctx context.Context, dbID descpb.ID) error {
ctx,
"delete-db-comment",
p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"DELETE FROM system.comments WHERE type=$1 AND object_id=$2 AND sub_id=0",
keys.DatabaseCommentType,
dbID)
diff --git a/pkg/sql/drop_index.go b/pkg/sql/drop_index.go
index 8df4aa459db0..698e1e7622d7 100644
--- a/pkg/sql/drop_index.go
+++ b/pkg/sql/drop_index.go
@@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/config"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -534,10 +535,10 @@ func (p *planner) dropIndexByName(
TableName string
IndexName string
Statement string
- User string
+ User security.SQLUsername
MutationID uint32
CascadeDroppedViews []string
- }{tn.FQString(), string(idxName), jobDesc, p.SessionData().User, uint32(mutationID),
+ }{tn.FQString(), string(idxName), jobDesc, p.SessionData().User(), uint32(mutationID),
droppedViews},
)
}
diff --git a/pkg/sql/drop_role.go b/pkg/sql/drop_role.go
index 47af55994857..aa7a88d937cd 100644
--- a/pkg/sql/drop_role.go
+++ b/pkg/sql/drop_role.go
@@ -82,7 +82,7 @@ func (n *DropRoleNode) startExec(params runParams) error {
}
// userNames maps users to the objects they own
- userNames := make(map[string][]objectAndType)
+ userNames := make(map[security.SQLUsername][]objectAndType)
for i := range names {
name := names[i]
normalizedUsername, err := NormalizeAndValidateUsername(name)
@@ -91,7 +91,7 @@ func (n *DropRoleNode) startExec(params runParams) error {
}
// Update the name in the names slice since we will re-use the name later.
- names[i] = normalizedUsername
+ names[i] = normalizedUsername.Normalized()
userNames[normalizedUsername] = make([]objectAndType, 0)
}
@@ -102,7 +102,9 @@ func (n *DropRoleNode) startExec(params runParams) error {
}
if !hasAdmin {
for i := range names {
- targetIsAdmin, err := params.p.UserHasAdminRole(params.ctx, names[i])
+ // Normalized above already.
+ name := security.MakeSQLUsernameFromPreNormalizedString(names[i])
+ targetIsAdmin, err := params.p.UserHasAdminRole(params.ctx, name)
if err != nil {
return err
}
@@ -118,16 +120,16 @@ func (n *DropRoleNode) startExec(params runParams) error {
// First check all the databases.
if err := forEachDatabaseDesc(params.ctx, params.p, nil /*nil prefix = all databases*/, true, /* requiresPrivileges */
func(db *dbdesc.Immutable) error {
- if _, ok := userNames[db.GetPrivileges().Owner]; ok {
- userNames[db.GetPrivileges().Owner] = append(
- userNames[db.GetPrivileges().Owner],
+ if _, ok := userNames[db.GetPrivileges().Owner()]; ok {
+ userNames[db.GetPrivileges().Owner()] = append(
+ userNames[db.GetPrivileges().Owner()],
objectAndType{
ObjectType: "database",
ObjectName: db.GetName(),
})
}
for _, u := range db.GetPrivileges().Users {
- if _, ok := userNames[u.User]; ok {
+ if _, ok := userNames[u.User()]; ok {
if f.Len() > 0 {
f.WriteString(", ")
}
@@ -158,20 +160,20 @@ func (n *DropRoleNode) startExec(params runParams) error {
if !descriptorIsVisible(table, true /*allowAdding*/) {
continue
}
- if _, ok := userNames[table.GetPrivileges().Owner]; ok {
+ if _, ok := userNames[table.GetPrivileges().Owner()]; ok {
tn, err := getTableNameFromTableDescriptor(lCtx, table, "")
if err != nil {
return err
}
- userNames[table.GetPrivileges().Owner] = append(
- userNames[table.GetPrivileges().Owner],
+ userNames[table.GetPrivileges().Owner()] = append(
+ userNames[table.GetPrivileges().Owner()],
objectAndType{
ObjectType: "table",
ObjectName: tn.String(),
})
}
for _, u := range table.GetPrivileges().Users {
- if _, ok := userNames[u.User]; ok {
+ if _, ok := userNames[u.User()]; ok {
if f.Len() > 0 {
f.WriteString(", ")
}
@@ -189,9 +191,9 @@ func (n *DropRoleNode) startExec(params runParams) error {
// TODO(arul): Ideally this should be the fully qualified name of the schema,
// but at the time of writing there doesn't seem to be a clean way of doing
// this.
- if _, ok := userNames[schemaDesc.GetPrivileges().Owner]; ok {
- userNames[schemaDesc.GetPrivileges().Owner] = append(
- userNames[schemaDesc.GetPrivileges().Owner],
+ if _, ok := userNames[schemaDesc.GetPrivileges().Owner()]; ok {
+ userNames[schemaDesc.GetPrivileges().Owner()] = append(
+ userNames[schemaDesc.GetPrivileges().Owner()],
objectAndType{
ObjectType: "schema",
ObjectName: schemaDesc.GetName(),
@@ -199,7 +201,7 @@ func (n *DropRoleNode) startExec(params runParams) error {
}
}
for _, typDesc := range lCtx.typDescs {
- if _, ok := userNames[typDesc.GetPrivileges().Owner]; ok {
+ if _, ok := userNames[typDesc.GetPrivileges().Owner()]; ok {
if !descriptorIsVisible(typDesc, true /* allowAdding */) {
continue
}
@@ -207,8 +209,8 @@ func (n *DropRoleNode) startExec(params runParams) error {
if err != nil {
return err
}
- userNames[typDesc.GetPrivileges().Owner] = append(
- userNames[typDesc.GetPrivileges().Owner],
+ userNames[typDesc.GetPrivileges().Owner()] = append(
+ userNames[typDesc.GetPrivileges().Owner()],
objectAndType{
ObjectType: "type",
ObjectName: tn.String(),
@@ -233,7 +235,9 @@ func (n *DropRoleNode) startExec(params runParams) error {
)
}
- for _, name := range names {
+ for i := range names {
+ // Name already normalized above.
+ name := security.MakeSQLUsernameFromPreNormalizedString(names[i])
// Did the user own any objects?
ownedObjects := userNames[name]
if len(ownedObjects) > 0 {
@@ -253,11 +257,11 @@ func (n *DropRoleNode) startExec(params runParams) error {
for normalizedUsername := range userNames {
// Specifically reject special users and roles. Some (root, admin) would fail with
// "privileges still exist" first.
- if normalizedUsername == security.AdminRole || normalizedUsername == security.PublicRole {
+ if normalizedUsername.IsAdminRole() || normalizedUsername.IsPublicRole() {
return pgerror.Newf(
pgcode.InvalidParameterValue, "cannot drop special role %s", normalizedUsername)
}
- if normalizedUsername == security.RootUser {
+ if normalizedUsername.IsRootUser() {
return pgerror.Newf(
pgcode.InvalidParameterValue, "cannot drop special user %s", normalizedUsername)
}
diff --git a/pkg/sql/drop_schema.go b/pkg/sql/drop_schema.go
index 512bbf963298..73939d8c5798 100644
--- a/pkg/sql/drop_schema.go
+++ b/pkg/sql/drop_schema.go
@@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
@@ -150,8 +151,8 @@ func (n *dropSchemaNode) startExec(params runParams) error {
struct {
SchemaName string
Statement string
- User string
- }{sc.Name, n.n.String(), p.SessionData().User},
+ User security.SQLUsername
+ }{sc.Name, n.n.String(), p.SessionData().User()},
); err != nil {
return err
}
diff --git a/pkg/sql/drop_sequence.go b/pkg/sql/drop_sequence.go
index 6a18be026cfc..2c3d774f44b9 100644
--- a/pkg/sql/drop_sequence.go
+++ b/pkg/sql/drop_sequence.go
@@ -13,6 +13,7 @@ package sql
import (
"context"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -90,8 +91,8 @@ func (n *dropSequenceNode) startExec(params runParams) error {
struct {
SequenceName string
Statement string
- User string
- }{toDel.tn.FQString(), n.n.String(), params.SessionData().User},
+ User security.SQLUsername
+ }{toDel.tn.FQString(), n.n.String(), params.SessionData().User()},
); err != nil {
return err
}
diff --git a/pkg/sql/drop_table.go b/pkg/sql/drop_table.go
index 2e9b26522b11..e2fef9d995ce 100644
--- a/pkg/sql/drop_table.go
+++ b/pkg/sql/drop_table.go
@@ -136,10 +136,10 @@ func (n *dropTableNode) startExec(params runParams) error {
struct {
TableName string
Statement string
- User string
+ User security.SQLUsername
CascadeDroppedViews []string
}{toDel.tn.FQString(), n.n.String(),
- params.SessionData().User, droppedViews},
+ params.SessionData().User(), droppedViews},
); err != nil {
return err
}
@@ -629,7 +629,7 @@ func (p *planner) removeTableComments(ctx context.Context, tableDesc *tabledesc.
ctx,
"delete-table-comments",
p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"DELETE FROM system.comments WHERE object_id=$1",
tableDesc.ID)
if err != nil {
diff --git a/pkg/sql/drop_test.go b/pkg/sql/drop_test.go
index 2bb55c7ae0db..76e24efc5730 100644
--- a/pkg/sql/drop_test.go
+++ b/pkg/sql/drop_test.go
@@ -200,7 +200,7 @@ INSERT INTO t.kv VALUES ('c', 'e'), ('a', 'c'), ('b', 'd');
// TODO (lucy): Maybe this test API should use an offset starting
// from the most recent job instead.
if err := jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: "DROP DATABASE t CASCADE",
DescriptorIDs: descpb.IDs{
tbDesc.ID,
@@ -316,7 +316,7 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a');
const migrationJobOffset = 0
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
if err := jobutils.VerifySystemJob(t, sqlRun, migrationJobOffset, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: "DROP DATABASE t CASCADE",
DescriptorIDs: descpb.IDs{
tbDesc.ID, tb2Desc.ID,
@@ -350,7 +350,7 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a');
testutils.SucceedsSoon(t, func() error {
return jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChangeGC, jobs.StatusRunning, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: "GC for DROP DATABASE t CASCADE",
DescriptorIDs: descpb.IDs{
tbDesc.ID, tb2Desc.ID,
@@ -377,7 +377,7 @@ INSERT INTO t.kv2 VALUES ('c', 'd'), ('a', 'b'), ('e', 'a');
tests.CheckKeyCount(t, kvDB, table2Span, 0)
if err := jobutils.VerifySystemJob(t, sqlRun, migrationJobOffset, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: "DROP DATABASE t CASCADE",
DescriptorIDs: descpb.IDs{
tbDesc.ID, tb2Desc.ID,
@@ -448,7 +448,7 @@ func TestDropIndex(t *testing.T) {
const migrationJobOffset = 0
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
if err := jobutils.VerifySystemJob(t, sqlRun, migrationJobOffset+1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: `DROP INDEX t.public.kv@foo`,
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -478,7 +478,7 @@ func TestDropIndex(t *testing.T) {
testutils.SucceedsSoon(t, func() error {
return jobutils.VerifySystemJob(t, sqlRun, migrationJobOffset+1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: `DROP INDEX t.public.kv@foo`,
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -488,7 +488,7 @@ func TestDropIndex(t *testing.T) {
testutils.SucceedsSoon(t, func() error {
return jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: `GC for DROP INDEX t.public.kv@foo`,
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -663,7 +663,7 @@ func TestDropTable(t *testing.T) {
// Job still running, waiting for GC.
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
if err := jobutils.VerifySystemJob(t, sqlRun, 1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: `DROP TABLE t.public.kv`,
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -749,7 +749,7 @@ func TestDropTableDeleteData(t *testing.T) {
tests.CheckKeyCount(t, kvDB, tableSpan, numKeys)
if err := jobutils.VerifySystemJob(t, sqlRun, 2*i+1+migrationJobOffset, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf(`DROP TABLE t.public.%s`, descs[i].GetName()),
DescriptorIDs: descpb.IDs{
descs[i].ID,
@@ -779,7 +779,7 @@ func TestDropTableDeleteData(t *testing.T) {
// Ensure that the job is marked as succeeded.
if err := jobutils.VerifySystemJob(t, sqlRun, 2*i+1+migrationJobOffset, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf(`DROP TABLE t.public.%s`, descs[i].GetName()),
DescriptorIDs: descpb.IDs{
descs[i].ID,
@@ -791,7 +791,7 @@ func TestDropTableDeleteData(t *testing.T) {
// Ensure that the job is marked as succeeded.
testutils.SucceedsSoon(t, func() error {
return jobutils.VerifySystemJob(t, sqlRun, i, jobspb.TypeSchemaChangeGC, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf(`GC for DROP TABLE t.public.%s`, descs[i].GetName()),
DescriptorIDs: descpb.IDs{
descs[i].ID,
@@ -1035,7 +1035,7 @@ func TestDropDatabaseAfterDropTable(t *testing.T) {
if err := jobutils.VerifySystemJob(
t, sqlRun, 1, jobspb.TypeSchemaChange, jobs.StatusSucceeded,
jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: "DROP TABLE t.public.kv",
DescriptorIDs: descpb.IDs{
tableDesc.ID,
diff --git a/pkg/sql/drop_type.go b/pkg/sql/drop_type.go
index ad6d224531a3..0b319becc723 100644
--- a/pkg/sql/drop_type.go
+++ b/pkg/sql/drop_type.go
@@ -14,6 +14,7 @@ import (
"context"
"fmt"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
@@ -134,8 +135,8 @@ func (n *dropTypeNode) startExec(params runParams) error {
struct {
TypeName string
Statement string
- User string
- }{typ.Name, tree.AsStringWithFQNames(n.n, params.Ann()), params.SessionData().User},
+ User security.SQLUsername
+ }{typ.Name, tree.AsStringWithFQNames(n.n, params.Ann()), params.SessionData().User()},
); err != nil {
return err
}
diff --git a/pkg/sql/drop_view.go b/pkg/sql/drop_view.go
index 40e3829fe187..38c09e23289e 100644
--- a/pkg/sql/drop_view.go
+++ b/pkg/sql/drop_view.go
@@ -14,6 +14,7 @@ import (
"context"
"fmt"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc"
@@ -109,9 +110,9 @@ func (n *dropViewNode) startExec(params runParams) error {
struct {
ViewName string
Statement string
- User string
+ User security.SQLUsername
CascadeDroppedViews []string
- }{toDel.tn.FQString(), n.n.String(), params.SessionData().User, cascadeDroppedViews},
+ }{toDel.tn.FQString(), n.n.String(), params.SessionData().User(), cascadeDroppedViews},
); err != nil {
return err
}
diff --git a/pkg/sql/event_log.go b/pkg/sql/event_log.go
index 3a9b3b300323..307b22a94bcd 100644
--- a/pkg/sql/event_log.go
+++ b/pkg/sql/event_log.go
@@ -15,6 +15,7 @@ import (
"encoding/json"
"github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)
@@ -140,7 +141,7 @@ const (
type EventLogSetClusterSettingDetail struct {
SettingName string
Value string
- User string
+ User security.SQLUsername
}
// An EventLogger exposes methods used to record events to the event table.
diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go
index c37d0c2ae782..fe0bba394b94 100644
--- a/pkg/sql/exec_util.go
+++ b/pkg/sql/exec_util.go
@@ -34,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/protectedts"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/rpc"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/status/statuspb"
"github.com/cockroachdb/cockroach/pkg/settings"
@@ -975,6 +976,7 @@ func golangFillQueryArguments(args ...interface{}) (tree.Datums, error) {
// A type switch to handle a few explicit types with special semantics:
// - Datums are passed along as is.
// - Time datatypes get special representation in the database.
+ // - Usernames are assumed pre-normalized for lookup and validation.
var d tree.Datum
switch t := arg.(type) {
case tree.Datum:
@@ -993,6 +995,8 @@ func golangFillQueryArguments(args ...interface{}) (tree.Datums, error) {
dd := &tree.DDecimal{}
dd.Set(t)
d = dd
+ case security.SQLUsername:
+ d = tree.NewDString(t.Normalized())
}
if d == nil {
// Handle all types which have an underlying type that can be stored in the
@@ -1239,7 +1243,7 @@ type SessionDefaults map[string]string
// SessionArgs contains arguments for serving a client connection.
type SessionArgs struct {
- User string
+ User security.SQLUsername
SessionDefaults SessionDefaults
// RemoteAddr is the client's address. This is nil iff this is an internal
// client.
@@ -1273,7 +1277,7 @@ func (r *SessionRegistry) deregister(id ClusterWideID) {
}
type registrySession interface {
- user() string
+ user() security.SQLUsername
cancelQuery(queryID ClusterWideID) bool
cancelSession()
// serialize serializes a Session into a serverpb.Session
diff --git a/pkg/sql/execinfrapb/api.go b/pkg/sql/execinfrapb/api.go
index bc97a2954719..0c79db7f9b99 100644
--- a/pkg/sql/execinfrapb/api.go
+++ b/pkg/sql/execinfrapb/api.go
@@ -11,6 +11,7 @@
package execinfrapb
import (
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
@@ -41,3 +42,33 @@ func MakeEvalContext(evalCtx *tree.EvalContext) EvalContext {
TxnTimestampNanos: evalCtx.TxnTimestamp.UnixNano(),
}
}
+
+// User accesses the user field.
+func (e EvalContext) User() security.SQLUsername {
+ return e.SessionData.User()
+}
+
+// User accesses the user field.
+func (m *BackupDataSpec) User() security.SQLUsername {
+ return m.UserProto.Decode()
+}
+
+// User accesses the user field.
+func (m *CSVWriterSpec) User() security.SQLUsername {
+ return m.UserProto.Decode()
+}
+
+// User accesses the user field.
+func (m *ReadImportDataSpec) User() security.SQLUsername {
+ return m.UserProto.Decode()
+}
+
+// User accesses the user field.
+func (m *ChangeAggregatorSpec) User() security.SQLUsername {
+ return m.UserProto.Decode()
+}
+
+// User accesses the user field.
+func (m *ChangeFrontierSpec) User() security.SQLUsername {
+ return m.UserProto.Decode()
+}
diff --git a/pkg/sql/execinfrapb/processors_bulk_io.pb.go b/pkg/sql/execinfrapb/processors_bulk_io.pb.go
index 18b2e790fe1d..be5d0a09a07b 100644
--- a/pkg/sql/execinfrapb/processors_bulk_io.pb.go
+++ b/pkg/sql/execinfrapb/processors_bulk_io.pb.go
@@ -17,6 +17,7 @@ import descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc"
import time "time"
+import github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security"
import github_com_gogo_protobuf_sortkeys "github.com/gogo/protobuf/sortkeys"
import encoding_binary "encoding/binary"
@@ -70,7 +71,7 @@ func (x *FileCompression) UnmarshalJSON(data []byte) error {
return nil
}
func (FileCompression) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{0}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{0}
}
type BackfillerSpec_Type int32
@@ -109,7 +110,7 @@ func (x *BackfillerSpec_Type) UnmarshalJSON(data []byte) error {
return nil
}
func (BackfillerSpec_Type) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{0, 0}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{0, 0}
}
// BackfillerSpec is the specification for a "schema change backfiller".
@@ -141,7 +142,7 @@ func (m *BackfillerSpec) Reset() { *m = BackfillerSpec{} }
func (m *BackfillerSpec) String() string { return proto.CompactTextString(m) }
func (*BackfillerSpec) ProtoMessage() {}
func (*BackfillerSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{0}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{0}
}
func (m *BackfillerSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -181,7 +182,7 @@ func (m *JobProgress) Reset() { *m = JobProgress{} }
func (m *JobProgress) String() string { return proto.CompactTextString(m) }
func (*JobProgress) ProtoMessage() {}
func (*JobProgress) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{1}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{1}
}
func (m *JobProgress) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -237,14 +238,14 @@ type ReadImportDataSpec struct {
ReaderParallelism int32 `protobuf:"varint,13,opt,name=readerParallelism" json:"readerParallelism"`
// User who initiated the import. This is used to check access privileges
// when using FileTable ExternalStorage.
- User string `protobuf:"bytes,15,opt,name=user" json:"user"`
+ UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,15,opt,name=user_proto,json=userProto,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto"`
}
func (m *ReadImportDataSpec) Reset() { *m = ReadImportDataSpec{} }
func (m *ReadImportDataSpec) String() string { return proto.CompactTextString(m) }
func (*ReadImportDataSpec) ProtoMessage() {}
func (*ReadImportDataSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{2}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{2}
}
func (m *ReadImportDataSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -282,7 +283,7 @@ func (m *ReadImportDataSpec_ImportTable) Reset() { *m = ReadImportDataSp
func (m *ReadImportDataSpec_ImportTable) String() string { return proto.CompactTextString(m) }
func (*ReadImportDataSpec_ImportTable) ProtoMessage() {}
func (*ReadImportDataSpec_ImportTable) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{2, 0}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{2, 0}
}
func (m *ReadImportDataSpec_ImportTable) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -321,14 +322,14 @@ type BackupDataSpec struct {
PKIDs map[uint64]bool `protobuf:"bytes,9,rep,name=pk_ids,json=pkIds" json:"pk_ids,omitempty" protobuf_key:"varint,1,opt,name=key" protobuf_val:"varint,2,opt,name=value"`
// User who initiated the backup. This is used to check access privileges
// when using FileTable ExternalStorage.
- User string `protobuf:"bytes,10,opt,name=user" json:"user"`
+ UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,10,opt,name=user_proto,json=userProto,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto"`
}
func (m *BackupDataSpec) Reset() { *m = BackupDataSpec{} }
func (m *BackupDataSpec) String() string { return proto.CompactTextString(m) }
func (*BackupDataSpec) ProtoMessage() {}
func (*BackupDataSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{3}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{3}
}
func (m *BackupDataSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -367,7 +368,7 @@ func (m *RestoreSpanEntry) Reset() { *m = RestoreSpanEntry{} }
func (m *RestoreSpanEntry) String() string { return proto.CompactTextString(m) }
func (*RestoreSpanEntry) ProtoMessage() {}
func (*RestoreSpanEntry) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{4}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{4}
}
func (m *RestoreSpanEntry) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -405,7 +406,7 @@ func (m *RestoreDataSpec) Reset() { *m = RestoreDataSpec{} }
func (m *RestoreDataSpec) String() string { return proto.CompactTextString(m) }
func (*RestoreDataSpec) ProtoMessage() {}
func (*RestoreDataSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{5}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{5}
}
func (m *RestoreDataSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -439,7 +440,7 @@ func (m *SplitAndScatterSpec) Reset() { *m = SplitAndScatterSpec{} }
func (m *SplitAndScatterSpec) String() string { return proto.CompactTextString(m) }
func (*SplitAndScatterSpec) ProtoMessage() {}
func (*SplitAndScatterSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{6}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{6}
}
func (m *SplitAndScatterSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -472,7 +473,7 @@ func (m *SplitAndScatterSpec_RestoreEntryChunk) Reset() { *m = SplitAndS
func (m *SplitAndScatterSpec_RestoreEntryChunk) String() string { return proto.CompactTextString(m) }
func (*SplitAndScatterSpec_RestoreEntryChunk) ProtoMessage() {}
func (*SplitAndScatterSpec_RestoreEntryChunk) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{6, 0}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{6, 0}
}
func (m *SplitAndScatterSpec_RestoreEntryChunk) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -512,14 +513,14 @@ type CSVWriterSpec struct {
CompressionCodec FileCompression `protobuf:"varint,5,opt,name=compression_codec,json=compressionCodec,enum=cockroach.sql.distsqlrun.FileCompression" json:"compression_codec"`
// User who initiated the export. This is used to check access privileges
// when using FileTable ExternalStorage.
- User string `protobuf:"bytes,6,opt,name=user" json:"user"`
+ UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,6,opt,name=user_proto,json=userProto,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto"`
}
func (m *CSVWriterSpec) Reset() { *m = CSVWriterSpec{} }
func (m *CSVWriterSpec) String() string { return proto.CompactTextString(m) }
func (*CSVWriterSpec) ProtoMessage() {}
func (*CSVWriterSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{7}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{7}
}
func (m *CSVWriterSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -554,7 +555,7 @@ func (m *BulkRowWriterSpec) Reset() { *m = BulkRowWriterSpec{} }
func (m *BulkRowWriterSpec) String() string { return proto.CompactTextString(m) }
func (*BulkRowWriterSpec) ProtoMessage() {}
func (*BulkRowWriterSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_bulk_io_2d25bde6e23c12db, []int{8}
+ return fileDescriptor_processors_bulk_io_e62f5714e36bbbd6, []int{8}
}
func (m *BulkRowWriterSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -807,8 +808,8 @@ func (m *ReadImportDataSpec) MarshalTo(dAtA []byte) (int, error) {
}
dAtA[i] = 0x7a
i++
- i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(len(m.User)))
- i += copy(dAtA[i:], m.User)
+ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(len(m.UserProto)))
+ i += copy(dAtA[i:], m.UserProto)
return i, nil
}
@@ -976,8 +977,8 @@ func (m *BackupDataSpec) MarshalTo(dAtA []byte) (int, error) {
}
dAtA[i] = 0x52
i++
- i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(len(m.User)))
- i += copy(dAtA[i:], m.User)
+ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(len(m.UserProto)))
+ i += copy(dAtA[i:], m.UserProto)
return i, nil
}
@@ -1206,8 +1207,8 @@ func (m *CSVWriterSpec) MarshalTo(dAtA []byte) (int, error) {
i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(m.CompressionCodec))
dAtA[i] = 0x32
i++
- i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(len(m.User)))
- i += copy(dAtA[i:], m.User)
+ i = encodeVarintProcessorsBulkIo(dAtA, i, uint64(len(m.UserProto)))
+ i += copy(dAtA[i:], m.UserProto)
return i, nil
}
@@ -1323,7 +1324,7 @@ func (m *ReadImportDataSpec) Size() (n int) {
n += mapEntrySize + 1 + sovProcessorsBulkIo(uint64(mapEntrySize))
}
}
- l = len(m.User)
+ l = len(m.UserProto)
n += 1 + l + sovProcessorsBulkIo(uint64(l))
return n
}
@@ -1392,7 +1393,7 @@ func (m *BackupDataSpec) Size() (n int) {
n += mapEntrySize + 1 + sovProcessorsBulkIo(uint64(mapEntrySize))
}
}
- l = len(m.User)
+ l = len(m.UserProto)
n += 1 + l + sovProcessorsBulkIo(uint64(l))
return n
}
@@ -1494,7 +1495,7 @@ func (m *CSVWriterSpec) Size() (n int) {
n += 1 + l + sovProcessorsBulkIo(uint64(l))
n += 1 + sovProcessorsBulkIo(uint64(m.ChunkRows))
n += 1 + sovProcessorsBulkIo(uint64(m.CompressionCodec))
- l = len(m.User)
+ l = len(m.UserProto)
n += 1 + l + sovProcessorsBulkIo(uint64(l))
return n
}
@@ -2314,7 +2315,7 @@ func (m *ReadImportDataSpec) Unmarshal(dAtA []byte) error {
iNdEx = postIndex
case 15:
if wireType != 2 {
- return fmt.Errorf("proto: wrong wireType = %d for field User", wireType)
+ return fmt.Errorf("proto: wrong wireType = %d for field UserProto", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
@@ -2339,7 +2340,7 @@ func (m *ReadImportDataSpec) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
- m.User = string(dAtA[iNdEx:postIndex])
+ m.UserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
@@ -2924,7 +2925,7 @@ func (m *BackupDataSpec) Unmarshal(dAtA []byte) error {
iNdEx = postIndex
case 10:
if wireType != 2 {
- return fmt.Errorf("proto: wrong wireType = %d for field User", wireType)
+ return fmt.Errorf("proto: wrong wireType = %d for field UserProto", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
@@ -2949,7 +2950,7 @@ func (m *BackupDataSpec) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
- m.User = string(dAtA[iNdEx:postIndex])
+ m.UserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
@@ -3694,7 +3695,7 @@ func (m *CSVWriterSpec) Unmarshal(dAtA []byte) error {
}
case 6:
if wireType != 2 {
- return fmt.Errorf("proto: wrong wireType = %d for field User", wireType)
+ return fmt.Errorf("proto: wrong wireType = %d for field UserProto", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
@@ -3719,7 +3720,7 @@ func (m *CSVWriterSpec) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
- m.User = string(dAtA[iNdEx:postIndex])
+ m.UserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
@@ -3928,111 +3929,114 @@ var (
)
func init() {
- proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_2d25bde6e23c12db)
-}
-
-var fileDescriptor_processors_bulk_io_2d25bde6e23c12db = []byte{
- // 1622 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xa4, 0x57, 0xdd, 0x6e, 0x1b, 0xb9,
- 0x15, 0xf6, 0x48, 0x23, 0x59, 0x3e, 0x8a, 0xed, 0x31, 0x37, 0xbb, 0x9d, 0x1a, 0xa8, 0x6d, 0x68,
- 0xd7, 0xa9, 0x9a, 0x22, 0x12, 0x36, 0x69, 0x8b, 0xa0, 0xed, 0x36, 0x8d, 0xe5, 0x38, 0x2b, 0xa7,
- 0x9b, 0x18, 0xe3, 0xd8, 0x0b, 0x2c, 0x5a, 0x0c, 0xa8, 0x19, 0xda, 0x61, 0x34, 0x1a, 0x8e, 0x49,
- 0x8e, 0x13, 0xe5, 0xaa, 0x40, 0x5f, 0xa0, 0x8f, 0xd0, 0x37, 0x68, 0x1f, 0xa0, 0x0f, 0x90, 0xcb,
- 0xbd, 0xcc, 0x55, 0xd0, 0x3a, 0xef, 0xd0, 0x8b, 0xf6, 0xa6, 0xe0, 0xcf, 0xc8, 0xe3, 0xf8, 0x27,
- 0xf6, 0xe6, 0x46, 0x1e, 0xf3, 0xf0, 0xfb, 0x78, 0x78, 0x78, 0xce, 0x77, 0x48, 0x68, 0x8b, 0x83,
- 0xa4, 0x4b, 0x5e, 0x92, 0x88, 0xa6, 0x7b, 0x1c, 0x67, 0x83, 0x6e, 0xc6, 0x59, 0x44, 0x84, 0x60,
- 0x5c, 0x84, 0x83, 0x3c, 0x19, 0x86, 0x94, 0x75, 0x32, 0xce, 0x24, 0x43, 0x7e, 0xc4, 0xa2, 0x21,
- 0x67, 0x38, 0x7a, 0xd6, 0x11, 0x07, 0x49, 0x27, 0xa6, 0x42, 0x8a, 0x83, 0x84, 0xe7, 0xe9, 0xe2,
- 0x67, 0xcf, 0xd9, 0x40, 0x74, 0xd5, 0x4f, 0x36, 0xd0, 0x7f, 0x0c, 0x62, 0xd1, 0xd7, 0xb3, 0xb3,
- 0x41, 0x97, 0xb2, 0x5b, 0x7b, 0x8c, 0x8f, 0xb0, 0x2c, 0x2c, 0x9f, 0xab, 0x55, 0x23, 0x2c, 0x71,
- 0xc2, 0xf6, 0xbb, 0x31, 0x11, 0x51, 0x36, 0xe8, 0x0a, 0xc9, 0xf3, 0x48, 0xe6, 0x9c, 0xc4, 0x76,
- 0xd2, 0xea, 0x45, 0xae, 0x61, 0x41, 0x8a, 0x55, 0x72, 0x49, 0x93, 0xee, 0xb3, 0x24, 0xea, 0x4a,
- 0x3a, 0x22, 0x42, 0xe2, 0x51, 0x66, 0x2d, 0xd7, 0xf7, 0xd9, 0x3e, 0xd3, 0x9f, 0x5d, 0xf5, 0x65,
- 0x47, 0x51, 0xe1, 0x55, 0x8c, 0x25, 0xb6, 0x63, 0x0b, 0xc5, 0x18, 0xce, 0xa8, 0x19, 0x6a, 0xfd,
- 0xbd, 0x0a, 0x73, 0x6b, 0x38, 0x1a, 0xee, 0xd1, 0x24, 0x21, 0x7c, 0x3b, 0x23, 0x11, 0x7a, 0x08,
- 0xae, 0x1c, 0x67, 0xc4, 0x77, 0x56, 0x9c, 0xf6, 0xdc, 0xed, 0x5b, 0x9d, 0xf3, 0x02, 0xd2, 0x39,
- 0x89, 0xeb, 0x3c, 0x1d, 0x67, 0x64, 0xcd, 0x7d, 0xfd, 0x76, 0x79, 0x2a, 0xd0, 0x04, 0x68, 0x0d,
- 0x6a, 0x12, 0x0f, 0x12, 0xe2, 0x57, 0x56, 0x9c, 0x76, 0xf3, 0xf6, 0x8d, 0xf7, 0x98, 0xc4, 0x41,
- 0xa2, 0xf7, 0xf7, 0x54, 0xcd, 0x59, 0x27, 0x22, 0xe2, 0x34, 0x93, 0x8c, 0x5b, 0x0a, 0x03, 0x45,
- 0x0f, 0xa0, 0x26, 0x32, 0x9c, 0x0a, 0xbf, 0xba, 0x52, 0x6d, 0x37, 0x6f, 0xff, 0xec, 0x7c, 0x6f,
- 0x34, 0x4d, 0x40, 0x70, 0xac, 0xdc, 0xc1, 0x69, 0x41, 0xa3, 0xd1, 0xe8, 0x4b, 0x68, 0xc4, 0x39,
- 0xc7, 0x92, 0xb2, 0xd4, 0x77, 0x57, 0x9c, 0x76, 0x75, 0xed, 0x53, 0x65, 0xfe, 0xef, 0xdb, 0xe5,
- 0x59, 0x15, 0xce, 0xce, 0xba, 0x35, 0x06, 0x93, 0x69, 0xe8, 0x73, 0x80, 0xe8, 0x59, 0x9e, 0x0e,
- 0x43, 0x41, 0x5f, 0x11, 0xbf, 0xa6, 0x41, 0x86, 0x73, 0x46, 0x8f, 0x6f, 0xd3, 0x57, 0x04, 0xdd,
- 0x83, 0x06, 0x27, 0x38, 0xbe, 0x2f, 0x9e, 0xec, 0xf9, 0xd3, 0x7a, 0x97, 0x3f, 0x29, 0x79, 0xa8,
- 0x8e, 0xac, 0xf3, 0x2c, 0x89, 0x3a, 0x4f, 0x8b, 0x23, 0xb3, 0x0c, 0x13, 0x50, 0xeb, 0x26, 0xb8,
- 0x2a, 0x6e, 0xa8, 0x09, 0xd3, 0xfd, 0xf4, 0x10, 0x27, 0x34, 0xf6, 0xa6, 0x10, 0x40, 0xbd, 0xc7,
- 0x92, 0x7c, 0x94, 0x7a, 0x0e, 0x9a, 0x81, 0x5a, 0x3f, 0x8d, 0xc9, 0x4b, 0xaf, 0xb2, 0xe9, 0x36,
- 0xea, 0xde, 0x74, 0xeb, 0x05, 0x34, 0x37, 0xd9, 0x60, 0x8b, 0xb3, 0x7d, 0x4e, 0x84, 0x40, 0x5f,
- 0x40, 0xfd, 0x39, 0x1b, 0x84, 0x34, 0xd6, 0xe7, 0x55, 0x5d, 0x9b, 0x55, 0x0b, 0x1c, 0xbd, 0x5d,
- 0xae, 0x6d, 0xb2, 0x41, 0x7f, 0x3d, 0xa8, 0x3d, 0x67, 0x83, 0x7e, 0x8c, 0xda, 0x70, 0x2d, 0x62,
- 0xa9, 0xe4, 0x74, 0x90, 0xeb, 0x18, 0xa8, 0x13, 0xa9, 0x58, 0x67, 0x4e, 0x58, 0x90, 0x0f, 0xae,
- 0x48, 0x98, 0xf4, 0xab, 0x2b, 0x4e, 0xbb, 0x56, 0x1c, 0xa7, 0x1a, 0x69, 0xfd, 0x6f, 0x1a, 0x90,
- 0x8a, 0x6f, 0x7f, 0x94, 0x31, 0x2e, 0xd7, 0xb1, 0xc4, 0x3a, 0x5d, 0x56, 0xa1, 0x29, 0xf0, 0x28,
- 0x4b, 0x88, 0x09, 0x54, 0xa5, 0x84, 0x03, 0x63, 0xd0, 0x91, 0x7a, 0x08, 0x8d, 0xcc, 0xfa, 0xec,
- 0xd7, 0x75, 0xa4, 0x56, 0xcf, 0x3f, 0xcb, 0xd2, 0x06, 0x8b, 0x88, 0x15, 0x60, 0xf4, 0x10, 0xaa,
- 0x39, 0xa7, 0xfe, 0xb4, 0xce, 0x87, 0x5f, 0x9e, 0xcf, 0x71, 0xda, 0xd5, 0xce, 0x0e, 0xa7, 0x0f,
- 0x52, 0xc9, 0xc7, 0x81, 0x62, 0x40, 0x5f, 0x41, 0xdd, 0x94, 0xab, 0xdf, 0xd0, 0xfe, 0x2c, 0x97,
- 0xb8, 0x6c, 0xa1, 0x74, 0xfa, 0x4f, 0x36, 0x68, 0x42, 0x36, 0xf4, 0x34, 0xeb, 0x89, 0x05, 0xa1,
- 0x5d, 0xa8, 0xeb, 0x14, 0x15, 0xfe, 0x8c, 0x76, 0xe5, 0xee, 0x95, 0x5c, 0xd1, 0xd9, 0x2a, 0xb4,
- 0x37, 0x9a, 0xd7, 0x09, 0x2c, 0x1b, 0xba, 0x07, 0x3f, 0x16, 0x43, 0x9a, 0x85, 0x23, 0x2a, 0x04,
- 0x4d, 0xf7, 0xc3, 0x3d, 0xc6, 0x09, 0xdd, 0x4f, 0xc3, 0x21, 0x19, 0x0b, 0x1f, 0x56, 0x9c, 0x76,
- 0xc3, 0x3a, 0xf2, 0x99, 0x9a, 0xf6, 0x8d, 0x99, 0xb5, 0x61, 0x26, 0x3d, 0x22, 0x63, 0x81, 0x6e,
- 0xc2, 0xec, 0x0b, 0x9c, 0x24, 0x2a, 0xaf, 0x1f, 0xe3, 0x94, 0x09, 0xbf, 0x59, 0xca, 0xdd, 0x93,
- 0x26, 0x74, 0x1b, 0x16, 0xb8, 0x2e, 0x99, 0x2d, 0xcc, 0x71, 0x92, 0x90, 0x84, 0x8a, 0x91, 0x3f,
- 0x5b, 0x3a, 0xc2, 0xd3, 0x66, 0xf4, 0x1d, 0x00, 0x27, 0x22, 0x1f, 0x91, 0x30, 0x63, 0xc2, 0x9f,
- 0xd3, 0x9b, 0xff, 0xcd, 0x95, 0x36, 0x1f, 0x68, 0xf8, 0x16, 0x33, 0xfb, 0x0f, 0x66, 0x78, 0xf1,
- 0xbf, 0xca, 0xbe, 0x5c, 0x10, 0xee, 0xcf, 0xaf, 0x38, 0xed, 0x99, 0x22, 0xfb, 0xd4, 0xc8, 0x62,
- 0x0e, 0x4d, 0xc3, 0xa2, 0x23, 0x87, 0x7e, 0x0f, 0xae, 0x12, 0x54, 0x9d, 0xf4, 0x57, 0x93, 0x16,
- 0x27, 0xd0, 0x48, 0xf4, 0x05, 0x80, 0xc4, 0x7c, 0x9f, 0xc8, 0x1e, 0x4b, 0x84, 0x5f, 0x59, 0xa9,
- 0xda, 0x05, 0x9d, 0xa0, 0x34, 0xbe, 0x28, 0xa0, 0x59, 0x3a, 0x2a, 0xe4, 0x41, 0x75, 0x48, 0xc6,
- 0x7a, 0xd5, 0x99, 0x40, 0x7d, 0xa2, 0xc7, 0x50, 0x3b, 0xc4, 0x49, 0x5e, 0x88, 0xdc, 0xd5, 0xb2,
- 0xa0, 0xb4, 0xa3, 0xc0, 0xd0, 0xfc, 0xba, 0x72, 0xd7, 0x59, 0xfc, 0x15, 0x34, 0x8a, 0x54, 0x2d,
- 0xaf, 0x58, 0x33, 0x2b, 0x5e, 0x2f, 0xaf, 0x38, 0x53, 0xc6, 0xfd, 0x16, 0xe6, 0x4e, 0x86, 0xf6,
- 0x43, 0xe8, 0x6a, 0x09, 0xbd, 0xe9, 0x36, 0x1c, 0x2d, 0x32, 0x55, 0xcf, 0xdd, 0x74, 0x1b, 0xae,
- 0x57, 0xdb, 0x74, 0x1b, 0x35, 0xaf, 0xbe, 0xe9, 0x36, 0xae, 0x79, 0xb3, 0xad, 0xff, 0xd4, 0x4d,
- 0xa3, 0xc8, 0xb3, 0x49, 0xe5, 0xdf, 0x29, 0xb4, 0xd9, 0xd1, 0x39, 0xf0, 0xa3, 0x33, 0xea, 0xe7,
- 0xb4, 0x12, 0x7f, 0x0d, 0x1e, 0x4d, 0x25, 0x67, 0x71, 0x1e, 0x91, 0x38, 0x34, 0xf8, 0xca, 0x65,
- 0xf0, 0xf3, 0xc7, 0xb0, 0x6d, 0xcd, 0x74, 0x07, 0x9a, 0x31, 0xd9, 0xc3, 0x79, 0x22, 0x43, 0x25,
- 0x08, 0x55, 0x9d, 0x32, 0xc8, 0xca, 0x1f, 0xac, 0x1b, 0xd3, 0x4e, 0xd0, 0x0f, 0xc0, 0x4e, 0xdb,
- 0xe1, 0x14, 0xfd, 0xc5, 0x81, 0x4f, 0x72, 0x4e, 0x45, 0x38, 0x18, 0x87, 0x09, 0x8b, 0x70, 0x42,
- 0xe5, 0x38, 0x1c, 0x1e, 0xfa, 0xae, 0x76, 0xe1, 0x77, 0x17, 0x37, 0xbb, 0xe3, 0xbd, 0x2b, 0x29,
- 0x11, 0x6b, 0xe3, 0x3f, 0x58, 0x86, 0x47, 0x87, 0xa6, 0x92, 0xaf, 0x1f, 0xbd, 0x5d, 0xf6, 0x76,
- 0x82, 0x7e, 0xd9, 0xb4, 0x1b, 0x78, 0xf9, 0x7b, 0x93, 0x51, 0x00, 0xcd, 0xd1, 0x61, 0x14, 0x85,
- 0x7b, 0x34, 0x91, 0x84, 0xeb, 0xe6, 0x32, 0x77, 0xa2, 0x73, 0x14, 0xfb, 0xff, 0x66, 0xb7, 0xd7,
- 0xdb, 0xd0, 0x93, 0x8e, 0x77, 0x76, 0x3c, 0x16, 0x80, 0x62, 0x31, 0xdf, 0xe8, 0x6b, 0x00, 0x92,
- 0x46, 0x7c, 0x9c, 0x69, 0x81, 0x37, 0x12, 0xdb, 0x3e, 0x83, 0x52, 0x09, 0xda, 0x83, 0xc9, 0xc4,
- 0x27, 0xfa, 0x57, 0x04, 0x25, 0x2c, 0x7a, 0x02, 0x0b, 0x03, 0xbd, 0xdb, 0x50, 0x48, 0xcc, 0x65,
- 0xa8, 0xe4, 0xe2, 0x2a, 0xdd, 0x6d, 0xde, 0xa0, 0xb7, 0x15, 0x58, 0xd9, 0xd0, 0x23, 0xb0, 0x43,
- 0x21, 0x49, 0x63, 0x43, 0xd7, 0xb8, 0x3c, 0xdd, 0xac, 0xc1, 0x3e, 0x48, 0x63, 0x4d, 0xb6, 0x03,
- 0xf5, 0x6c, 0x18, 0xd2, 0xb8, 0xd0, 0xdd, 0x3b, 0x97, 0x3e, 0xb3, 0xad, 0x61, 0x3f, 0xb6, 0x92,
- 0x3b, 0xa3, 0x3a, 0xe4, 0xd6, 0xa3, 0xfe, 0xba, 0x08, 0x6a, 0x99, 0x1a, 0x9e, 0x28, 0x0f, 0x9c,
- 0x52, 0x9e, 0x1e, 0x7c, 0x7a, 0xe6, 0x69, 0x9f, 0x21, 0x06, 0xe7, 0x97, 0xe6, 0x5d, 0x80, 0xe3,
- 0xe5, 0xcb, 0x48, 0xf7, 0x0c, 0x64, 0xa3, 0x84, 0x6c, 0xfd, 0xc3, 0x01, 0x2f, 0x20, 0x42, 0x32,
- 0x4e, 0x54, 0xde, 0x1b, 0x82, 0x2f, 0xc1, 0x55, 0xa5, 0x63, 0xe5, 0xef, 0x03, 0x95, 0xa3, 0xa7,
- 0xa2, 0xfb, 0x50, 0xdb, 0xa3, 0xaa, 0x5d, 0x99, 0x6a, 0x5b, 0x3d, 0xab, 0xdb, 0x69, 0x39, 0x0a,
- 0xc8, 0x41, 0x4e, 0x84, 0xd4, 0x89, 0x52, 0xd4, 0xae, 0x46, 0xa2, 0x1b, 0xd0, 0x2c, 0xda, 0x70,
- 0x3f, 0x7e, 0xa9, 0x2b, 0xae, 0xe8, 0x2b, 0x65, 0x43, 0xeb, 0xcf, 0x55, 0x98, 0xb7, 0x2e, 0x4f,
- 0xc4, 0x62, 0x03, 0xae, 0x71, 0x33, 0x64, 0x12, 0xc0, 0xb9, 0x7c, 0x02, 0x34, 0x2d, 0x50, 0x1f,
- 0xff, 0xc9, 0x34, 0xaf, 0x7c, 0x44, 0x9a, 0xf7, 0xa1, 0xce, 0x89, 0xee, 0xaa, 0xe6, 0x6e, 0xf9,
- 0xf3, 0x0f, 0x46, 0xc4, 0x5e, 0x31, 0x87, 0x64, 0x5c, 0xdc, 0x05, 0x0c, 0x81, 0xba, 0x0b, 0xd8,
- 0x9c, 0x34, 0x3a, 0xf2, 0x8b, 0x8b, 0xba, 0xc0, 0x89, 0xb8, 0x5c, 0x98, 0x94, 0x1f, 0x91, 0x35,
- 0x7f, 0xab, 0xc0, 0x27, 0xdb, 0x59, 0x42, 0xe5, 0xfd, 0x34, 0xde, 0x8e, 0xb0, 0x94, 0xf6, 0x72,
- 0xff, 0x27, 0xa8, 0xeb, 0xdb, 0x6b, 0x21, 0xda, 0xf7, 0xce, 0xf7, 0xf4, 0x0c, 0x78, 0xe1, 0xbd,
- 0xf6, 0xa7, 0xa7, 0x78, 0x8a, 0x40, 0x18, 0xd2, 0x52, 0x4c, 0x2b, 0x1f, 0x19, 0xd3, 0xc5, 0x10,
- 0x16, 0x4e, 0xad, 0x86, 0x36, 0x61, 0x9a, 0xa8, 0xcb, 0x2a, 0x29, 0xfc, 0xbf, 0xf9, 0xc1, 0x48,
- 0x4f, 0x8a, 0xc6, 0xf2, 0x17, 0x04, 0xad, 0x7f, 0x56, 0x60, 0xb6, 0xb7, 0xbd, 0xfb, 0x2d, 0xa7,
- 0x45, 0x70, 0x6e, 0xa8, 0x8e, 0x22, 0x24, 0x4d, 0xcd, 0x43, 0xc1, 0x29, 0x49, 0x41, 0xd9, 0x80,
- 0x7e, 0x0a, 0xd7, 0x52, 0xac, 0xee, 0x3f, 0x3a, 0x30, 0x26, 0x0b, 0x27, 0x13, 0x95, 0x65, 0xcb,
- 0x18, 0xd0, 0x57, 0x30, 0xcd, 0x4c, 0xe6, 0xe9, 0x62, 0x69, 0x9e, 0xa9, 0xf1, 0xbd, 0xed, 0x5d,
- 0x9b, 0x9e, 0x85, 0x87, 0x16, 0x73, 0xfc, 0x04, 0xe1, 0xec, 0x85, 0xb0, 0xef, 0x96, 0xf2, 0x13,
- 0x24, 0x60, 0x2f, 0x04, 0xfa, 0x23, 0x2c, 0x44, 0x6c, 0x94, 0xa9, 0xda, 0xa3, 0x2c, 0x0d, 0x23,
- 0x16, 0x93, 0xc8, 0x76, 0x94, 0x0b, 0x5e, 0x4b, 0xaa, 0x3c, 0x7a, 0xc7, 0x30, 0x4b, 0xeb, 0x95,
- 0x98, 0x7a, 0x8a, 0x68, 0x22, 0x8b, 0xf5, 0xf7, 0x65, 0xb1, 0xf5, 0x2d, 0x2c, 0xac, 0xe5, 0x89,
- 0xf2, 0xa1, 0x14, 0xc1, 0xc9, 0x93, 0xcf, 0xf9, 0xc1, 0x4f, 0xbe, 0x9b, 0xab, 0x30, 0xff, 0x9e,
- 0x77, 0xa8, 0x01, 0xee, 0x63, 0x96, 0x12, 0x6f, 0x4a, 0x7d, 0x3d, 0x7c, 0x45, 0x33, 0xcf, 0x59,
- 0xbb, 0xf5, 0xfa, 0xdf, 0x4b, 0x53, 0xaf, 0x8f, 0x96, 0x9c, 0xef, 0x8f, 0x96, 0x9c, 0x37, 0x47,
- 0x4b, 0xce, 0xbf, 0x8e, 0x96, 0x9c, 0xbf, 0xbe, 0x5b, 0x9a, 0xfa, 0xfe, 0xdd, 0xd2, 0xd4, 0x9b,
- 0x77, 0x4b, 0x53, 0xdf, 0x35, 0x4b, 0xaf, 0xea, 0xff, 0x07, 0x00, 0x00, 0xff, 0xff, 0xfd, 0x7d,
- 0xb1, 0x38, 0x02, 0x10, 0x00, 0x00,
+ proto.RegisterFile("sql/execinfrapb/processors_bulk_io.proto", fileDescriptor_processors_bulk_io_e62f5714e36bbbd6)
+}
+
+var fileDescriptor_processors_bulk_io_e62f5714e36bbbd6 = []byte{
+ // 1673 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x58, 0x4f, 0x6f, 0x1b, 0xc7,
+ 0x15, 0xd7, 0x92, 0x4b, 0x8a, 0x7c, 0xb4, 0x24, 0x6a, 0xe2, 0xa4, 0x5b, 0x01, 0x95, 0x04, 0x26,
+ 0x72, 0x59, 0x17, 0x26, 0x11, 0xbb, 0x2d, 0x8c, 0xb6, 0x89, 0x6b, 0x52, 0x96, 0x43, 0x39, 0xb1,
+ 0xd5, 0xa5, 0xa5, 0x00, 0x41, 0x8b, 0xc5, 0x72, 0x77, 0x44, 0x8d, 0xb9, 0xdc, 0x59, 0xcd, 0xcc,
+ 0xca, 0xa6, 0x4f, 0x05, 0x7a, 0xea, 0xad, 0x1f, 0xa1, 0xdf, 0xa0, 0xfd, 0x0e, 0xbd, 0xf8, 0x98,
+ 0x63, 0x4e, 0x46, 0x2b, 0x7f, 0x8b, 0x9e, 0x8a, 0xf9, 0xb3, 0xd4, 0x4a, 0xa6, 0x64, 0x29, 0x86,
+ 0x2f, 0xf4, 0x7a, 0xde, 0xfc, 0x7e, 0xf3, 0xe6, 0xcd, 0x7b, 0xbf, 0x37, 0x23, 0x68, 0xf2, 0xc3,
+ 0xa8, 0x8d, 0x5f, 0xe0, 0x80, 0xc4, 0xfb, 0xcc, 0x4f, 0x06, 0xed, 0x84, 0xd1, 0x00, 0x73, 0x4e,
+ 0x19, 0xf7, 0x06, 0x69, 0x34, 0xf2, 0x08, 0x6d, 0x25, 0x8c, 0x0a, 0x8a, 0x9c, 0x80, 0x06, 0x23,
+ 0x46, 0xfd, 0xe0, 0xa0, 0xc5, 0x0f, 0xa3, 0x56, 0x48, 0xb8, 0xe0, 0x87, 0x11, 0x4b, 0xe3, 0x95,
+ 0x4f, 0x9e, 0xd1, 0x01, 0x6f, 0xcb, 0x9f, 0x64, 0xa0, 0xfe, 0xd1, 0x88, 0x15, 0x47, 0xcd, 0x4e,
+ 0x06, 0x6d, 0x42, 0x6f, 0xed, 0x53, 0x36, 0xf6, 0x45, 0x66, 0xf9, 0x54, 0xae, 0x1a, 0xf8, 0xc2,
+ 0x8f, 0xe8, 0xb0, 0x1d, 0x62, 0x1e, 0x24, 0x83, 0x36, 0x17, 0x2c, 0x0d, 0x44, 0xca, 0x70, 0x68,
+ 0x26, 0x6d, 0x5c, 0xe4, 0x9a, 0xcf, 0x71, 0xb6, 0x4a, 0x2a, 0x48, 0xd4, 0x3e, 0x88, 0x82, 0xb6,
+ 0x20, 0x63, 0xcc, 0x85, 0x3f, 0x4e, 0x8c, 0xe5, 0xfa, 0x90, 0x0e, 0xa9, 0xfa, 0x6c, 0xcb, 0x2f,
+ 0x33, 0x8a, 0x32, 0xaf, 0x42, 0x5f, 0xf8, 0x66, 0x6c, 0x39, 0x1b, 0xf3, 0x13, 0xa2, 0x87, 0x1a,
+ 0xff, 0x2c, 0xc2, 0x62, 0xc7, 0x0f, 0x46, 0xfb, 0x24, 0x8a, 0x30, 0xeb, 0x27, 0x38, 0x40, 0x0f,
+ 0xc1, 0x16, 0x93, 0x04, 0x3b, 0xd6, 0xba, 0xd5, 0x5c, 0xbc, 0x7d, 0xab, 0x75, 0x5e, 0x40, 0x5a,
+ 0xa7, 0x71, 0xad, 0xa7, 0x93, 0x04, 0x77, 0xec, 0x57, 0xaf, 0xd7, 0xe6, 0x5c, 0x45, 0x80, 0x3a,
+ 0x50, 0x12, 0xfe, 0x20, 0xc2, 0x4e, 0x61, 0xdd, 0x6a, 0xd6, 0x6e, 0xdf, 0x38, 0xc3, 0xc4, 0x0f,
+ 0x23, 0xb5, 0xbf, 0xa7, 0x72, 0xce, 0x26, 0xe6, 0x01, 0x23, 0x89, 0xa0, 0xcc, 0x50, 0x68, 0x28,
+ 0x7a, 0x00, 0x25, 0x9e, 0xf8, 0x31, 0x77, 0x8a, 0xeb, 0xc5, 0x66, 0xed, 0xf6, 0x2f, 0xce, 0xf7,
+ 0x46, 0xd1, 0xb8, 0xd8, 0x0f, 0xa5, 0x3b, 0x7e, 0x9c, 0xd1, 0x28, 0x34, 0xfa, 0x1c, 0x2a, 0x61,
+ 0xca, 0x7c, 0x41, 0x68, 0xec, 0xd8, 0xeb, 0x56, 0xb3, 0xd8, 0xf9, 0x58, 0x9a, 0xff, 0xf7, 0x7a,
+ 0x6d, 0x41, 0x86, 0xb3, 0xb5, 0x69, 0x8c, 0xee, 0x74, 0x1a, 0xfa, 0x14, 0x20, 0x38, 0x48, 0xe3,
+ 0x91, 0xc7, 0xc9, 0x4b, 0xec, 0x94, 0x14, 0x48, 0x73, 0x56, 0xd5, 0x78, 0x9f, 0xbc, 0xc4, 0xe8,
+ 0x1e, 0x54, 0x18, 0xf6, 0xc3, 0xfb, 0xfc, 0xc9, 0xbe, 0x33, 0xaf, 0x76, 0xf9, 0xb3, 0x9c, 0x87,
+ 0xf2, 0xc8, 0x5a, 0x07, 0x51, 0xd0, 0x7a, 0x9a, 0x1d, 0x99, 0x61, 0x98, 0x82, 0x1a, 0x37, 0xc1,
+ 0x96, 0x71, 0x43, 0x35, 0x98, 0xef, 0xc5, 0x47, 0x7e, 0x44, 0xc2, 0xfa, 0x1c, 0x02, 0x28, 0x77,
+ 0x69, 0x94, 0x8e, 0xe3, 0xba, 0x85, 0xaa, 0x50, 0xea, 0xc5, 0x21, 0x7e, 0x51, 0x2f, 0x6c, 0xdb,
+ 0x95, 0x72, 0x7d, 0xbe, 0xf1, 0x1c, 0x6a, 0xdb, 0x74, 0xb0, 0xc3, 0xe8, 0x90, 0x61, 0xce, 0xd1,
+ 0x67, 0x50, 0x7e, 0x46, 0x07, 0x1e, 0x09, 0xd5, 0x79, 0x15, 0x3b, 0x0b, 0x72, 0x81, 0xe3, 0xd7,
+ 0x6b, 0xa5, 0x6d, 0x3a, 0xe8, 0x6d, 0xba, 0xa5, 0x67, 0x74, 0xd0, 0x0b, 0x51, 0x13, 0xae, 0x05,
+ 0x34, 0x16, 0x8c, 0x0c, 0x52, 0x15, 0x03, 0x79, 0x22, 0x05, 0xe3, 0xcc, 0x29, 0x0b, 0x72, 0xc0,
+ 0xe6, 0x11, 0x15, 0x4e, 0x71, 0xdd, 0x6a, 0x96, 0xb2, 0xe3, 0x94, 0x23, 0x8d, 0x57, 0x15, 0x40,
+ 0x32, 0xbe, 0xbd, 0x71, 0x42, 0x99, 0xd8, 0xf4, 0x85, 0xaf, 0xd2, 0x65, 0x03, 0x6a, 0xdc, 0x1f,
+ 0x27, 0x11, 0xd6, 0x81, 0x2a, 0xe4, 0x70, 0xa0, 0x0d, 0x2a, 0x52, 0x0f, 0xa1, 0x92, 0x18, 0x9f,
+ 0x9d, 0xb2, 0x8a, 0xd4, 0xc6, 0xf9, 0x67, 0x99, 0xdb, 0x60, 0x16, 0xb1, 0x0c, 0x8c, 0x1e, 0x42,
+ 0x31, 0x65, 0xc4, 0x99, 0x57, 0xf9, 0xf0, 0xeb, 0xf3, 0x39, 0xde, 0x76, 0xb5, 0xb5, 0xcb, 0xc8,
+ 0x83, 0x58, 0xb0, 0x89, 0x2b, 0x19, 0xd0, 0x17, 0x50, 0xd6, 0xe5, 0xea, 0x54, 0x94, 0x3f, 0x6b,
+ 0x39, 0x2e, 0x53, 0x28, 0xad, 0xde, 0x93, 0x2d, 0x12, 0xe1, 0x2d, 0x35, 0xcd, 0x78, 0x62, 0x40,
+ 0x68, 0x0f, 0xca, 0x2a, 0x45, 0xb9, 0x53, 0x55, 0xae, 0xdc, 0xbd, 0x92, 0x2b, 0x2a, 0x5b, 0xb9,
+ 0xf2, 0x46, 0xf1, 0x5a, 0xae, 0x61, 0x43, 0xf7, 0xe0, 0xa7, 0x7c, 0x44, 0x12, 0x6f, 0x4c, 0x38,
+ 0x27, 0xf1, 0xd0, 0xdb, 0xa7, 0x0c, 0x93, 0x61, 0xec, 0x8d, 0xf0, 0x84, 0x3b, 0xb0, 0x6e, 0x35,
+ 0x2b, 0xc6, 0x91, 0x4f, 0xe4, 0xb4, 0x6f, 0xf4, 0xac, 0x2d, 0x3d, 0xe9, 0x11, 0x9e, 0x70, 0x74,
+ 0x13, 0x16, 0x9e, 0xfb, 0x51, 0x24, 0xf3, 0xfa, 0xb1, 0x1f, 0x53, 0xee, 0xd4, 0x72, 0xb9, 0x7b,
+ 0xda, 0x84, 0x6e, 0xc3, 0x32, 0x53, 0x25, 0xb3, 0xe3, 0x33, 0x3f, 0x8a, 0x70, 0x44, 0xf8, 0xd8,
+ 0x59, 0xc8, 0x1d, 0xe1, 0xdb, 0x66, 0xf4, 0x1d, 0x00, 0xc3, 0x3c, 0x1d, 0x63, 0x2f, 0xa1, 0xdc,
+ 0x59, 0x54, 0x9b, 0xff, 0xdd, 0x95, 0x36, 0xef, 0x2a, 0xf8, 0x0e, 0xd5, 0xfb, 0x77, 0xab, 0x2c,
+ 0xfb, 0x3f, 0xc2, 0x00, 0x29, 0xc7, 0xcc, 0x53, 0xe2, 0xe4, 0x2c, 0xad, 0x5b, 0xcd, 0x6a, 0x67,
+ 0xcb, 0x54, 0xea, 0x97, 0x43, 0x22, 0x0e, 0xd2, 0x41, 0x2b, 0xa0, 0xe3, 0xf6, 0x74, 0xb5, 0x70,
+ 0x70, 0xf2, 0xdd, 0x4e, 0x46, 0xc3, 0x36, 0xc7, 0x41, 0xca, 0x88, 0x98, 0xb4, 0xfa, 0x7f, 0xfc,
+ 0x7a, 0x97, 0x63, 0x16, 0xfb, 0x63, 0xbc, 0x23, 0xd9, 0xdc, 0xaa, 0x64, 0x56, 0x9f, 0x2b, 0x29,
+ 0xd4, 0xb4, 0x4b, 0xea, 0x18, 0xd0, 0x1f, 0xc0, 0x96, 0xea, 0xac, 0x2a, 0xe8, 0x6a, 0x3a, 0x65,
+ 0xb9, 0x0a, 0x89, 0x3e, 0x03, 0x10, 0x3e, 0x1b, 0x62, 0xd1, 0xa5, 0x11, 0x77, 0x0a, 0xeb, 0xc5,
+ 0x66, 0xd5, 0xd8, 0x73, 0xe3, 0x2b, 0x1c, 0x6a, 0xb9, 0x73, 0x47, 0x75, 0x28, 0x8e, 0xf0, 0x44,
+ 0xad, 0x5a, 0x75, 0xe5, 0x27, 0x7a, 0x0c, 0xa5, 0x23, 0x3f, 0x4a, 0x33, 0xc5, 0xbc, 0x5a, 0x4a,
+ 0xe5, 0x76, 0xe4, 0x6a, 0x9a, 0xdf, 0x16, 0xee, 0x5a, 0x2b, 0xbf, 0x81, 0x4a, 0x96, 0xf7, 0xf9,
+ 0x15, 0x4b, 0x7a, 0xc5, 0xeb, 0xf9, 0x15, 0xab, 0x79, 0xdc, 0xef, 0x61, 0xf1, 0xf4, 0x39, 0xbd,
+ 0x0b, 0x5d, 0xcc, 0xa1, 0xb7, 0xed, 0x8a, 0xa5, 0x14, 0xab, 0x58, 0xb7, 0xb7, 0xed, 0x8a, 0x5d,
+ 0x2f, 0x6d, 0xdb, 0x95, 0x52, 0xbd, 0xbc, 0x6d, 0x57, 0xae, 0xd5, 0x17, 0x1a, 0xff, 0x9e, 0xd7,
+ 0x5d, 0x27, 0x4d, 0xa6, 0x32, 0x72, 0x27, 0x13, 0x7a, 0x4b, 0x25, 0xd4, 0x4f, 0x66, 0x14, 0xe3,
+ 0xdb, 0xb2, 0xfe, 0x15, 0xd4, 0x49, 0x2c, 0x18, 0x0d, 0xd3, 0x00, 0x87, 0x9e, 0xc6, 0x17, 0x2e,
+ 0x83, 0x5f, 0x3a, 0x81, 0xf5, 0x15, 0xd3, 0x1d, 0xa8, 0x85, 0x78, 0xdf, 0x4f, 0x23, 0xe1, 0x49,
+ 0x75, 0x29, 0xaa, 0xcc, 0x43, 0x46, 0x4b, 0x61, 0x53, 0x9b, 0x76, 0xdd, 0x9e, 0x0b, 0x66, 0xda,
+ 0x2e, 0x23, 0xe8, 0xaf, 0x16, 0x7c, 0x94, 0x32, 0xc2, 0xbd, 0xc1, 0xc4, 0x8b, 0x68, 0xe0, 0x47,
+ 0x44, 0x4c, 0xbc, 0xd1, 0x91, 0x63, 0x2b, 0x17, 0xbe, 0xbc, 0xb8, 0x73, 0x9e, 0xec, 0x5d, 0xea,
+ 0x12, 0xef, 0x4c, 0xbe, 0x36, 0x0c, 0x8f, 0x8e, 0xb4, 0x2c, 0x5c, 0x3f, 0x7e, 0xbd, 0x56, 0xdf,
+ 0x75, 0x7b, 0x79, 0xd3, 0x9e, 0x5b, 0x4f, 0xcf, 0x4c, 0x46, 0x2e, 0xd4, 0xc6, 0x47, 0x41, 0xe0,
+ 0xed, 0x93, 0x48, 0x60, 0xa6, 0x3a, 0xd5, 0xe2, 0xa9, 0x36, 0x94, 0xed, 0xff, 0x9b, 0xbd, 0x6e,
+ 0x77, 0x4b, 0x4d, 0x3a, 0xd9, 0xd9, 0xc9, 0x98, 0x0b, 0x92, 0x45, 0x7f, 0xa3, 0xaf, 0x00, 0x70,
+ 0x1c, 0xb0, 0x49, 0xa2, 0xba, 0x85, 0xd6, 0xeb, 0xe6, 0x0c, 0x4a, 0xa9, 0x8e, 0x0f, 0xa6, 0x13,
+ 0x9f, 0xa8, 0x5f, 0xee, 0xe6, 0xb0, 0xe8, 0x09, 0x2c, 0x0f, 0xd4, 0x6e, 0x3d, 0x2e, 0x7c, 0x26,
+ 0x3c, 0xa9, 0x3d, 0x57, 0x69, 0x95, 0x4b, 0x1a, 0xdd, 0x97, 0x60, 0x69, 0x43, 0x8f, 0xc0, 0x0c,
+ 0x79, 0x38, 0x0e, 0x35, 0x5d, 0xe5, 0xf2, 0x74, 0x0b, 0x1a, 0xfb, 0x20, 0x0e, 0x15, 0xd9, 0x2e,
+ 0x94, 0x93, 0x91, 0x47, 0xc2, 0x4c, 0xc4, 0xef, 0x5c, 0xfa, 0xcc, 0x76, 0x46, 0xbd, 0xd0, 0xe8,
+ 0x77, 0x55, 0xb6, 0xdb, 0x9d, 0x47, 0xbd, 0x4d, 0xee, 0x96, 0x12, 0x39, 0x7c, 0x46, 0xc6, 0xe0,
+ 0x43, 0xc9, 0x58, 0x17, 0x3e, 0x9e, 0x99, 0x3a, 0x33, 0x94, 0xe5, 0xfc, 0x3a, 0xbf, 0x0b, 0x70,
+ 0xb2, 0x97, 0x3c, 0xd2, 0x9e, 0x81, 0xac, 0xe4, 0x90, 0x8d, 0x7f, 0x59, 0x50, 0x77, 0x31, 0x17,
+ 0x94, 0x61, 0x59, 0x44, 0x9a, 0xe0, 0x73, 0xb0, 0x65, 0x1d, 0x1a, 0x2d, 0x7d, 0x47, 0x19, 0xaa,
+ 0xa9, 0xe8, 0x3e, 0x94, 0xf6, 0x89, 0x6c, 0xa4, 0xba, 0x74, 0x37, 0x66, 0xf5, 0x61, 0xa5, 0x6d,
+ 0x2e, 0x3e, 0x4c, 0x31, 0x17, 0x2a, 0xeb, 0x32, 0x21, 0x50, 0x48, 0x74, 0x03, 0x6a, 0xd9, 0x05,
+ 0xa1, 0x17, 0xbe, 0x50, 0xe5, 0x9b, 0x75, 0xbc, 0xbc, 0xa1, 0xf1, 0x97, 0x22, 0x2c, 0x19, 0x97,
+ 0xa7, 0xca, 0xb3, 0x05, 0xd7, 0x98, 0x1e, 0xd2, 0xd9, 0x64, 0x5d, 0x3e, 0x9b, 0x6a, 0x06, 0xa8,
+ 0x72, 0xe9, 0x74, 0xcd, 0x14, 0xde, 0xa3, 0x66, 0x7a, 0x50, 0x66, 0x58, 0xf5, 0x7b, 0x7d, 0xeb,
+ 0xfd, 0xe5, 0x3b, 0x23, 0x62, 0x2e, 0xbf, 0x23, 0x3c, 0xc9, 0x6e, 0x29, 0x9a, 0x40, 0xde, 0x52,
+ 0x4c, 0x82, 0x6b, 0x51, 0xfa, 0xd5, 0x45, 0x2d, 0xe5, 0x54, 0x5c, 0x2e, 0xcc, 0xf0, 0xf7, 0xc8,
+ 0x9a, 0x7f, 0x14, 0xe0, 0xa3, 0x7e, 0x12, 0x11, 0x71, 0x3f, 0x0e, 0xfb, 0x81, 0x2f, 0x84, 0x79,
+ 0x76, 0xfc, 0x19, 0xca, 0xea, 0x5e, 0x9d, 0x75, 0x80, 0x7b, 0xe7, 0x7b, 0x3a, 0x03, 0x9e, 0x79,
+ 0xaf, 0xfc, 0xe9, 0x4a, 0x9e, 0x2c, 0x10, 0x9a, 0x34, 0x17, 0xd3, 0xc2, 0x7b, 0xc6, 0x74, 0xc5,
+ 0x83, 0xe5, 0xb7, 0x56, 0x43, 0xdb, 0x30, 0x8f, 0xe5, 0x35, 0x1a, 0x67, 0xfe, 0xdf, 0x7c, 0x67,
+ 0xa4, 0xa7, 0x45, 0x63, 0xf8, 0x33, 0x82, 0xc6, 0xdf, 0x8a, 0xb0, 0xd0, 0xed, 0xef, 0x7d, 0xcb,
+ 0x48, 0x16, 0x9c, 0x1b, 0xb2, 0x3d, 0x71, 0x41, 0x62, 0xfd, 0x84, 0x51, 0x85, 0x9d, 0xe5, 0x60,
+ 0xce, 0x80, 0x7e, 0x0e, 0xd7, 0xa4, 0x52, 0x78, 0x89, 0x0a, 0x8c, 0xce, 0xc2, 0xe9, 0x44, 0xa5,
+ 0x21, 0xda, 0x80, 0xbe, 0x80, 0x79, 0xaa, 0x33, 0x4f, 0x15, 0x4b, 0x6d, 0x66, 0xc3, 0xe8, 0xf6,
+ 0xf7, 0x4c, 0x7a, 0x66, 0x1e, 0x1a, 0xcc, 0xc9, 0xe3, 0x88, 0xd1, 0xe7, 0xdc, 0xbc, 0xa8, 0xf2,
+ 0x8f, 0x23, 0x97, 0x3e, 0xe7, 0xe8, 0x4f, 0xb0, 0x1c, 0xd0, 0x71, 0x22, 0x6b, 0x8f, 0xd0, 0xd8,
+ 0x0b, 0x68, 0x88, 0x03, 0xd3, 0x9e, 0x2e, 0x78, 0xc7, 0xc9, 0xf2, 0xe8, 0x9e, 0xc0, 0x0c, 0x6d,
+ 0x3d, 0xc7, 0xd4, 0x95, 0x44, 0x67, 0x34, 0xb6, 0xfc, 0x81, 0x34, 0xb6, 0xf1, 0x2d, 0x2c, 0x77,
+ 0xd2, 0x48, 0x6e, 0x28, 0x77, 0x1c, 0xd3, 0x97, 0xad, 0xf5, 0xa3, 0x5f, 0xb6, 0x37, 0x37, 0x60,
+ 0xe9, 0xcc, 0x56, 0x51, 0x05, 0xec, 0xc7, 0x34, 0xc6, 0xf5, 0x39, 0xf9, 0xf5, 0xf0, 0x25, 0x49,
+ 0xea, 0x56, 0xe7, 0xd6, 0xab, 0xff, 0xae, 0xce, 0xbd, 0x3a, 0x5e, 0xb5, 0xbe, 0x3f, 0x5e, 0xb5,
+ 0x7e, 0x38, 0x5e, 0xb5, 0xfe, 0x73, 0xbc, 0x6a, 0xfd, 0xfd, 0xcd, 0xea, 0xdc, 0xf7, 0x6f, 0x56,
+ 0xe7, 0x7e, 0x78, 0xb3, 0x3a, 0xf7, 0x5d, 0x2d, 0xf7, 0xc7, 0x83, 0xff, 0x07, 0x00, 0x00, 0xff,
+ 0xff, 0x3b, 0xa7, 0xba, 0x6f, 0xe9, 0x10, 0x00, 0x00,
}
diff --git a/pkg/sql/execinfrapb/processors_bulk_io.proto b/pkg/sql/execinfrapb/processors_bulk_io.proto
index 9f101f862acc..00e6af3e6565 100644
--- a/pkg/sql/execinfrapb/processors_bulk_io.proto
+++ b/pkg/sql/execinfrapb/processors_bulk_io.proto
@@ -130,7 +130,7 @@ message ReadImportDataSpec {
// User who initiated the import. This is used to check access privileges
// when using FileTable ExternalStorage.
- optional string user = 15 [(gogoproto.nullable) = false];
+ optional string user_proto = 15 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"];
// NEXTID: 16
}
@@ -151,7 +151,7 @@ message BackupDataSpec {
// User who initiated the backup. This is used to check access privileges
// when using FileTable ExternalStorage.
- optional string user = 10 [(gogoproto.nullable) = false];
+ optional string user_proto = 10 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"];
}
// RestoreDataEntry will be specified at planning time to the SplitAndScatter
@@ -207,7 +207,7 @@ message CSVWriterSpec {
// User who initiated the export. This is used to check access privileges
// when using FileTable ExternalStorage.
- optional string user = 6 [(gogoproto.nullable) = false];
+ optional string user_proto = 6 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"];
}
// BulkRowWriterSpec is the specification for a processor that consumes rows and
diff --git a/pkg/sql/execinfrapb/processors_changefeeds.pb.go b/pkg/sql/execinfrapb/processors_changefeeds.pb.go
index f3f75f8e34e7..bd0049a97766 100644
--- a/pkg/sql/execinfrapb/processors_changefeeds.pb.go
+++ b/pkg/sql/execinfrapb/processors_changefeeds.pb.go
@@ -16,6 +16,8 @@ import jobspb "github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
import roachpb "github.com/cockroachdb/cockroach/pkg/roachpb"
import hlc "github.com/cockroachdb/cockroach/pkg/util/hlc"
+import github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security"
+
import io "io"
// Reference imports to suppress errors if they are not otherwise used.
@@ -37,14 +39,14 @@ type ChangeAggregatorSpec struct {
Feed jobspb.ChangefeedDetails `protobuf:"bytes,2,opt,name=feed" json:"feed"`
// User who initiated the changefeed. This is used to check access privileges
// when using FileTable ExternalStorage.
- User string `protobuf:"bytes,3,opt,name=user" json:"user"`
+ UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,3,opt,name=user_proto,json=userProto,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto"`
}
func (m *ChangeAggregatorSpec) Reset() { *m = ChangeAggregatorSpec{} }
func (m *ChangeAggregatorSpec) String() string { return proto.CompactTextString(m) }
func (*ChangeAggregatorSpec) ProtoMessage() {}
func (*ChangeAggregatorSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_changefeeds_4ee18332ae724819, []int{0}
+ return fileDescriptor_processors_changefeeds_b7addf517cbc7dc0, []int{0}
}
func (m *ChangeAggregatorSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -78,7 +80,7 @@ func (m *ChangeAggregatorSpec_Watch) Reset() { *m = ChangeAggregatorSpec
func (m *ChangeAggregatorSpec_Watch) String() string { return proto.CompactTextString(m) }
func (*ChangeAggregatorSpec_Watch) ProtoMessage() {}
func (*ChangeAggregatorSpec_Watch) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_changefeeds_4ee18332ae724819, []int{0, 0}
+ return fileDescriptor_processors_changefeeds_b7addf517cbc7dc0, []int{0, 0}
}
func (m *ChangeAggregatorSpec_Watch) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -117,14 +119,14 @@ type ChangeFrontierSpec struct {
JobID int64 `protobuf:"varint,3,opt,name=job_id,json=jobId" json:"job_id"`
// User who initiated the changefeed. This is used to check access privileges
// when using FileTable ExternalStorage.
- User string `protobuf:"bytes,4,opt,name=user" json:"user"`
+ UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,4,opt,name=user_proto,json=userProto,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto"`
}
func (m *ChangeFrontierSpec) Reset() { *m = ChangeFrontierSpec{} }
func (m *ChangeFrontierSpec) String() string { return proto.CompactTextString(m) }
func (*ChangeFrontierSpec) ProtoMessage() {}
func (*ChangeFrontierSpec) Descriptor() ([]byte, []int) {
- return fileDescriptor_processors_changefeeds_4ee18332ae724819, []int{1}
+ return fileDescriptor_processors_changefeeds_b7addf517cbc7dc0, []int{1}
}
func (m *ChangeFrontierSpec) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -191,8 +193,8 @@ func (m *ChangeAggregatorSpec) MarshalTo(dAtA []byte) (int, error) {
i += n1
dAtA[i] = 0x1a
i++
- i = encodeVarintProcessorsChangefeeds(dAtA, i, uint64(len(m.User)))
- i += copy(dAtA[i:], m.User)
+ i = encodeVarintProcessorsChangefeeds(dAtA, i, uint64(len(m.UserProto)))
+ i += copy(dAtA[i:], m.UserProto)
return i, nil
}
@@ -270,8 +272,8 @@ func (m *ChangeFrontierSpec) MarshalTo(dAtA []byte) (int, error) {
i = encodeVarintProcessorsChangefeeds(dAtA, i, uint64(m.JobID))
dAtA[i] = 0x22
i++
- i = encodeVarintProcessorsChangefeeds(dAtA, i, uint64(len(m.User)))
- i += copy(dAtA[i:], m.User)
+ i = encodeVarintProcessorsChangefeeds(dAtA, i, uint64(len(m.UserProto)))
+ i += copy(dAtA[i:], m.UserProto)
return i, nil
}
@@ -298,7 +300,7 @@ func (m *ChangeAggregatorSpec) Size() (n int) {
}
l = m.Feed.Size()
n += 1 + l + sovProcessorsChangefeeds(uint64(l))
- l = len(m.User)
+ l = len(m.UserProto)
n += 1 + l + sovProcessorsChangefeeds(uint64(l))
return n
}
@@ -331,7 +333,7 @@ func (m *ChangeFrontierSpec) Size() (n int) {
l = m.Feed.Size()
n += 1 + l + sovProcessorsChangefeeds(uint64(l))
n += 1 + sovProcessorsChangefeeds(uint64(m.JobID))
- l = len(m.User)
+ l = len(m.UserProto)
n += 1 + l + sovProcessorsChangefeeds(uint64(l))
return n
}
@@ -441,7 +443,7 @@ func (m *ChangeAggregatorSpec) Unmarshal(dAtA []byte) error {
iNdEx = postIndex
case 3:
if wireType != 2 {
- return fmt.Errorf("proto: wrong wireType = %d for field User", wireType)
+ return fmt.Errorf("proto: wrong wireType = %d for field UserProto", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
@@ -466,7 +468,7 @@ func (m *ChangeAggregatorSpec) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
- m.User = string(dAtA[iNdEx:postIndex])
+ m.UserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
@@ -710,7 +712,7 @@ func (m *ChangeFrontierSpec) Unmarshal(dAtA []byte) error {
}
case 4:
if wireType != 2 {
- return fmt.Errorf("proto: wrong wireType = %d for field User", wireType)
+ return fmt.Errorf("proto: wrong wireType = %d for field UserProto", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
@@ -735,7 +737,7 @@ func (m *ChangeFrontierSpec) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
- m.User = string(dAtA[iNdEx:postIndex])
+ m.UserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])
iNdEx = postIndex
default:
iNdEx = preIndex
@@ -864,37 +866,41 @@ var (
)
func init() {
- proto.RegisterFile("sql/execinfrapb/processors_changefeeds.proto", fileDescriptor_processors_changefeeds_4ee18332ae724819)
-}
-
-var fileDescriptor_processors_changefeeds_4ee18332ae724819 = []byte{
- // 447 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xac, 0x93, 0x41, 0x6b, 0xd4, 0x40,
- 0x14, 0xc7, 0x33, 0xdd, 0xac, 0xe2, 0xac, 0x45, 0x09, 0x45, 0xc3, 0x82, 0xd3, 0xa5, 0x78, 0xd8,
- 0x43, 0x9d, 0x60, 0xf1, 0x0b, 0xb8, 0x96, 0x42, 0x3d, 0x78, 0xd8, 0x16, 0x04, 0x2f, 0x61, 0x32,
- 0x99, 0x26, 0xb3, 0x9d, 0x66, 0x66, 0x67, 0x66, 0xd5, 0xcf, 0xe0, 0xc9, 0x8f, 0xb5, 0xc7, 0x1e,
- 0x0b, 0x42, 0xa9, 0xd9, 0x2f, 0x22, 0x33, 0x99, 0xe0, 0x52, 0x14, 0x2f, 0x5e, 0x92, 0xc7, 0x7b,
- 0xef, 0xff, 0x7f, 0xef, 0xfd, 0x42, 0xe0, 0xa1, 0x59, 0x8a, 0x8c, 0x7d, 0x65, 0x94, 0x37, 0x17,
- 0x9a, 0xa8, 0x22, 0x53, 0x5a, 0x52, 0x66, 0x8c, 0xd4, 0x26, 0xa7, 0x35, 0x69, 0x2a, 0x76, 0xc1,
- 0x58, 0x69, 0xb0, 0xd2, 0xd2, 0xca, 0x24, 0xa5, 0x92, 0x5e, 0x6a, 0x49, 0x68, 0x8d, 0xcd, 0x52,
- 0xe0, 0x92, 0x1b, 0x6b, 0x96, 0x42, 0xaf, 0x9a, 0xf1, 0xb3, 0x85, 0x2c, 0x4c, 0xe6, 0x1e, 0xaa,
- 0xf0, 0xaf, 0x4e, 0x31, 0x4e, 0x7c, 0xb7, 0x2a, 0xb2, 0x92, 0x58, 0x12, 0x72, 0xe9, 0xca, 0x72,
- 0x91, 0xd5, 0x82, 0x66, 0x96, 0x5f, 0x31, 0x63, 0xc9, 0x95, 0x0a, 0x95, 0xbd, 0x4a, 0x56, 0xd2,
- 0x87, 0x99, 0x8b, 0xba, 0xec, 0xc1, 0x8f, 0x1d, 0xb8, 0xf7, 0xce, 0xef, 0xf2, 0xb6, 0xaa, 0x34,
- 0xab, 0x88, 0x95, 0xfa, 0x4c, 0x31, 0x9a, 0x9c, 0xc3, 0x87, 0x5f, 0x88, 0xa5, 0x35, 0x33, 0x29,
- 0x98, 0x0c, 0xa6, 0xa3, 0xa3, 0x37, 0xf8, 0x6f, 0x0b, 0xe2, 0x3f, 0x19, 0xe0, 0x8f, 0x4e, 0x3d,
- 0x8b, 0xd7, 0xb7, 0xfb, 0xd1, 0xbc, 0xb7, 0x4a, 0x4e, 0x60, 0xec, 0x6e, 0x4e, 0x77, 0x26, 0x60,
- 0x3a, 0x3a, 0x3a, 0xbc, 0x67, 0xe9, 0x6f, 0xeb, 0xee, 0x0c, 0x9e, 0xae, 0xf9, 0x98, 0x59, 0xc2,
- 0x85, 0x09, 0x56, 0x5e, 0x9f, 0xa4, 0x30, 0x5e, 0x19, 0xa6, 0xd3, 0xc1, 0x04, 0x4c, 0x1f, 0xf5,
- 0x15, 0x97, 0x19, 0x7f, 0x03, 0x70, 0xe8, 0x47, 0x27, 0x1f, 0xe0, 0x53, 0xde, 0x70, 0xcb, 0x89,
- 0xc8, 0x35, 0x33, 0x52, 0x7c, 0x66, 0x65, 0x0a, 0xfc, 0xdc, 0x17, 0x5b, 0x73, 0x1d, 0x2f, 0x5c,
- 0x0b, 0x8a, 0xcf, 0x7b, 0x5e, 0xc1, 0xee, 0x49, 0x10, 0xcf, 0x83, 0x36, 0x79, 0x0d, 0x63, 0xa3,
- 0x48, 0x13, 0x76, 0x7f, 0xbe, 0xe5, 0x11, 0xbe, 0x03, 0x3e, 0x53, 0xa4, 0xe9, 0x97, 0x71, 0xad,
- 0x07, 0x77, 0x00, 0x26, 0xdd, 0x21, 0x27, 0x5a, 0x36, 0x96, 0xb3, 0x8e, 0xed, 0x0c, 0xee, 0x5a,
- 0x4d, 0xe8, 0x25, 0x2b, 0x73, 0xd7, 0xd6, 0x13, 0xfe, 0x87, 0xe5, 0xe3, 0xa0, 0x71, 0xa9, 0xff,
- 0x47, 0xf2, 0x25, 0x7c, 0xb0, 0x90, 0x45, 0xce, 0x4b, 0xcf, 0x72, 0x30, 0xdb, 0x75, 0xb5, 0xf6,
- 0x76, 0x7f, 0xf8, 0x5e, 0x16, 0xa7, 0xc7, 0xf3, 0xe1, 0x42, 0x16, 0xa7, 0xbf, 0x79, 0xc7, 0xf7,
- 0x79, 0xcf, 0x5e, 0xad, 0x7f, 0xa2, 0x68, 0xdd, 0x22, 0x70, 0xdd, 0x22, 0x70, 0xd3, 0x22, 0x70,
- 0xd7, 0x22, 0xf0, 0x7d, 0x83, 0xa2, 0xeb, 0x0d, 0x8a, 0x6e, 0x36, 0x28, 0xfa, 0x34, 0xda, 0xfa,
- 0x05, 0x7e, 0x05, 0x00, 0x00, 0xff, 0xff, 0xd4, 0x53, 0x2b, 0x0f, 0x14, 0x03, 0x00, 0x00,
+ proto.RegisterFile("sql/execinfrapb/processors_changefeeds.proto", fileDescriptor_processors_changefeeds_b7addf517cbc7dc0)
+}
+
+var fileDescriptor_processors_changefeeds_b7addf517cbc7dc0 = []byte{
+ // 500 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x93, 0xcf, 0x8a, 0x13, 0x41,
+ 0x10, 0xc6, 0x33, 0x9b, 0xac, 0xb2, 0x1d, 0x17, 0x65, 0x58, 0x74, 0x08, 0x38, 0x09, 0x8b, 0x87,
+ 0x1c, 0xd6, 0x6e, 0x5c, 0x3c, 0x0b, 0xc6, 0x25, 0xb0, 0x22, 0xa2, 0xc9, 0x8a, 0xe0, 0x25, 0xf4,
+ 0xf4, 0xd4, 0xce, 0x74, 0x32, 0x99, 0xee, 0x74, 0x77, 0xfc, 0xf3, 0x0a, 0x9e, 0x7c, 0x0c, 0x2f,
+ 0xbe, 0x47, 0x8e, 0x7b, 0xdc, 0x53, 0xd0, 0xe4, 0x2d, 0x3c, 0x49, 0xf7, 0x74, 0x34, 0x2c, 0x8a,
+ 0x17, 0xbd, 0x24, 0x45, 0x57, 0x7d, 0x3f, 0xea, 0xab, 0x8f, 0x41, 0x47, 0x7a, 0x56, 0x10, 0x78,
+ 0x0f, 0x8c, 0x97, 0xe7, 0x8a, 0xca, 0x84, 0x48, 0x25, 0x18, 0x68, 0x2d, 0x94, 0x1e, 0xb1, 0x9c,
+ 0x96, 0x19, 0x9c, 0x03, 0xa4, 0x1a, 0x4b, 0x25, 0x8c, 0x08, 0x23, 0x26, 0xd8, 0x44, 0x09, 0xca,
+ 0x72, 0xac, 0x67, 0x05, 0x4e, 0xb9, 0x36, 0x7a, 0x56, 0xa8, 0x79, 0xd9, 0xba, 0x3d, 0x16, 0x89,
+ 0x26, 0xf6, 0x47, 0x26, 0xee, 0xaf, 0x52, 0xb4, 0x42, 0x37, 0x2d, 0x13, 0x92, 0x52, 0x43, 0xfd,
+ 0x5b, 0x34, 0x37, 0xbc, 0x20, 0x79, 0xc1, 0x88, 0xe1, 0x53, 0xd0, 0x86, 0x4e, 0xa5, 0xef, 0x1c,
+ 0x64, 0x22, 0x13, 0xae, 0x24, 0xb6, 0xaa, 0x5e, 0x0f, 0x3f, 0xd7, 0xd1, 0xc1, 0x13, 0xb7, 0xcb,
+ 0xe3, 0x2c, 0x53, 0x90, 0x51, 0x23, 0xd4, 0x50, 0x02, 0x0b, 0xcf, 0xd0, 0xf5, 0x77, 0xd4, 0xb0,
+ 0x1c, 0x74, 0x14, 0x74, 0xea, 0xdd, 0xe6, 0xf1, 0x43, 0xfc, 0xa7, 0x05, 0xf1, 0xef, 0x00, 0xf8,
+ 0xb5, 0x55, 0xf7, 0x1a, 0x8b, 0x65, 0xbb, 0x36, 0xd8, 0xa0, 0xc2, 0x3e, 0x6a, 0x58, 0xcf, 0xd1,
+ 0x4e, 0x27, 0xe8, 0x36, 0x8f, 0x8f, 0xae, 0x20, 0x9d, 0xb7, 0xca, 0xa7, 0x67, 0xda, 0xe1, 0x13,
+ 0x30, 0x94, 0x17, 0xda, 0xa3, 0x9c, 0x3e, 0x04, 0x84, 0xe6, 0x1a, 0xd4, 0xc8, 0x99, 0x88, 0xea,
+ 0x9d, 0xa0, 0xbb, 0xd7, 0xeb, 0xdb, 0xfe, 0xf7, 0x65, 0xfb, 0x51, 0xc6, 0x4d, 0x3e, 0x4f, 0x30,
+ 0x13, 0x53, 0xf2, 0x93, 0x9f, 0x26, 0xbf, 0x6a, 0x22, 0x27, 0x19, 0xd1, 0xc0, 0xe6, 0x8a, 0x9b,
+ 0x0f, 0x78, 0xf8, 0xf2, 0xd9, 0x2b, 0x0d, 0xaa, 0xa4, 0x53, 0x78, 0x61, 0x69, 0x83, 0x3d, 0x4b,
+ 0x76, 0x65, 0xeb, 0x63, 0x80, 0x76, 0x9d, 0x8f, 0xf0, 0x39, 0xba, 0xc5, 0x4b, 0x6e, 0x38, 0x2d,
+ 0x46, 0x0a, 0xb4, 0x28, 0xde, 0x42, 0x1a, 0x05, 0xce, 0xc4, 0xdd, 0x2d, 0x13, 0xf6, 0xf8, 0x38,
+ 0x2f, 0x18, 0x3e, 0xdb, 0x1c, 0xdf, 0x6f, 0x7d, 0xd3, 0x8b, 0x07, 0x5e, 0x1b, 0x3e, 0x40, 0x0d,
+ 0x2d, 0x69, 0xe9, 0x0f, 0x71, 0x67, 0x8b, 0xe1, 0x43, 0xc5, 0x43, 0x49, 0xcb, 0x8d, 0x67, 0x3b,
+ 0x7a, 0xf8, 0x65, 0x07, 0x85, 0xd5, 0x55, 0xfa, 0x4a, 0x94, 0x86, 0x43, 0x15, 0x54, 0x0f, 0xed,
+ 0x1b, 0x45, 0xd9, 0x04, 0xd2, 0x91, 0x1d, 0xdb, 0xc4, 0xf5, 0x17, 0xe4, 0x0d, 0xaf, 0xb1, 0x4f,
+ 0xff, 0x2e, 0x96, 0x7b, 0xe8, 0xda, 0x58, 0x24, 0x23, 0x9e, 0xba, 0x48, 0xea, 0xbd, 0x7d, 0xdb,
+ 0x5b, 0x2d, 0xdb, 0xbb, 0x4f, 0x45, 0x72, 0x7a, 0x32, 0xd8, 0x1d, 0x8b, 0xe4, 0xf4, 0x6a, 0x78,
+ 0x8d, 0xff, 0x14, 0x5e, 0xef, 0xfe, 0xe2, 0x5b, 0x5c, 0x5b, 0xac, 0xe2, 0xe0, 0x62, 0x15, 0x07,
+ 0x97, 0xab, 0x38, 0xf8, 0xba, 0x8a, 0x83, 0x4f, 0xeb, 0xb8, 0x76, 0xb1, 0x8e, 0x6b, 0x97, 0xeb,
+ 0xb8, 0xf6, 0xa6, 0xb9, 0xf5, 0x71, 0xfe, 0x08, 0x00, 0x00, 0xff, 0xff, 0xc2, 0x4f, 0xbd, 0x95,
+ 0xae, 0x03, 0x00, 0x00,
}
diff --git a/pkg/sql/execinfrapb/processors_changefeeds.proto b/pkg/sql/execinfrapb/processors_changefeeds.proto
index a023f6ea5994..f681e3684bf5 100644
--- a/pkg/sql/execinfrapb/processors_changefeeds.proto
+++ b/pkg/sql/execinfrapb/processors_changefeeds.proto
@@ -38,7 +38,7 @@ message ChangeAggregatorSpec {
// User who initiated the changefeed. This is used to check access privileges
// when using FileTable ExternalStorage.
- optional string user = 3 [(gogoproto.nullable) = false];
+ optional string user_proto = 3 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"];
}
// ChangeFrontierSpec is the specification for a processor that receives
@@ -61,5 +61,5 @@ message ChangeFrontierSpec {
// User who initiated the changefeed. This is used to check access privileges
// when using FileTable ExternalStorage.
- optional string user = 4 [(gogoproto.nullable) = false];
+ optional string user_proto = 4 [(gogoproto.nullable) = false, (gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"];
}
diff --git a/pkg/sql/explain_tree_test.go b/pkg/sql/explain_tree_test.go
index 25d0650b99fe..42e9d32d0c23 100644
--- a/pkg/sql/explain_tree_test.go
+++ b/pkg/sql/explain_tree_test.go
@@ -57,7 +57,7 @@ func TestPlanToTreeAndPlanToString(t *testing.T) {
internalPlanner, cleanup := NewInternalPlanner(
"test",
kv.NewTxn(ctx, db, s.NodeID()),
- security.RootUser,
+ security.RootUserName(),
&MemoryMetrics{},
&execCfg,
)
diff --git a/pkg/sql/flowinfra/flow_test.go b/pkg/sql/flowinfra/flow_test.go
index af737f499b58..eb4e917866dc 100644
--- a/pkg/sql/flowinfra/flow_test.go
+++ b/pkg/sql/flowinfra/flow_test.go
@@ -47,7 +47,7 @@ func BenchmarkFlowSetup(b *testing.B) {
planner, cleanup := sql.NewInternalPlanner(
"test",
kv.NewTxn(ctx, s.DB(), s.NodeID()),
- security.RootUser,
+ security.RootUserName(),
&sql.MemoryMetrics{},
&execCfg,
)
diff --git a/pkg/sql/gcjob_test/gc_job_test.go b/pkg/sql/gcjob_test/gc_job_test.go
index af5d367d5239..46f2a371b90d 100644
--- a/pkg/sql/gcjob_test/gc_job_test.go
+++ b/pkg/sql/gcjob_test/gc_job_test.go
@@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -164,7 +165,7 @@ func TestSchemaChangeGCJob(t *testing.T) {
jobRecord := jobs.Record{
Description: fmt.Sprintf("GC test"),
- Username: "user",
+ Username: security.TestUserName(),
DescriptorIDs: descpb.IDs{myTableID},
Details: details,
Progress: jobspb.SchemaChangeGCProgress{},
@@ -174,7 +175,7 @@ func TestSchemaChangeGCJob(t *testing.T) {
// The job record that will be used to lookup this job.
lookupJR := jobs.Record{
Description: fmt.Sprintf("GC test"),
- Username: "user",
+ Username: security.TestUserName(),
DescriptorIDs: descpb.IDs{myTableID},
Details: details,
}
diff --git a/pkg/sql/grant_revoke.go b/pkg/sql/grant_revoke.go
index 5bad34943999..89776dddb313 100644
--- a/pkg/sql/grant_revoke.go
+++ b/pkg/sql/grant_revoke.go
@@ -58,11 +58,20 @@ func (p *planner) Grant(ctx context.Context, n *tree.Grant) (planNode, error) {
return nil, err
}
+ // TODO(solon): there are SQL identifiers (tree.Name) in n.Grantees,
+ // but we want SQL usernames. Do we normalize or not? For reference,
+ // REASSIGN / OWNER TO do normalize.
+ // Related: https://github.com/cockroachdb/cockroach/issues/54696
+ grantees := make([]security.SQLUsername, len(n.Grantees))
+ for i, grantee := range n.Grantees {
+ grantees[i] = security.MakeSQLUsernameFromPreNormalizedString(string(grantee))
+ }
+
return &changePrivilegesNode{
targets: n.Targets,
- grantees: n.Grantees,
+ grantees: grantees,
desiredprivs: n.Privileges,
- changePrivilege: func(privDesc *descpb.PrivilegeDescriptor, grantee string) {
+ changePrivilege: func(privDesc *descpb.PrivilegeDescriptor, grantee security.SQLUsername) {
privDesc.Grant(grantee, n.Privileges)
},
grantOn: grantOn,
@@ -100,11 +109,20 @@ func (p *planner) Revoke(ctx context.Context, n *tree.Revoke) (planNode, error)
return nil, err
}
+ // TODO(solon): there are SQL identifiers (tree.Name) in n.Grantees,
+ // but we want SQL usernames. Do we normalize or not? For reference,
+ // REASSIGN / OWNER TO do normalize.
+ // Related: https://github.com/cockroachdb/cockroach/issues/54696
+ grantees := make([]security.SQLUsername, len(n.Grantees))
+ for i, grantee := range n.Grantees {
+ grantees[i] = security.MakeSQLUsernameFromPreNormalizedString(string(grantee))
+ }
+
return &changePrivilegesNode{
targets: n.Targets,
- grantees: n.Grantees,
+ grantees: grantees,
desiredprivs: n.Privileges,
- changePrivilege: func(privDesc *descpb.PrivilegeDescriptor, grantee string) {
+ changePrivilege: func(privDesc *descpb.PrivilegeDescriptor, grantee security.SQLUsername) {
privDesc.Revoke(grantee, n.Privileges, grantOn)
},
grantOn: grantOn,
@@ -114,9 +132,9 @@ func (p *planner) Revoke(ctx context.Context, n *tree.Revoke) (planNode, error)
type changePrivilegesNode struct {
targets tree.TargetList
- grantees tree.NameList
+ grantees []security.SQLUsername
desiredprivs privilege.List
- changePrivilege func(*descpb.PrivilegeDescriptor, string)
+ changePrivilege func(*descpb.PrivilegeDescriptor, security.SQLUsername)
grantOn privilege.ObjectType
eventLogType EventLogType
}
@@ -137,11 +155,12 @@ func (n *changePrivilegesNode) startExec(params runParams) error {
// We're allowed to grant/revoke privileges to/from the "public" role even though
// it does not exist: add it to the list of all users and roles.
- users[security.PublicRole] = true // isRole
+ users[security.PublicRoleName()] = true // isRole
- for _, grantee := range n.grantees {
- if _, ok := users[string(grantee)]; !ok {
- return errors.Errorf("user or role %s does not exist", &grantee)
+ for i, grantee := range n.grantees {
+ if _, ok := users[grantee]; !ok {
+ sqlName := tree.Name(n.grantees[i].Normalized())
+ return errors.Errorf("user or role %s does not exist", &sqlName)
}
}
@@ -173,7 +192,7 @@ func (n *changePrivilegesNode) startExec(params runParams) error {
privileges := descriptor.GetPrivileges()
for _, grantee := range n.grantees {
- n.changePrivilege(privileges, string(grantee))
+ n.changePrivilege(privileges, grantee)
}
// Validate privilege descriptors directly as the db/table level Validate
@@ -235,6 +254,15 @@ func (n *changePrivilegesNode) startExec(params runParams) error {
fmtCtx := tree.NewFmtCtx(tree.FmtSimple)
n.targets.Format(fmtCtx)
targets := fmtCtx.CloseAndGetString()
+
+ var grantees strings.Builder
+ comma := ""
+ for _, g := range n.grantees {
+ grantees.WriteString(comma)
+ grantees.WriteString(g.Normalized())
+ comma = ","
+ }
+
return MakeEventLogger(params.extendedEvalCtx.ExecCfg).InsertEventRecord(
params.ctx,
params.p.txn,
@@ -243,10 +271,10 @@ func (n *changePrivilegesNode) startExec(params runParams) error {
int32(params.extendedEvalCtx.NodeID.SQLInstanceID()),
struct {
Target string
- User string
+ User security.SQLUsername
Grantees string
Privileges string
- }{targets, p.SessionData().User, strings.Join(n.grantees.ToStrings(), ","), n.desiredprivs.String()},
+ }{targets, p.SessionData().User(), grantees.String(), n.desiredprivs.String()},
)
}
diff --git a/pkg/sql/grant_role.go b/pkg/sql/grant_role.go
index c6a6687c8e75..5dd58335e0e0 100644
--- a/pkg/sql/grant_role.go
+++ b/pkg/sql/grant_role.go
@@ -59,20 +59,26 @@ func (p *planner) GrantRoleNode(ctx context.Context, n *tree.GrantRole) (*GrantR
if err != nil {
return nil, err
}
- for _, r := range n.Roles {
+ for i := range n.Roles {
+ // TODO(solon): there are SQL identifiers (tree.Name) in
+ // n.Roles, but we want SQL usernames. Do we normalize or not? For
+ // reference, REASSIGN / OWNER TO do normalize. Related:
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ r := security.MakeSQLUsernameFromPreNormalizedString(string(n.Roles[i]))
+
// If the user is an admin, don't check if the user is allowed to add/drop
// roles in the role. However, if the role being modified is the admin role, then
// make sure the user is an admin with the admin option.
- if hasAdminRole && string(r) != security.AdminRole {
+ if hasAdminRole && !r.IsAdminRole() {
continue
}
- if isAdmin, ok := allRoles[string(r)]; !ok || !isAdmin {
- if string(r) == security.AdminRole {
+ if isAdmin, ok := allRoles[r]; !ok || !isAdmin {
+ if r.IsAdminRole() {
return nil, pgerror.Newf(pgcode.InsufficientPrivilege,
- "%s is not a role admin for role %s", p.User(), r)
+ "%s is not a role admin for role %s", p.User(), n.Roles[i])
}
return nil, pgerror.Newf(pgcode.InsufficientPrivilege,
- "%s is not a superuser or role admin for role %s", p.User(), r)
+ "%s is not a superuser or role admin for role %s", p.User(), n.Roles[i])
}
}
@@ -87,22 +93,37 @@ func (p *planner) GrantRoleNode(ctx context.Context, n *tree.GrantRole) (*GrantR
// NOTE: membership manipulation involving the "public" pseudo-role fails with
// "role public does not exist". This matches postgres behavior.
- for _, r := range n.Roles {
- if _, ok := roles[string(r)]; !ok {
+ for i := range n.Roles {
+ // TODO(solon): there are SQL identifiers (tree.Name) in
+ // n.Roles, but we want SQL usernames. Do we normalize or not? For
+ // reference, REASSIGN / OWNER TO do normalize. Related:
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ r := security.MakeSQLUsernameFromPreNormalizedString(string(n.Roles[i]))
+
+ if _, ok := roles[r]; !ok {
+ maybeOption := strings.ToUpper(r.Normalized())
for name := range roleoption.ByName {
- if uppercase := strings.ToUpper(string(r)); uppercase == name {
+ if maybeOption == name {
return nil, errors.WithHintf(
- pgerror.Newf(pgcode.UndefinedObject, "role/user %s does not exist", r),
- "%s is a role option, try using ALTER ROLE to change a role's options.", uppercase)
+ pgerror.Newf(pgcode.UndefinedObject,
+ "role/user %s does not exist", n.Roles[i]),
+ "%s is a role option, try using ALTER ROLE to change a role's options.", maybeOption)
}
}
- return nil, pgerror.Newf(pgcode.UndefinedObject, "role/user %s does not exist", r)
+ return nil, pgerror.Newf(pgcode.UndefinedObject, "role/user %s does not exist", n.Roles[i])
}
}
- for _, m := range n.Members {
- if _, ok := roles[string(m)]; !ok {
- return nil, pgerror.Newf(pgcode.UndefinedObject, "role/user %s does not exist", m)
+ for i := range n.Members {
+ // TODO(solon): there are SQL identifiers (tree.Name) in
+ // n.Members but we want SQL usernames. Do we normalize or not? For
+ // reference, REASSIGN / OWNER TO do normalize. Related:
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ m := security.MakeSQLUsernameFromPreNormalizedString(string(n.Members[i]))
+
+ if _, ok := roles[m]; !ok {
+ return nil, pgerror.Newf(pgcode.UndefinedObject,
+ "role/user %s does not exist", n.Members[i])
}
}
@@ -110,9 +131,14 @@ func (p *planner) GrantRoleNode(ctx context.Context, n *tree.GrantRole) (*GrantR
// means checking whether we have an expanded relationship (grant.Role ∈ ... ∈ grant.Member)
// For each grant.Role, we lookup all the roles it is a member of.
// After adding a given edge (grant.Member ∈ grant.Role), we add the edge to the list as well.
- allRoleMemberships := make(map[string]map[string]bool)
+ allRoleMemberships := make(map[security.SQLUsername]map[security.SQLUsername]bool)
for _, rawR := range n.Roles {
- r := string(rawR)
+ // TODO(solon): there are SQL identifiers (tree.Name) in
+ // n.Roles but we want SQL usernames. Do we normalize or not? For
+ // reference, REASSIGN / OWNER TO do normalize. Related:
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ r := security.MakeSQLUsernameFromPreNormalizedString(string(rawR))
+
allRoles, err := p.MemberOfWithAdminOption(ctx, r)
if err != nil {
return nil, err
@@ -123,23 +149,28 @@ func (p *planner) GrantRoleNode(ctx context.Context, n *tree.GrantRole) (*GrantR
// Since we perform no queries here, check all role/member pairs for cycles.
// Only if there are no errors do we proceed to write them.
for _, rawR := range n.Roles {
- r := string(rawR)
+ // TODO(solon): there are SQL identifiers (tree.Name) in
+ // n.Roles but we want SQL usernames. Do we normalize or not? For
+ // reference, REASSIGN / OWNER TO do normalize. Related:
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ r := security.MakeSQLUsernameFromPreNormalizedString(string(rawR))
for _, rawM := range n.Members {
- m := string(rawM)
+ // TODO(solon): ditto above, names in n.Members.
+ m := security.MakeSQLUsernameFromPreNormalizedString(string(rawM))
if r == m {
// self-cycle.
- return nil, pgerror.Newf(pgcode.InvalidGrantOperation, "%s cannot be a member of itself", m)
+ return nil, pgerror.Newf(pgcode.InvalidGrantOperation, "%s cannot be a member of itself", rawM)
}
// Check if grant.Role ∈ ... ∈ grant.Member
if memberOf, ok := allRoleMemberships[r]; ok {
if _, ok = memberOf[m]; ok {
return nil, pgerror.Newf(pgcode.InvalidGrantOperation,
- "making %s a member of %s would create a cycle", m, r)
+ "making %s a member of %s would create a cycle", rawM, rawR)
}
}
// Add the new membership. We don't care about the actual bool value.
if _, ok := allRoleMemberships[m]; !ok {
- allRoleMemberships[m] = make(map[string]bool)
+ allRoleMemberships[m] = make(map[security.SQLUsername]bool)
}
allRoleMemberships[m][r] = false
}
@@ -172,7 +203,7 @@ func (n *GrantRoleNode) startExec(params runParams) error {
params.ctx,
opName,
params.p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
memberStmt,
r, m, n.adminOption,
)
diff --git a/pkg/sql/information_schema.go b/pkg/sql/information_schema.go
index 904c3de8a707..427a2e8f74a1 100644
--- a/pkg/sql/information_schema.go
+++ b/pkg/sql/information_schema.go
@@ -208,13 +208,13 @@ var informationSchemaAdministrableRoleAuthorizations = virtualSchemaTable{
https://www.postgresql.org/docs/9.5/infoschema-administrable-role-authorizations.html`,
schema: vtable.InformationSchemaAdministrableRoleAuthorizations,
populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error {
- currentUser := p.SessionData().User
+ currentUser := p.SessionData().User()
memberMap, err := p.MemberOfWithAdminOption(ctx, currentUser)
if err != nil {
return err
}
- grantee := tree.NewDString(currentUser)
+ grantee := tree.NewDString(currentUser.Normalized())
for roleName, isAdmin := range memberMap {
if !isAdmin {
// We only show memberships with the admin option.
@@ -222,9 +222,9 @@ https://www.postgresql.org/docs/9.5/infoschema-administrable-role-authorizations
}
if err := addRow(
- grantee, // grantee: always the current user
- tree.NewDString(roleName), // role_name
- yesString, // is_grantable: always YES
+ grantee, // grantee: always the current user
+ tree.NewDString(roleName.Normalized()), // role_name
+ yesString, // is_grantable: always YES
); err != nil {
return err
}
@@ -240,19 +240,19 @@ var informationSchemaApplicableRoles = virtualSchemaTable{
https://www.postgresql.org/docs/9.5/infoschema-applicable-roles.html`,
schema: vtable.InformationSchemaApplicableRoles,
populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error {
- currentUser := p.SessionData().User
+ currentUser := p.SessionData().User()
memberMap, err := p.MemberOfWithAdminOption(ctx, currentUser)
if err != nil {
return err
}
- grantee := tree.NewDString(currentUser)
+ grantee := tree.NewDString(currentUser.Normalized())
for roleName, isAdmin := range memberMap {
if err := addRow(
- grantee, // grantee: always the current user
- tree.NewDString(roleName), // role_name
- yesOrNoDatum(isAdmin), // is_grantable
+ grantee, // grantee: always the current user
+ tree.NewDString(roleName.Normalized()), // role_name
+ yesOrNoDatum(isAdmin), // is_grantable
); err != nil {
return err
}
@@ -350,14 +350,14 @@ https://www.postgresql.org/docs/9.5/infoschema-column-privileges.html`,
for i := range columns {
cd := &columns[i]
if err := addRow(
- tree.DNull, // grantor
- tree.NewDString(u.User), // grantee
- dbNameStr, // table_catalog
- scNameStr, // table_schema
- tree.NewDString(table.GetName()), // table_name
- tree.NewDString(cd.Name), // column_name
- tree.NewDString(priv.String()), // privilege_type
- tree.DNull, // is_grantable
+ tree.DNull, // grantor
+ tree.NewDString(u.User().Normalized()), // grantee
+ dbNameStr, // table_catalog
+ scNameStr, // table_schema
+ tree.NewDString(table.GetName()), // table_name
+ tree.NewDString(cd.Name), // column_name
+ tree.NewDString(priv.String()), // privilege_type
+ tree.DNull, // is_grantable
); err != nil {
return err
}
@@ -501,7 +501,7 @@ CREATE TABLE information_schema.enabled_roles (
ROLE_NAME STRING NOT NULL
)`,
populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error {
- currentUser := p.SessionData().User
+ currentUser := p.SessionData().User()
memberMap, err := p.MemberOfWithAdminOption(ctx, currentUser)
if err != nil {
return err
@@ -509,14 +509,14 @@ CREATE TABLE information_schema.enabled_roles (
// The current user is always listed.
if err := addRow(
- tree.NewDString(currentUser), // role_name: the current user
+ tree.NewDString(currentUser.Normalized()), // role_name: the current user
); err != nil {
return err
}
for roleName := range memberMap {
if err := addRow(
- tree.NewDString(roleName), // role_name
+ tree.NewDString(roleName.Normalized()), // role_name
); err != nil {
return err
}
@@ -1084,7 +1084,7 @@ CREATE TABLE information_schema.type_privileges (
// https://github.com/cockroachdb/cockroach/issues/35572
privs := typeDesc.TypeDescriptor.GetPrivileges().Show(privilege.Type)
for _, u := range privs {
- userNameStr := tree.NewDString(u.User)
+ userNameStr := tree.NewDString(u.User.Normalized())
for _, priv := range u.Privileges {
if err := addRow(
userNameStr, // grantee
@@ -1132,7 +1132,7 @@ CREATE TABLE information_schema.schema_privileges (
// TODO(knz): This should filter for the current user, see
// https://github.com/cockroachdb/cockroach/issues/35572
for _, u := range privs {
- userNameStr := tree.NewDString(u.User)
+ userNameStr := tree.NewDString(u.User.Normalized())
for _, priv := range u.Privileges {
if err := addRow(
userNameStr, // grantee
@@ -1461,14 +1461,14 @@ func populateTablePrivileges(
for _, u := range table.GetPrivileges().Show(privilege.Table) {
for _, priv := range u.Privileges {
if err := addRow(
- tree.DNull, // grantor
- tree.NewDString(u.User), // grantee
- dbNameStr, // table_catalog
- scNameStr, // table_schema
- tbNameStr, // table_name
- tree.NewDString(priv), // privilege_type
- tree.DNull, // is_grantable
- yesOrNoDatum(priv == "SELECT"), // with_hierarchy
+ tree.DNull, // grantor
+ tree.NewDString(u.User.Normalized()), // grantee
+ dbNameStr, // table_catalog
+ scNameStr, // table_schema
+ tbNameStr, // table_name
+ tree.NewDString(priv), // privilege_type
+ tree.DNull, // is_grantable
+ yesOrNoDatum(priv == "SELECT"), // with_hierarchy
); err != nil {
return err
}
@@ -1950,7 +1950,7 @@ func forEachTableDescWithTableLookupInternal(
func forEachRole(
ctx context.Context,
p *planner,
- fn func(username string, isRole bool, noLogin bool, rolValidUntil *time.Time) error,
+ fn func(username security.SQLUsername, isRole bool, noLogin bool, rolValidUntil *time.Time) error,
) error {
query := `
SELECT
@@ -1981,7 +1981,7 @@ FROM
}
for _, row := range rows {
- username := tree.MustBeDString(row[0])
+ usernameS := tree.MustBeDString(row[0])
isRole, ok := row[1].(*tree.DBool)
if !ok {
return errors.Errorf("isRole should be a boolean value, found %s instead", row[1].ResolvedType())
@@ -1996,8 +1996,9 @@ FROM
} else if row[3] != tree.DNull {
return errors.Errorf("rolValidUntil should be a timestamp or null value, found %s instead", row[3].ResolvedType())
}
-
- if err := fn(string(username), bool(*isRole), bool(*noLogin), rolValidUntil); err != nil {
+ // system tables already contain normalized usernames.
+ username := security.MakeSQLUsernameFromPreNormalizedString(string(usernameS))
+ if err := fn(username, bool(*isRole), bool(*noLogin), rolValidUntil); err != nil {
return err
}
}
@@ -2006,7 +2007,7 @@ FROM
}
func forEachRoleMembership(
- ctx context.Context, p *planner, fn func(role, member string, isAdmin bool) error,
+ ctx context.Context, p *planner, fn func(role, member security.SQLUsername, isAdmin bool) error,
) error {
query := `SELECT "role", "member", "isAdmin" FROM system.role_members`
rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.Query(
@@ -2021,7 +2022,11 @@ func forEachRoleMembership(
memberName := tree.MustBeDString(row[1])
isAdmin := row[2].(*tree.DBool)
- if err := fn(string(roleName), string(memberName), bool(*isAdmin)); err != nil {
+ // The names in the system tables are already normalized.
+ if err := fn(
+ security.MakeSQLUsernameFromPreNormalizedString(string(roleName)),
+ security.MakeSQLUsernameFromPreNormalizedString(string(memberName)),
+ bool(*isAdmin)); err != nil {
return err
}
}
diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go
index c81417fb7f22..00d39f8503c6 100644
--- a/pkg/sql/internal.go
+++ b/pkg/sql/internal.go
@@ -327,8 +327,8 @@ type result struct {
// applyOverrides overrides the respective fields from sd for all the fields set on o.
func applyOverrides(o sessiondata.InternalExecutorOverride, sd *sessiondata.SessionData) {
- if o.User != "" {
- sd.User = o.User
+ if !o.User.Undefined() {
+ sd.UserProto = o.User.EncodeProto()
}
if o.Database != "" {
sd.Database = o.Database
@@ -349,13 +349,13 @@ func (ie *InternalExecutor) maybeRootSessionDataOverride(
) sessiondata.InternalExecutorOverride {
if ie.sessionData == nil {
return sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
ApplicationName: catconstants.InternalAppNamePrefix + "-" + opName,
}
}
o := sessiondata.InternalExecutorOverride{}
- if ie.sessionData.User == "" {
- o.User = security.RootUser
+ if ie.sessionData.User().Undefined() {
+ o.User = security.RootUserName()
}
if ie.sessionData.ApplicationName == "" {
o.ApplicationName = catconstants.InternalAppNamePrefix + "-" + opName
@@ -387,7 +387,7 @@ func (ie *InternalExecutor) execInternal(
sd = ie.s.newSessionData(SessionArgs{})
}
applyOverrides(sessionDataOverride, sd)
- if sd.User == "" {
+ if sd.User().Undefined() {
return result{}, errors.AssertionFailedf("no user specified for internal query")
}
if sd.ApplicationName == "" {
diff --git a/pkg/sql/internal_test.go b/pkg/sql/internal_test.go
index c96447c62e1e..4329e9edf154 100644
--- a/pkg/sql/internal_test.go
+++ b/pkg/sql/internal_test.go
@@ -48,7 +48,7 @@ func TestInternalExecutor(t *testing.T) {
ie := s.InternalExecutor().(*sql.InternalExecutor)
row, err := ie.QueryRowEx(ctx, "test", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SELECT 1")
if err != nil {
t.Fatal(err)
@@ -68,7 +68,7 @@ func TestInternalExecutor(t *testing.T) {
// The following statement will succeed on the 2nd try.
row, err = ie.QueryRowEx(
ctx, "test", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"select case nextval('test.seq') when 1 then crdb_internal.force_retry('1h') else 99 end",
)
if err != nil {
@@ -91,7 +91,7 @@ func TestInternalExecutor(t *testing.T) {
cnt++
row, err = ie.QueryRowEx(
ctx, "test", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"select case nextval('test.seq') when 2 then crdb_internal.force_retry('1h') else 99 end",
)
if cnt == 1 {
@@ -150,8 +150,8 @@ func TestInternalFullTableScan(t *testing.T) {
ie.SetSessionData(
&sessiondata.SessionData{
SessionData: sessiondatapb.SessionData{
- Database: "db",
- User: security.RootUser,
+ Database: "db",
+ UserProto: security.RootUserName().EncodeProto(),
},
LocalOnlySessionData: sessiondata.LocalOnlySessionData{
DisallowFullTableScans: true,
@@ -181,12 +181,12 @@ func TestQueryIsAdminWithNoTxn(t *testing.T) {
ie := s.InternalExecutor().(*sql.InternalExecutor)
testData := []struct {
- user string
+ user security.SQLUsername
expAdmin bool
}{
- {security.NodeUser, true},
- {security.RootUser, true},
- {"testuser", false},
+ {security.NodeUserName(), true},
+ {security.RootUserName(), true},
+ {security.TestUserName(), false},
}
for _, tc := range testData {
@@ -283,8 +283,8 @@ func TestSessionBoundInternalExecutor(t *testing.T) {
ie.SetSessionData(
&sessiondata.SessionData{
SessionData: sessiondatapb.SessionData{
- Database: expDB,
- User: security.RootUser,
+ Database: expDB,
+ UserProto: security.RootUserName().EncodeProto(),
},
SequenceState: &sessiondata.SequenceState{},
})
@@ -351,7 +351,7 @@ func TestInternalExecAppNameInitialization(t *testing.T) {
ie.SetSessionData(
&sessiondata.SessionData{
SessionData: sessiondatapb.SessionData{
- User: security.RootUser,
+ UserProto: security.RootUserName().EncodeProto(),
Database: "defaultdb",
ApplicationName: "appname_findme",
},
@@ -530,7 +530,7 @@ func TestInternalExecutorInLeafTxnDoesNotPanic(t *testing.T) {
ie := s.InternalExecutor().(*sql.InternalExecutor)
_, err := ie.QueryEx(
- ctx, "leaf-query", leafTxn, sessiondata.InternalExecutorOverride{User: security.RootUser}, "SELECT 1",
+ ctx, "leaf-query", leafTxn, sessiondata.InternalExecutorOverride{User: security.RootUserName()}, "SELECT 1",
)
require.NoError(t, err)
}
diff --git a/pkg/sql/lex/encode.go b/pkg/sql/lex/encode.go
index 6868bd2e20dc..94947e23941f 100644
--- a/pkg/sql/lex/encode.go
+++ b/pkg/sql/lex/encode.go
@@ -25,6 +25,7 @@ import (
"encoding/hex"
"unicode/utf8"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
@@ -39,35 +40,10 @@ var mustQuoteMap = map[byte]bool{
'}': true,
}
-// EncodeFlags influence the formatting of strings and identifiers.
-type EncodeFlags int
-
-// HasFlags tests whether the given flags are set.
-func (f EncodeFlags) HasFlags(subset EncodeFlags) bool {
- return f&subset == subset
-}
-
-const (
- // EncNoFlags indicates nothing special should happen while encoding.
- EncNoFlags EncodeFlags = 0
-
- // EncBareStrings indicates that strings will be rendered without
- // wrapping quotes if they contain no special characters.
- EncBareStrings EncodeFlags = 1 << iota
-
- // EncBareIdentifiers indicates that identifiers will be rendered
- // without wrapping quotes.
- EncBareIdentifiers
-
- // EncFirstFreeFlagBit needs to remain unused; it is used as base
- // bit offset for tree.FmtFlags.
- EncFirstFreeFlagBit
-)
-
// EncodeSQLString writes a string literal to buf. All unicode and
// non-printable characters are escaped.
func EncodeSQLString(buf *bytes.Buffer, in string) {
- EncodeSQLStringWithFlags(buf, in, EncNoFlags)
+ EncodeSQLStringWithFlags(buf, in, lexbase.EncNoFlags)
}
// EscapeSQLString returns an escaped SQL representation of the given
@@ -84,11 +60,11 @@ func EscapeSQLString(in string) string {
// the output format: if encodeBareString is set, the output string
// will not be wrapped in quotes if the strings contains no special
// characters.
-func EncodeSQLStringWithFlags(buf *bytes.Buffer, in string, flags EncodeFlags) {
+func EncodeSQLStringWithFlags(buf *bytes.Buffer, in string, flags lexbase.EncodeFlags) {
// See http://www.postgresql.org/docs/9.4/static/sql-syntax-lexical.html
start := 0
escapedString := false
- bareStrings := flags.HasFlags(EncBareStrings)
+ bareStrings := flags.HasFlags(lexbase.EncBareStrings)
// Loop through each unicode code point.
for i, r := range in {
if i < start {
@@ -131,53 +107,6 @@ func EncodeSQLStringWithFlags(buf *bytes.Buffer, in string, flags EncodeFlags) {
}
}
-// EncodeUnrestrictedSQLIdent writes the identifier in s to buf.
-// The identifier is only quoted if the flags don't tell otherwise and
-// the identifier contains special characters.
-func EncodeUnrestrictedSQLIdent(buf *bytes.Buffer, s string, flags EncodeFlags) {
- if flags.HasFlags(EncBareIdentifiers) || isBareIdentifier(s) {
- buf.WriteString(s)
- return
- }
- EncodeEscapedSQLIdent(buf, s)
-}
-
-// EncodeRestrictedSQLIdent writes the identifier in s to buf. The
-// identifier is quoted if either the flags ask for it, the identifier
-// contains special characters, or the identifier is a reserved SQL
-// keyword.
-func EncodeRestrictedSQLIdent(buf *bytes.Buffer, s string, flags EncodeFlags) {
- if flags.HasFlags(EncBareIdentifiers) || (!isReservedKeyword(s) && isBareIdentifier(s)) {
- buf.WriteString(s)
- return
- }
- EncodeEscapedSQLIdent(buf, s)
-}
-
-// EncodeEscapedSQLIdent writes the identifier in s to buf. The
-// identifier is always quoted. Double quotes inside the identifier
-// are escaped.
-func EncodeEscapedSQLIdent(buf *bytes.Buffer, s string) {
- buf.WriteByte('"')
- start := 0
- for i, n := 0, len(s); i < n; i++ {
- ch := s[i]
- // The only character that requires escaping is a double quote.
- if ch == '"' {
- if start != i {
- buf.WriteString(s[start:i])
- }
- start = i + 1
- buf.WriteByte(ch)
- buf.WriteByte(ch) // add extra copy of ch
- }
- }
- if start < len(s) {
- buf.WriteString(s[start:])
- }
- buf.WriteByte('"')
-}
-
// EncodeLocaleName writes the locale identifier in s to buf. Any dash
// characters are mapped to underscore characters. Underscore characters do not
// need to be quoted, and they are considered equivalent to dash characters by
diff --git a/pkg/sql/lex/encode_test.go b/pkg/sql/lex/encode_test.go
index 1abe7966f570..855b20b05c4e 100644
--- a/pkg/sql/lex/encode_test.go
+++ b/pkg/sql/lex/encode_test.go
@@ -18,6 +18,7 @@ import (
"unicode/utf8"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb"
)
@@ -83,45 +84,7 @@ func testEncodeString(t *testing.T, input []byte, encode func(*bytes.Buffer, str
func BenchmarkEncodeSQLString(b *testing.B) {
str := strings.Repeat("foo", 10000)
for i := 0; i < b.N; i++ {
- lex.EncodeSQLStringWithFlags(bytes.NewBuffer(nil), str, lex.EncBareStrings)
- }
-}
-
-func TestEncodeRestrictedSQLIdent(t *testing.T) {
- testCases := []struct {
- input string
- output string
- }{
- {`foo`, `foo`},
- {``, `""`},
- {`3`, `"3"`},
- {`foo3`, `foo3`},
- {`foo"`, `"foo"""`},
- {`fo"o"`, `"fo""o"""`},
- {`fOo`, `"fOo"`},
- {`_foo`, `_foo`},
- {`-foo`, `"-foo"`},
- {`select`, `"select"`},
- {`integer`, `"integer"`},
- // N.B. These type names are examples of type names that *should* be
- // unrestricted (left out of the reserved keyword list) because they're not
- // part of the sql standard type name list. This is important for Postgres
- // compatibility. If you find yourself about to change this, don't - you can
- // convince yourself of such by looking at the output of `quote_ident`
- // against a Postgres instance.
- {`int8`, `int8`},
- {`date`, `date`},
- {`inet`, `inet`},
- }
-
- for _, tc := range testCases {
- var buf bytes.Buffer
- lex.EncodeRestrictedSQLIdent(&buf, tc.input, lex.EncBareStrings)
- out := buf.String()
-
- if out != tc.output {
- t.Errorf("`%s`: expected `%s`, got `%s`", tc.input, tc.output, out)
- }
+ lex.EncodeSQLStringWithFlags(bytes.NewBuffer(nil), str, lexbase.EncBareStrings)
}
}
diff --git a/pkg/sql/lexbase/encode.go b/pkg/sql/lexbase/encode.go
new file mode 100644
index 000000000000..e0412f3a9b67
--- /dev/null
+++ b/pkg/sql/lexbase/encode.go
@@ -0,0 +1,94 @@
+// Copyright 2012, Google Inc. All rights reserved.
+// Use of this source code is governed by a BSD-style
+// license that can be found in licenses/BSD-vitess.txt.
+
+// Portions of this file are additionally subject to the following
+// license and copyright.
+//
+// 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.
+
+// This code was derived from https://github.com/youtube/vitess.
+
+package lexbase
+
+import "bytes"
+
+// EncodeFlags influence the formatting of strings and identifiers.
+type EncodeFlags int
+
+// HasFlags tests whether the given flags are set.
+func (f EncodeFlags) HasFlags(subset EncodeFlags) bool {
+ return f&subset == subset
+}
+
+const (
+ // EncNoFlags indicates nothing special should happen while encoding.
+ EncNoFlags EncodeFlags = 0
+
+ // EncBareStrings indicates that strings will be rendered without
+ // wrapping quotes if they contain no special characters.
+ EncBareStrings EncodeFlags = 1 << iota
+
+ // EncBareIdentifiers indicates that identifiers will be rendered
+ // without wrapping quotes.
+ EncBareIdentifiers
+
+ // EncFirstFreeFlagBit needs to remain unused; it is used as base
+ // bit offset for tree.FmtFlags.
+ EncFirstFreeFlagBit
+)
+
+// EncodeRestrictedSQLIdent writes the identifier in s to buf. The
+// identifier is quoted if either the flags ask for it, the identifier
+// contains special characters, or the identifier is a reserved SQL
+// keyword.
+func EncodeRestrictedSQLIdent(buf *bytes.Buffer, s string, flags EncodeFlags) {
+ if flags.HasFlags(EncBareIdentifiers) || (!isReservedKeyword(s) && isBareIdentifier(s)) {
+ buf.WriteString(s)
+ return
+ }
+ EncodeEscapedSQLIdent(buf, s)
+}
+
+// EncodeUnrestrictedSQLIdent writes the identifier in s to buf.
+// The identifier is only quoted if the flags don't tell otherwise and
+// the identifier contains special characters.
+func EncodeUnrestrictedSQLIdent(buf *bytes.Buffer, s string, flags EncodeFlags) {
+ if flags.HasFlags(EncBareIdentifiers) || isBareIdentifier(s) {
+ buf.WriteString(s)
+ return
+ }
+ EncodeEscapedSQLIdent(buf, s)
+}
+
+// EncodeEscapedSQLIdent writes the identifier in s to buf. The
+// identifier is always quoted. Double quotes inside the identifier
+// are escaped.
+func EncodeEscapedSQLIdent(buf *bytes.Buffer, s string) {
+ buf.WriteByte('"')
+ start := 0
+ for i, n := 0, len(s); i < n; i++ {
+ ch := s[i]
+ // The only character that requires escaping is a double quote.
+ if ch == '"' {
+ if start != i {
+ buf.WriteString(s[start:i])
+ }
+ start = i + 1
+ buf.WriteByte(ch)
+ buf.WriteByte(ch) // add extra copy of ch
+ }
+ }
+ if start < len(s) {
+ buf.WriteString(s[start:])
+ }
+ buf.WriteByte('"')
+}
diff --git a/pkg/sql/lexbase/encode_test.go b/pkg/sql/lexbase/encode_test.go
new file mode 100644
index 000000000000..5d2fd88307a5
--- /dev/null
+++ b/pkg/sql/lexbase/encode_test.go
@@ -0,0 +1,56 @@
+// 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 lexbase_test
+
+import (
+ "bytes"
+ "testing"
+
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
+)
+
+func TestEncodeRestrictedSQLIdent(t *testing.T) {
+ testCases := []struct {
+ input string
+ output string
+ }{
+ {`foo`, `foo`},
+ {``, `""`},
+ {`3`, `"3"`},
+ {`foo3`, `foo3`},
+ {`foo"`, `"foo"""`},
+ {`fo"o"`, `"fo""o"""`},
+ {`fOo`, `"fOo"`},
+ {`_foo`, `_foo`},
+ {`-foo`, `"-foo"`},
+ {`select`, `"select"`},
+ {`integer`, `"integer"`},
+ // N.B. These type names are examples of type names that *should* be
+ // unrestricted (left out of the reserved keyword list) because they're not
+ // part of the sql standard type name list. This is important for Postgres
+ // compatibility. If you find yourself about to change this, don't - you can
+ // convince yourself of such by looking at the output of `quote_ident`
+ // against a Postgres instance.
+ {`int8`, `int8`},
+ {`date`, `date`},
+ {`inet`, `inet`},
+ }
+
+ for _, tc := range testCases {
+ var buf bytes.Buffer
+ lexbase.EncodeRestrictedSQLIdent(&buf, tc.input, lexbase.EncBareStrings)
+ out := buf.String()
+
+ if out != tc.output {
+ t.Errorf("`%s`: expected `%s`, got `%s`", tc.input, tc.output, out)
+ }
+ }
+}
diff --git a/pkg/sql/lex/normalize.go b/pkg/sql/lexbase/normalize.go
similarity index 98%
rename from pkg/sql/lex/normalize.go
rename to pkg/sql/lexbase/normalize.go
index 097d8925eb32..ab2a3cec848f 100644
--- a/pkg/sql/lex/normalize.go
+++ b/pkg/sql/lexbase/normalize.go
@@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
-package lex
+package lexbase
import (
"strings"
diff --git a/pkg/sql/lex/predicates.go b/pkg/sql/lexbase/predicates.go
similarity index 99%
rename from pkg/sql/lex/predicates.go
rename to pkg/sql/lexbase/predicates.go
index 81fefc7ae6b3..408759a256b8 100644
--- a/pkg/sql/lex/predicates.go
+++ b/pkg/sql/lexbase/predicates.go
@@ -8,7 +8,7 @@
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
-package lex
+package lexbase
import (
"unicode"
diff --git a/pkg/sql/lexbase/reserved_keywords.go b/pkg/sql/lexbase/reserved_keywords.go
new file mode 100644
index 000000000000..6c65a67acb26
--- /dev/null
+++ b/pkg/sql/lexbase/reserved_keywords.go
@@ -0,0 +1,160 @@
+// Code generated by reserved_keywords.awk. DO NOT EDIT.
+// GENERATED FILE DO NOT EDIT
+
+package lexbase
+
+var reservedKeywords = map[string]struct{}{
+ "all": {},
+ "analyse": {},
+ "analyze": {},
+ "and": {},
+ "annotate_type": {},
+ "any": {},
+ "array": {},
+ "as": {},
+ "asc": {},
+ "asymmetric": {},
+ "authorization": {},
+ "between": {},
+ "bigint": {},
+ "bit": {},
+ "boolean": {},
+ "both": {},
+ "box2d": {},
+ "case": {},
+ "cast": {},
+ "char": {},
+ "character": {},
+ "characteristics": {},
+ "check": {},
+ "coalesce": {},
+ "cockroachdb_extra_reserved_keyword": {},
+ "collate": {},
+ "collation": {},
+ "column": {},
+ "concurrently": {},
+ "constraint": {},
+ "create": {},
+ "cross": {},
+ "current_catalog": {},
+ "current_date": {},
+ "current_role": {},
+ "current_schema": {},
+ "current_time": {},
+ "current_timestamp": {},
+ "current_user": {},
+ "dec": {},
+ "decimal": {},
+ "default": {},
+ "deferrable": {},
+ "desc": {},
+ "distinct": {},
+ "do": {},
+ "else": {},
+ "end": {},
+ "except": {},
+ "exists": {},
+ "extract": {},
+ "extract_duration": {},
+ "false": {},
+ "family": {},
+ "fetch": {},
+ "float": {},
+ "for": {},
+ "foreign": {},
+ "from": {},
+ "full": {},
+ "geography": {},
+ "geometry": {},
+ "grant": {},
+ "greatest": {},
+ "group": {},
+ "grouping": {},
+ "having": {},
+ "if": {},
+ "iferror": {},
+ "ifnull": {},
+ "ilike": {},
+ "in": {},
+ "index": {},
+ "initially": {},
+ "inner": {},
+ "int": {},
+ "integer": {},
+ "intersect": {},
+ "interval": {},
+ "into": {},
+ "is": {},
+ "iserror": {},
+ "isnull": {},
+ "join": {},
+ "lateral": {},
+ "leading": {},
+ "least": {},
+ "left": {},
+ "like": {},
+ "limit": {},
+ "localtime": {},
+ "localtimestamp": {},
+ "natural": {},
+ "none": {},
+ "not": {},
+ "nothing": {},
+ "notnull": {},
+ "null": {},
+ "nullif": {},
+ "numeric": {},
+ "offset": {},
+ "on": {},
+ "only": {},
+ "or": {},
+ "order": {},
+ "out": {},
+ "outer": {},
+ "overlaps": {},
+ "overlay": {},
+ "placing": {},
+ "point": {},
+ "polygon": {},
+ "position": {},
+ "precision": {},
+ "primary": {},
+ "real": {},
+ "references": {},
+ "returning": {},
+ "right": {},
+ "row": {},
+ "select": {},
+ "session_user": {},
+ "similar": {},
+ "smallint": {},
+ "some": {},
+ "string": {},
+ "substring": {},
+ "symmetric": {},
+ "table": {},
+ "then": {},
+ "time": {},
+ "timestamp": {},
+ "timestamptz": {},
+ "timetz": {},
+ "to": {},
+ "trailing": {},
+ "treat": {},
+ "trim": {},
+ "true": {},
+ "union": {},
+ "unique": {},
+ "user": {},
+ "using": {},
+ "values": {},
+ "varbit": {},
+ "varchar": {},
+ "variadic": {},
+ "virtual": {},
+ "when": {},
+ "where": {},
+ "window": {},
+ "with": {},
+ "work": {},
+}
diff --git a/pkg/sql/logictest/logic.go b/pkg/sql/logictest/logic.go
index 988e869fcc0d..64ac7647244b 100644
--- a/pkg/sql/logictest/logic.go
+++ b/pkg/sql/logictest/logic.go
@@ -1577,7 +1577,7 @@ CREATE DATABASE test;
t.Fatal(err)
}
- if _, err := t.db.Exec(fmt.Sprintf("CREATE USER %s;", server.TestUser)); err != nil {
+ if _, err := t.db.Exec(fmt.Sprintf("CREATE USER %s;", security.TestUser)); err != nil {
t.Fatal(err)
}
diff --git a/pkg/sql/logictest/testdata/logic_test/drop_user b/pkg/sql/logictest/testdata/logic_test/drop_user
index a71eed0a6cc5..766c6eb173b5 100644
--- a/pkg/sql/logictest/testdata/logic_test/drop_user
+++ b/pkg/sql/logictest/testdata/logic_test/drop_user
@@ -56,7 +56,7 @@ DROP USER IF EXISTS user1
statement error username "node" reserved
DROP USER node
-statement error pq: username "foo☂" invalid
+statement error pq: "foo☂": username is invalid
DROP USER foo☂
statement ok
diff --git a/pkg/sql/logictest/testdata/logic_test/user b/pkg/sql/logictest/testdata/logic_test/user
index 3ddb39fa46ed..bd0283c10555 100644
--- a/pkg/sql/logictest/testdata/logic_test/user
+++ b/pkg/sql/logictest/testdata/logic_test/user
@@ -50,10 +50,10 @@ CREATE USER uSEr2 WITH PASSWORD 'cockroach'
statement ok
CREATE USER user3 WITH PASSWORD '蟑螂'
-statement error pq: username "foo☂" invalid
+statement error pq: "foo☂": username is invalid
CREATE USER foo☂
-statement error pq: username "-foo" invalid
+statement error pq: "-foo": username is invalid
CREATE USER "-foo"
statement error at or near "-": syntax error
diff --git a/pkg/sql/opt/exec/execbuilder/mutation.go b/pkg/sql/opt/exec/execbuilder/mutation.go
index d58aea834494..cf0a2a722d74 100644
--- a/pkg/sql/opt/exec/execbuilder/mutation.go
+++ b/pkg/sql/opt/exec/execbuilder/mutation.go
@@ -16,7 +16,7 @@ import (
"fmt"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
- "github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/mutations"
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
@@ -800,9 +800,9 @@ func mkFKCheckErr(md *opt.Metadata, c *memo.FKChecksItem, keyVals tree.Datums) e
fk := origin.Table.OutboundForeignKey(c.FKOrdinal)
constraintName = fk.Name()
fmt.Fprintf(&msg, "%s on table ", c.OpName)
- lex.EncodeEscapedSQLIdent(&msg, string(origin.Alias.ObjectName))
+ lexbase.EncodeEscapedSQLIdent(&msg, string(origin.Alias.ObjectName))
msg.WriteString(" violates foreign key constraint ")
- lex.EncodeEscapedSQLIdent(&msg, fk.Name())
+ lexbase.EncodeEscapedSQLIdent(&msg, fk.Name())
details.WriteString("Key (")
for i := 0; i < fk.ColumnCount(); i++ {
@@ -831,7 +831,7 @@ func mkFKCheckErr(md *opt.Metadata, c *memo.FKChecksItem, keyVals tree.Datums) e
details.WriteString("MATCH FULL does not allow mixing of null and nonnull key values.")
} else {
details.WriteString(") is not present in table ")
- lex.EncodeEscapedSQLIdent(&details, string(referenced.Alias.ObjectName))
+ lexbase.EncodeEscapedSQLIdent(&details, string(referenced.Alias.ObjectName))
details.WriteByte('.')
}
} else {
@@ -842,11 +842,11 @@ func mkFKCheckErr(md *opt.Metadata, c *memo.FKChecksItem, keyVals tree.Datums) e
fk := referenced.Table.InboundForeignKey(c.FKOrdinal)
constraintName = fk.Name()
fmt.Fprintf(&msg, "%s on table ", c.OpName)
- lex.EncodeEscapedSQLIdent(&msg, string(referenced.Alias.ObjectName))
+ lexbase.EncodeEscapedSQLIdent(&msg, string(referenced.Alias.ObjectName))
msg.WriteString(" violates foreign key constraint ")
- lex.EncodeEscapedSQLIdent(&msg, fk.Name())
+ lexbase.EncodeEscapedSQLIdent(&msg, fk.Name())
msg.WriteString(" on table ")
- lex.EncodeEscapedSQLIdent(&msg, string(origin.Alias.ObjectName))
+ lexbase.EncodeEscapedSQLIdent(&msg, string(origin.Alias.ObjectName))
details.WriteString("Key (")
for i := 0; i < fk.ColumnCount(); i++ {
@@ -864,7 +864,7 @@ func mkFKCheckErr(md *opt.Metadata, c *memo.FKChecksItem, keyVals tree.Datums) e
details.WriteString(d.String())
}
details.WriteString(") is still referenced from table ")
- lex.EncodeEscapedSQLIdent(&details, string(origin.Alias.ObjectName))
+ lexbase.EncodeEscapedSQLIdent(&details, string(origin.Alias.ObjectName))
details.WriteByte('.')
}
diff --git a/pkg/sql/opt/testutils/opttester/opt_tester.go b/pkg/sql/opt/testutils/opttester/opt_tester.go
index 03f025f4238e..d809ca4a3cb1 100644
--- a/pkg/sql/opt/testutils/opttester/opt_tester.go
+++ b/pkg/sql/opt/testutils/opttester/opt_tester.go
@@ -29,6 +29,7 @@ import (
"time"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
@@ -223,7 +224,7 @@ func New(catalog cat.Catalog, sql string) *OptTester {
// Set any OptTester-wide session flags here.
- ot.evalCtx.SessionData.User = "opttester"
+ ot.evalCtx.SessionData.User = security.MakeSQLUsernameFromPreNormalizedString("opttester")
ot.evalCtx.SessionData.Database = "defaultdb"
ot.evalCtx.SessionData.ZigzagJoinEnabled = true
ot.evalCtx.SessionData.OptimizerUseHistograms = true
diff --git a/pkg/sql/parser/reserved_keywords.awk b/pkg/sql/parser/reserved_keywords.awk
index b89410014a2d..d22fed05d0ac 100644
--- a/pkg/sql/parser/reserved_keywords.awk
+++ b/pkg/sql/parser/reserved_keywords.awk
@@ -21,7 +21,7 @@ BEGIN {
print "// Code generated by reserved_keywords.awk. DO NOT EDIT."
print "// GENERATED FILE DO NOT EDIT"
print
- print "package lex"
+ print "package lexbase"
print
print "var reservedKeywords = map[string]struct{}{"
diff --git a/pkg/sql/parser/scan.go b/pkg/sql/parser/scan.go
index 75cd82c631f5..bfde2a4f6f59 100644
--- a/pkg/sql/parser/scan.go
+++ b/pkg/sql/parser/scan.go
@@ -20,6 +20,7 @@ import (
"unsafe"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
)
@@ -111,7 +112,7 @@ func (s *scanner) scan(lval *sqlSymType) {
switch ch {
case '$':
// placeholder? $[0-9]+
- if lex.IsDigit(s.peek()) {
+ if lexbase.IsDigit(s.peek()) {
s.scanPlaceholder(lval)
return
} else if s.scanDollarQuotedString(lval) {
@@ -192,7 +193,7 @@ func (s *scanner) scan(lval *sqlSymType) {
s.pos++
lval.id = DOT_DOT
return
- case lex.IsDigit(t):
+ case lexbase.IsDigit(t):
s.scanNumber(lval, ch)
return
}
@@ -385,11 +386,11 @@ func (s *scanner) scan(lval *sqlSymType) {
return
default:
- if lex.IsDigit(ch) {
+ if lexbase.IsDigit(ch) {
s.scanNumber(lval, ch)
return
}
- if lex.IsIdentStart(ch) {
+ if lexbase.IsIdentStart(ch) {
s.scanIdent(lval)
return
}
@@ -526,7 +527,7 @@ func (s *scanner) scanIdent(lval *sqlSymType) {
isLower = false
}
- if !lex.IsIdentMiddle(ch) {
+ if !lexbase.IsIdentMiddle(ch) {
break
}
@@ -551,7 +552,7 @@ func (s *scanner) scanIdent(lval *sqlSymType) {
lval.str = *(*string)(unsafe.Pointer(&b))
} else {
// The string has unicode in it. No choice but to run Normalize.
- lval.str = lex.NormalizeName(s.in[start:s.pos])
+ lval.str = lexbase.NormalizeName(s.in[start:s.pos])
}
isExperimental := false
@@ -594,7 +595,7 @@ func (s *scanner) scanNumber(lval *sqlSymType, ch int) {
for {
ch := s.peek()
- if (isHex && lex.IsHexDigit(ch)) || lex.IsDigit(ch) {
+ if (isHex && lexbase.IsHexDigit(ch)) || lexbase.IsDigit(ch) {
s.pos++
continue
}
@@ -636,7 +637,7 @@ func (s *scanner) scanNumber(lval *sqlSymType, ch int) {
s.pos++
}
ch = s.peek()
- if !lex.IsDigit(ch) {
+ if !lexbase.IsDigit(ch) {
lval.id = ERROR
lval.str = "invalid floating point literal"
return
@@ -686,7 +687,7 @@ func (s *scanner) scanNumber(lval *sqlSymType, ch int) {
func (s *scanner) scanPlaceholder(lval *sqlSymType) {
start := s.pos
- for lex.IsDigit(s.peek()) {
+ for lexbase.IsDigit(s.peek()) {
s.pos++
}
lval.str = s.in[start:s.pos]
@@ -943,7 +944,7 @@ outer:
default:
// If we haven't found a start tag yet, check whether the current characters is a valid for a tag.
- if !foundStartTag && !lex.IsIdentStart(ch) {
+ if !foundStartTag && !lexbase.IsIdentStart(ch) {
return false
}
s.pos++
diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y
index 024cadaea38f..f81336e706f6 100644
--- a/pkg/sql/parser/sql.y
+++ b/pkg/sql/parser/sql.y
@@ -31,6 +31,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/geo/geopb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
@@ -140,6 +141,12 @@ func (u *sqlSymUnion) strPtr() *string {
func (u *sqlSymUnion) strs() []string {
return u.val.([]string)
}
+func (u *sqlSymUnion) user() security.SQLUsername {
+ return u.val.(security.SQLUsername)
+}
+func (u *sqlSymUnion) users() []security.SQLUsername {
+ return u.val.([]security.SQLUsername)
+}
func (u *sqlSymUnion) newTableIndexName() *tree.TableIndexName {
tn := u.val.(tree.TableIndexName)
return &tn
@@ -1114,8 +1121,11 @@ func (u *sqlSymUnion) survive() tree.Survive {
%type unrestricted_name type_function_name type_function_name_no_crdb_extra
%type non_reserved_word
%type non_reserved_word_or_sconst
-%type role_spec
-%type <[]string> role_spec_list
+%type username_or_sconst
+// TODO(solon): The type for role_spec needs to be updated to fix
+// https://github.com/cockroachdb/cockroach/issues/54696
+%type role_spec
+%type <[]security.SQLUsername> role_spec_list
%type zone_value
%type string_or_placeholder
%type string_or_placeholder_list
@@ -1444,10 +1454,10 @@ alter_database_stmt:
| ALTER DATABASE error // SHOW HELP: ALTER DATABASE
alter_database_owner:
- ALTER DATABASE database_name OWNER TO role_spec
- {
- $$.val = &tree.AlterDatabaseOwner{Name: tree.Name($3), Owner: tree.Name($6)}
- }
+ ALTER DATABASE database_name OWNER TO role_spec
+ {
+ $$.val = &tree.AlterDatabaseOwner{Name: tree.Name($3), Owner: $6.user()}
+ }
alter_database_add_region_stmt:
ALTER DATABASE database_name ADD region_or_regions name_list
@@ -1967,7 +1977,7 @@ alter_table_cmd:
| OWNER TO role_spec
{
$$.val = &tree.AlterTableOwner{
- Owner: tree.Name($3),
+ Owner: $3.user(),
}
}
@@ -2113,7 +2123,7 @@ alter_type_stmt:
$$.val = &tree.AlterType{
Type: $3.unresolvedObjectName(),
Cmd: &tree.AlterTypeOwner{
- Owner: tree.Name($6),
+ Owner: $6.user(),
},
}
}
@@ -2148,18 +2158,44 @@ opt_add_val_placement:
}
role_spec_list:
- role_spec {
- $$.val = []string{$1}
- }
+ role_spec
+ {
+ $$.val = []security.SQLUsername{$1.user()}
+ }
| role_spec_list ',' role_spec
- {
- $$.val = append($1.strs(), $3)
- }
+ {
+ $$.val = append($1.users(), $3.user())
+ }
role_spec:
- non_reserved_word_or_sconst
+ username_or_sconst { $$.val = $1.user() }
| CURRENT_USER
+ {
+ // This is incorrect, see https://github.com/cockroachdb/cockroach/issues/54696
+ $$.val = security.MakeSQLUsernameFromPreNormalizedString($1)
+ }
| SESSION_USER
+ {
+ // This is incorrect, see https://github.com/cockroachdb/cockroach/issues/54696
+ $$.val = security.MakeSQLUsernameFromPreNormalizedString($1)
+ }
+
+username_or_sconst:
+ non_reserved_word
+ {
+ // Username was entered as a SQL keyword, or as a SQL identifier
+ // already subject to case normalization and NFC reduction.
+ // (or is it? In fact, there is a bug here: https://github.com/cockroachdb/cockroach/issues/55396
+ // which needs to be fixed to make this fully correct.)
+ $$.val = security.MakeSQLUsernameFromPreNormalizedString($1)
+ }
+| SCONST
+ {
+ // We use UsernameValidation because username_or_sconst and role_spec
+ // are only used for usernames of existing accounts, not when
+ // creating new users or roles.
+ $$.val, _ = security.MakeSQLUsernameFromUserInput($1, security.UsernameValidation)
+ }
alter_attribute_action_list:
alter_attribute_action
@@ -4125,7 +4161,7 @@ set_rest_more:
/* SKIP DOC */
$$.val = &tree.SetSessionAuthorizationDefault{}
}
-| SESSION AUTHORIZATION non_reserved_word_or_sconst
+| SESSION AUTHORIZATION username_or_sconst
{
return unimplementedWithIssue(sqllex, 40283)
}
@@ -5436,7 +5472,7 @@ create_schema_stmt:
{
$$.val = &tree.CreateSchema{
Schema: tree.Name($3),
- AuthRole: $5,
+ AuthRole: $5.user(),
}
}
| CREATE SCHEMA IF NOT EXISTS opt_schema_name AUTHORIZATION role_spec
@@ -5444,7 +5480,7 @@ create_schema_stmt:
$$.val = &tree.CreateSchema{
Schema: tree.Name($6),
IfNotExists: true,
- AuthRole: $8,
+ AuthRole: $8.user(),
}
}
| CREATE SCHEMA error // SHOW HELP: CREATE SCHEMA
@@ -5471,7 +5507,7 @@ alter_schema_stmt:
$$.val = &tree.AlterSchema{
Schema: tree.Name($3),
Cmd: &tree.AlterSchemaOwner{
- Owner: tree.Name($6),
+ Owner: $6.user(),
},
}
}
@@ -7727,13 +7763,13 @@ multiple_set_clause:
// TO { | CURRENT_USER | SESSION_USER}
// %SeeAlso: DROP OWNED BY
reassign_owned_by_stmt:
- REASSIGN OWNED BY role_spec_list TO role_spec
-{
- $$.val = &tree.ReassignOwnedBy{
- OldRoles: $4.strs(),
- NewRole: $6,
- }
-}
+ REASSIGN OWNED BY role_spec_list TO role_spec
+ {
+ $$.val = &tree.ReassignOwnedBy{
+ OldRoles: $4.users(),
+ NewRole: $6.user(),
+ }
+ }
| REASSIGN OWNED BY error // SHOW HELP: REASSIGN OWNED BY
// %Help: DROP OWNED BY - remove database objects owned by role(s).
@@ -7742,13 +7778,13 @@ reassign_owned_by_stmt:
// [RESTRICT | CASCADE]
// %SeeAlso: REASSIGN OWNED BY
drop_owned_by_stmt:
- DROP OWNED BY role_spec_list opt_drop_behavior
-{
- $$.val = &tree.DropOwnedBy{
- Roles: $4.strs(),
+ DROP OWNED BY role_spec_list opt_drop_behavior
+ {
+ $$.val = &tree.DropOwnedBy{
+ Roles: $4.users(),
DropBehavior: $5.dropBehavior(),
- }
-}
+ }
+ }
| DROP OWNED BY error // SHOW HELP: DROP OWNED BY
// A complete SELECT statement looks like this.
diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go
index 25d22ed81523..ef296df2c13f 100644
--- a/pkg/sql/pg_catalog.go
+++ b/pkg/sql/pg_catalog.go
@@ -476,8 +476,8 @@ https://www.postgresql.org/docs/9.5/catalog-pg-authid.html`,
schema: vtable.PGCatalogAuthID,
populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error {
h := makeOidHasher()
- return forEachRole(ctx, p, func(username string, isRole bool, noLogin bool, rolValidUntil *time.Time) error {
- isRoot := tree.DBool(username == security.RootUser || username == security.AdminRole)
+ return forEachRole(ctx, p, func(username security.SQLUsername, isRole bool, noLogin bool, rolValidUntil *time.Time) error {
+ isRoot := tree.DBool(username.IsRootUser() || username.IsAdminRole())
isRoleDBool := tree.DBool(isRole)
roleCanLogin := tree.DBool(!noLogin)
roleValidUntilValue := tree.DNull
@@ -490,18 +490,18 @@ https://www.postgresql.org/docs/9.5/catalog-pg-authid.html`,
}
return addRow(
- h.UserOid(username), // oid
- tree.NewDName(username), // rolname
- tree.MakeDBool(isRoot), // rolsuper
- tree.MakeDBool(isRoleDBool), // rolinherit. Roles inherit by default.
- tree.MakeDBool(isRoot), // rolcreaterole
- tree.MakeDBool(isRoot), // rolcreatedb
- tree.MakeDBool(roleCanLogin), // rolcanlogin.
- tree.DBoolFalse, // rolreplication
- tree.DBoolFalse, // rolbypassrls
- negOneVal, // rolconnlimit
- passwdStarString, // rolpassword
- roleValidUntilValue, // rolvaliduntil
+ h.UserOid(username), // oid
+ tree.NewDName(username.Normalized()), // rolname
+ tree.MakeDBool(isRoot), // rolsuper
+ tree.MakeDBool(isRoleDBool), // rolinherit. Roles inherit by default.
+ tree.MakeDBool(isRoot), // rolcreaterole
+ tree.MakeDBool(isRoot), // rolcreatedb
+ tree.MakeDBool(roleCanLogin), // rolcanlogin.
+ tree.DBoolFalse, // rolreplication
+ tree.DBoolFalse, // rolbypassrls
+ negOneVal, // rolconnlimit
+ passwdStarString, // rolpassword
+ roleValidUntilValue, // rolvaliduntil
)
})
},
@@ -514,7 +514,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-auth-members.html`,
populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error {
h := makeOidHasher()
return forEachRoleMembership(ctx, p,
- func(roleName, memberName string, isAdmin bool) error {
+ func(roleName, memberName security.SQLUsername, isAdmin bool) error {
return addRow(
h.UserOid(roleName), // roleid
h.UserOid(memberName), // member
@@ -543,7 +543,7 @@ func getOwnerOID(desc catalog.Descriptor) tree.Datum {
func getOwnerName(desc catalog.Descriptor) tree.Datum {
owner := getOwnerOfDesc(desc)
- return tree.NewDName(owner)
+ return tree.NewDName(owner.Normalized())
}
var (
@@ -1625,7 +1625,7 @@ https://www.postgresql.org/docs/9.5/catalog-pg-namespace.html`,
ownerOID = getOwnerOID(sc.Desc)
} else if sc.Kind == catalog.SchemaPublic {
// admin is the owner of the public schema.
- ownerOID = h.UserOid("admin")
+ ownerOID = h.UserOid(security.MakeSQLUsernameFromPreNormalizedString("admin"))
}
return addRow(
h.NamespaceOid(db.GetID(), sc.Name), // oid
@@ -1970,8 +1970,8 @@ https://www.postgresql.org/docs/9.5/view-pg-roles.html`,
// include sensitive information such as password hashes.
h := makeOidHasher()
return forEachRole(ctx, p,
- func(username string, isRole bool, noLogin bool, rolValidUntil *time.Time) error {
- isRoot := tree.DBool(username == security.RootUser || username == security.AdminRole)
+ func(username security.SQLUsername, isRole bool, noLogin bool, rolValidUntil *time.Time) error {
+ isRoot := tree.DBool(username.IsRootUser() || username.IsAdminRole())
isRoleDBool := tree.DBool(isRole)
roleCanLogin := tree.DBool(!noLogin)
roleValidUntilValue := tree.DNull
@@ -1984,20 +1984,20 @@ https://www.postgresql.org/docs/9.5/view-pg-roles.html`,
}
return addRow(
- h.UserOid(username), // oid
- tree.NewDName(username), // rolname
- tree.MakeDBool(isRoot), // rolsuper
- tree.MakeDBool(isRoleDBool), // rolinherit. Roles inherit by default.
- tree.MakeDBool(isRoot), // rolcreaterole
- tree.MakeDBool(isRoot), // rolcreatedb
- tree.DBoolFalse, // rolcatupdate
- tree.MakeDBool(roleCanLogin), // rolcanlogin.
- tree.DBoolFalse, // rolreplication
- negOneVal, // rolconnlimit
- passwdStarString, // rolpassword
- roleValidUntilValue, // rolvaliduntil
- tree.DBoolFalse, // rolbypassrls
- tree.DNull, // rolconfig
+ h.UserOid(username), // oid
+ tree.NewDName(username.Normalized()), // rolname
+ tree.MakeDBool(isRoot), // rolsuper
+ tree.MakeDBool(isRoleDBool), // rolinherit. Roles inherit by default.
+ tree.MakeDBool(isRoot), // rolcreaterole
+ tree.MakeDBool(isRoot), // rolcreatedb
+ tree.DBoolFalse, // rolcatupdate
+ tree.MakeDBool(roleCanLogin), // rolcanlogin.
+ tree.DBoolFalse, // rolreplication
+ negOneVal, // rolconnlimit
+ passwdStarString, // rolpassword
+ roleValidUntilValue, // rolvaliduntil
+ tree.DBoolFalse, // rolbypassrls
+ tree.DNull, // rolconfig
)
})
},
@@ -2367,21 +2367,21 @@ https://www.postgresql.org/docs/9.5/view-pg-user.html`,
populate: func(ctx context.Context, p *planner, _ *dbdesc.Immutable, addRow func(...tree.Datum) error) error {
h := makeOidHasher()
return forEachRole(ctx, p,
- func(username string, isRole bool, noLogin bool, rolValidUntil *time.Time) error {
+ func(username security.SQLUsername, isRole bool, noLogin bool, rolValidUntil *time.Time) error {
if isRole {
return nil
}
- isRoot := tree.DBool(username == security.RootUser)
+ isRoot := tree.DBool(username.IsRootUser())
return addRow(
- tree.NewDName(username), // usename
- h.UserOid(username), // usesysid
- tree.MakeDBool(isRoot), // usecreatedb
- tree.MakeDBool(isRoot), // usesuper
- tree.DBoolFalse, // userepl
- tree.DBoolFalse, // usebypassrls
- passwdStarString, // passwd
- tree.DNull, // valuntil
- tree.DNull, // useconfig
+ tree.NewDName(username.Normalized()), // usename
+ h.UserOid(username), // usesysid
+ tree.MakeDBool(isRoot), // usecreatedb
+ tree.MakeDBool(isRoot), // usesuper
+ tree.DBoolFalse, // userepl
+ tree.DBoolFalse, // usebypassrls
+ passwdStarString, // passwd
+ tree.DNull, // valuntil
+ tree.DNull, // useconfig
)
})
},
@@ -2803,9 +2803,9 @@ func (h oidHasher) RegProc(name string) tree.Datum {
return h.BuiltinOid(name, &overloads[0]).AsRegProc(name)
}
-func (h oidHasher) UserOid(username string) *tree.DOid {
+func (h oidHasher) UserOid(username security.SQLUsername) *tree.DOid {
h.writeTypeTag(userTypeTag)
- h.writeStr(username)
+ h.writeStr(username.Normalized())
return h.getOid()
}
diff --git a/pkg/sql/pgwire/auth_methods.go b/pkg/sql/pgwire/auth_methods.go
index bf7c41f737ec..9856a8f8cadc 100644
--- a/pkg/sql/pgwire/auth_methods.go
+++ b/pkg/sql/pgwire/auth_methods.go
@@ -183,7 +183,7 @@ func authTrust(
_ *sql.ExecutorConfig,
_ *hba.Entry,
) (security.UserAuthHook, error) {
- return func(_ string, _ bool) (func(), error) { return nil, nil }, nil
+ return func(_ security.SQLUsername, _ bool) (func(), error) { return nil, nil }, nil
}
func authReject(
@@ -195,7 +195,7 @@ func authReject(
_ *sql.ExecutorConfig,
_ *hba.Entry,
) (security.UserAuthHook, error) {
- return func(_ string, _ bool) (func(), error) {
+ return func(_ security.SQLUsername, _ bool) (func(), error) {
return nil, errors.New("authentication rejected by configuration")
}, nil
}
diff --git a/pkg/sql/pgwire/auth_test.go b/pkg/sql/pgwire/auth_test.go
index c530f8b79e1d..e5f2840bb94d 100644
--- a/pkg/sql/pgwire/auth_test.go
+++ b/pkg/sql/pgwire/auth_test.go
@@ -170,7 +170,7 @@ func hbaRunTest(t *testing.T, insecure bool) {
}
httpHBAUrl := httpScheme + s.HTTPAddr() + "/debug/hba_conf"
- if _, err := conn.ExecContext(context.Background(), `CREATE USER $1`, server.TestUser); err != nil {
+ if _, err := conn.ExecContext(context.Background(), `CREATE USER $1`, securit.TestUser); err != nil {
t.Fatal(err)
}
@@ -321,7 +321,7 @@ func hbaRunTest(t *testing.T, insecure bool) {
// However, certs are only generated for users "root" and "testuser" specifically.
sqlURL, cleanupFn := sqlutils.PGUrlWithOptionalClientCerts(
t, s.ServingSQLAddr(), t.Name(), url.User(user),
- user == security.RootUser || user == server.TestUser /* withClientCerts */)
+ user == security.RootUser || user == security.TestUser /* withClientCerts */)
defer cleanupFn()
var host, port string
diff --git a/pkg/sql/pgwire/conn.go b/pkg/sql/pgwire/conn.go
index 076329dfb745..2fa909c5e5da 100644
--- a/pkg/sql/pgwire/conn.go
+++ b/pkg/sql/pgwire/conn.go
@@ -22,7 +22,6 @@ import (
"sync/atomic"
"time"
- "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
@@ -214,7 +213,7 @@ func (c *conn) serveImpl(
) {
defer func() { _ = c.conn.Close() }()
- if c.sessionArgs.User == security.RootUser || c.sessionArgs.User == security.NodeUser {
+ if c.sessionArgs.User.IsRootUser() || c.sessionArgs.User.IsNodeUser() {
ctx = logtags.AddTag(ctx, "user", log.Safe(c.sessionArgs.User))
} else {
ctx = logtags.AddTag(ctx, "user", c.sessionArgs.User)
@@ -673,14 +672,14 @@ func (c *conn) sendInitialConnData(
}
// The two following status parameters have no equivalent session
// variable.
- if err := c.sendParamStatus("session_authorization", c.sessionArgs.User); err != nil {
+ if err := c.sendParamStatus("session_authorization", c.sessionArgs.User.Normalized()); err != nil {
return sql.ConnectionHandler{}, err
}
// TODO(knz): this should retrieve the admin status during
// authentication using the roles table, instead of using a
// simple/naive username match.
- isSuperUser := c.sessionArgs.User == security.RootUser
+ isSuperUser := c.sessionArgs.User.IsRootUser()
superUserVal := "off"
if isSuperUser {
superUserVal = "on"
diff --git a/pkg/sql/pgwire/conn_test.go b/pkg/sql/pgwire/conn_test.go
index 7dc8ea3a699c..056ac846ae06 100644
--- a/pkg/sql/pgwire/conn_test.go
+++ b/pkg/sql/pgwire/conn_test.go
@@ -434,7 +434,7 @@ func execQuery(
) error {
rows, cols, err := s.InternalExecutor().(sqlutil.InternalExecutor).QueryWithCols(
ctx, "test", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser, Database: "system"},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName(), Database: "system"},
query,
)
if err != nil {
diff --git a/pkg/sql/pgwire/hba/hba.go b/pkg/sql/pgwire/hba/hba.go
index 60455015918b..901492b7cd07 100644
--- a/pkg/sql/pgwire/hba/hba.go
+++ b/pkg/sql/pgwire/hba/hba.go
@@ -24,6 +24,7 @@ import (
"reflect"
"strings"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/errors"
"github.com/olekukonko/tablewriter"
@@ -211,12 +212,12 @@ func (h Entry) ConnMatches(clientConn ConnType, ip net.IP) (bool, error) {
// The provided username must be normalized already.
// The function assumes the entry was normalized to contain only
// one user and its username normalized. See ParseAndNormalize().
-func (h Entry) UserMatches(userName string) bool {
+func (h Entry) UserMatches(userName security.SQLUsername) bool {
if h.User == nil {
return true
}
for _, u := range h.User {
- if u.Value == userName {
+ if u.Value == userName.Normalized() {
return true
}
}
diff --git a/pkg/sql/pgwire/server.go b/pkg/sql/pgwire/server.go
index 34f58c5459db..6bfb41951fe7 100644
--- a/pkg/sql/pgwire/server.go
+++ b/pkg/sql/pgwire/server.go
@@ -20,6 +20,7 @@ import (
"time"
"github.com/cockroachdb/cockroach/pkg/base"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
@@ -29,7 +30,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
- "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/util/contextutil"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
@@ -644,7 +644,7 @@ func parseClientProvidedSessionParameters(
// case-insensitive. Therefore we need to normalize the username
// here, so that further lookups for authentication have the correct
// identifier.
- args.User = tree.Name(value).Normalize()
+ args.User, _ = security.MakeSQLUsernameFromUserInput(value, security.UsernameValidation)
case "results_buffer_size":
if args.ConnResultsBufferSize, err = humanizeutil.ParseBytes(value); err != nil {
diff --git a/pkg/sql/plan_opt.go b/pkg/sql/plan_opt.go
index 661b7b0288a3..13b00b9d80c0 100644
--- a/pkg/sql/plan_opt.go
+++ b/pkg/sql/plan_opt.go
@@ -15,7 +15,6 @@ import (
"strings"
"github.com/cockroachdb/cockroach/pkg/keys"
- "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -359,14 +358,14 @@ func (opc *optPlanningCtx) buildReusableMemo(ctx context.Context) (_ *memo.Memo,
)
}
- if p.SessionData().User != security.RootUser {
+ if !p.SessionData().User().IsRootUser() {
return nil, pgerror.New(pgcode.InsufficientPrivilege,
"PREPARE AS OPT PLAN may only be used by root",
)
}
}
- if p.SessionData().SaveTablesPrefix != "" && p.SessionData().User != security.RootUser {
+ if p.SessionData().SaveTablesPrefix != "" && !p.SessionData().User().IsRootUser() {
return nil, pgerror.New(pgcode.InsufficientPrivilege,
"sub-expression tables creation may only be used by root",
)
diff --git a/pkg/sql/planhook.go b/pkg/sql/planhook.go
index 6e91934a6c53..da1cd8d157ae 100644
--- a/pkg/sql/planhook.go
+++ b/pkg/sql/planhook.go
@@ -13,6 +13,7 @@ package sql
import (
"context"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -85,11 +86,11 @@ type PlanHookState interface {
TypeAsStringOpts(
ctx context.Context, opts tree.KVOptions, optsValidate map[string]KVStringOptValidate,
) (func() (map[string]string, error), error)
- User() string
+ User() security.SQLUsername
AuthorizationAccessor
// The role create/drop call into OSS code to reuse plan nodes.
// TODO(mberhault): it would be easier to just pass a planner to plan hooks.
- GetAllRoles(ctx context.Context) (map[string]bool, error)
+ GetAllRoles(ctx context.Context) (map[security.SQLUsername]bool, error)
BumpRoleMembershipTableVersion(ctx context.Context) error
EvalAsOfTimestamp(ctx context.Context, asOf tree.AsOfClause) (hlc.Timestamp, error)
ResolveUncachedDatabaseByName(
diff --git a/pkg/sql/planner.go b/pkg/sql/planner.go
index a56c30387693..c6bfee9e0a7b 100644
--- a/pkg/sql/planner.go
+++ b/pkg/sql/planner.go
@@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -237,7 +238,11 @@ var noteworthyInternalMemoryUsageBytes = envutil.EnvOrDefaultInt64("COCKROACH_NO
// NewInternalPlanner is an exported version of newInternalPlanner. It
// returns an interface{} so it can be used outside of the sql package.
func NewInternalPlanner(
- opName string, txn *kv.Txn, user string, memMetrics *MemoryMetrics, execCfg *ExecutorConfig,
+ opName string,
+ txn *kv.Txn,
+ user security.SQLUsername,
+ memMetrics *MemoryMetrics,
+ execCfg *ExecutorConfig,
) (interface{}, func()) {
return newInternalPlanner(opName, txn, user, memMetrics, execCfg)
}
@@ -251,7 +256,11 @@ func NewInternalPlanner(
// Returns a cleanup function that must be called once the caller is done with
// the planner.
func newInternalPlanner(
- opName string, txn *kv.Txn, user string, memMetrics *MemoryMetrics, execCfg *ExecutorConfig,
+ opName string,
+ txn *kv.Txn,
+ user security.SQLUsername,
+ memMetrics *MemoryMetrics,
+ execCfg *ExecutorConfig,
) (*planner, func()) {
// We need a context that outlives all the uses of the planner (since the
// planner captures it in the EvalCtx, and so does the cleanup function that
@@ -264,8 +273,8 @@ func newInternalPlanner(
sd := &sessiondata.SessionData{
SessionData: sessiondatapb.SessionData{
- Database: "system",
- User: user,
+ Database: "system",
+ UserProto: user.EncodeProto(),
},
SearchPath: sessiondata.DefaultSearchPathForUser(user),
SequenceState: sessiondata.NewSequenceState(),
@@ -456,8 +465,8 @@ func (p *planner) Txn() *kv.Txn {
return p.txn
}
-func (p *planner) User() string {
- return p.SessionData().User
+func (p *planner) User() security.SQLUsername {
+ return p.SessionData().User()
}
func (p *planner) TemporarySchemaName() string {
diff --git a/pkg/sql/privileged_accessor.go b/pkg/sql/privileged_accessor.go
index 97ac324fa4ff..f8873ba1a385 100644
--- a/pkg/sql/privileged_accessor.go
+++ b/pkg/sql/privileged_accessor.go
@@ -48,7 +48,7 @@ func (p *planner) LookupNamespaceID(
ctx,
"crdb-internal-get-descriptor-id",
p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
query,
parentID,
name,
@@ -83,7 +83,7 @@ func (p *planner) LookupZoneConfigByNamespaceID(
ctx,
"crdb-internal-get-zone",
p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
query,
id,
)
diff --git a/pkg/sql/rename_database.go b/pkg/sql/rename_database.go
index e8c8d583a7fc..a45b1d0128c7 100644
--- a/pkg/sql/rename_database.go
+++ b/pkg/sql/rename_database.go
@@ -14,6 +14,7 @@ import (
"context"
"fmt"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
@@ -256,9 +257,9 @@ func (n *renameDatabaseNode) startExec(params runParams) error {
struct {
DatabaseName string
Statement string
- User string
+ User security.SQLUsername
NewDatabaseName string
- }{n.n.Name.String(), n.n.String(), p.SessionData().User, n.newName},
+ }{n.n.Name.String(), n.n.String(), p.SessionData().User(), n.newName},
)
}
diff --git a/pkg/sql/rename_table.go b/pkg/sql/rename_table.go
index 6decdf9ed2aa..c7d92c9d5c91 100644
--- a/pkg/sql/rename_table.go
+++ b/pkg/sql/rename_table.go
@@ -14,6 +14,7 @@ import (
"context"
"github.com/cockroachdb/cockroach/pkg/kv"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
@@ -252,9 +253,9 @@ func (n *renameTableNode) startExec(params runParams) error {
struct {
TableName string
Statement string
- User string
+ User security.SQLUsername
NewTableName string
- }{oldTn.FQString(), n.n.String(), params.SessionData().User, newTn.FQString()},
+ }{oldTn.FQString(), n.n.String(), params.SessionData().User(), newTn.FQString()},
)
}
diff --git a/pkg/sql/reparent_database.go b/pkg/sql/reparent_database.go
index 301f8fd27577..3290af5bfb5c 100644
--- a/pkg/sql/reparent_database.go
+++ b/pkg/sql/reparent_database.go
@@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/clusterversion"
"github.com/cockroachdb/cockroach/pkg/keys"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/dbdesc"
@@ -298,8 +299,8 @@ func (n *reparentDatabaseNode) startExec(params runParams) error {
struct {
DatabaseName string
NewDatabaseName string
- User string
- }{n.db.Name, n.newParent.Name, p.SessionData().User},
+ User security.SQLUsername
+ }{n.db.Name, n.newParent.Name, p.SessionData().User()},
)
}
diff --git a/pkg/sql/revoke_role.go b/pkg/sql/revoke_role.go
index f49d0f55fb57..2ac4e4e5d098 100644
--- a/pkg/sql/revoke_role.go
+++ b/pkg/sql/revoke_role.go
@@ -56,15 +56,21 @@ func (p *planner) RevokeRoleNode(ctx context.Context, n *tree.RevokeRole) (*Revo
if err != nil {
return nil, err
}
- for _, r := range n.Roles {
+ for i := range n.Roles {
+ // TODO(solon): there are SQL identifiers (tree.Name) in n.Roles,
+ // but we want SQL usernames. Do we normalize or not? For reference,
+ // REASSIGN / OWNER TO do normalize.
+ // Related: https://github.com/cockroachdb/cockroach/issues/54696
+ r := security.MakeSQLUsernameFromPreNormalizedString(string(n.Roles[i]))
+
// If the user is an admin, don't check if the user is allowed to add/drop
// roles in the role. However, if the role being modified is the admin role, then
// make sure the user is an admin with the admin option.
- if hasAdminRole && string(r) != security.AdminRole {
+ if hasAdminRole && !r.IsAdminRole() {
continue
}
- if isAdmin, ok := allRoles[string(r)]; !ok || !isAdmin {
- if string(r) == security.AdminRole {
+ if isAdmin, ok := allRoles[r]; !ok || !isAdmin {
+ if r.IsAdminRole() {
return nil, pgerror.Newf(pgcode.InsufficientPrivilege,
"%s is not a role admin for role %s", p.User(), r)
}
@@ -81,15 +87,27 @@ func (p *planner) RevokeRoleNode(ctx context.Context, n *tree.RevokeRole) (*Revo
return nil, err
}
- for _, r := range n.Roles {
- if _, ok := roles[string(r)]; !ok {
- return nil, pgerror.Newf(pgcode.UndefinedObject, "role/user %s does not exist", r)
+ for i := range n.Roles {
+ // TODO(solon): there are SQL identifiers (tree.Name) in n.Roles,
+ // but we want SQL usernames. Do we normalize or not? For reference,
+ // REASSIGN / OWNER TO do normalize.
+ // Related: https://github.com/cockroachdb/cockroach/issues/54696
+ r := security.MakeSQLUsernameFromPreNormalizedString(string(n.Roles[i]))
+
+ if _, ok := roles[r]; !ok {
+ return nil, pgerror.Newf(pgcode.UndefinedObject, "role/user %s does not exist", n.Roles[i])
}
}
- for _, m := range n.Members {
- if _, ok := roles[string(m)]; !ok {
- return nil, pgerror.Newf(pgcode.UndefinedObject, "role/user %s does not exist", m)
+ for i := range n.Members {
+ // TODO(solon): there are SQL identifiers (tree.Name) in n.Roles,
+ // but we want SQL usernames. Do we normalize or not? For reference,
+ // REASSIGN / OWNER TO do normalize.
+ // Related: https://github.com/cockroachdb/cockroach/issues/54696
+ m := security.MakeSQLUsernameFromPreNormalizedString(string(n.Members[i]))
+
+ if _, ok := roles[m]; !ok {
+ return nil, pgerror.Newf(pgcode.UndefinedObject, "role/user %s does not exist", n.Members[i])
}
}
@@ -113,9 +131,18 @@ func (n *RevokeRoleNode) startExec(params runParams) error {
}
var rowsAffected int
- for _, r := range n.roles {
- for _, m := range n.members {
- if string(r) == security.AdminRole && string(m) == security.RootUser {
+ for i := range n.roles {
+ // TODO(solon): there are SQL identifiers (tree.Name) in
+ // n.Roles, but we want SQL usernames. Do we normalize or not? For
+ // reference, REASSIGN / OWNER TO do normalize. Related:
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ r := security.MakeSQLUsernameFromPreNormalizedString(string(n.roles[i]))
+
+ for j := range n.members {
+ // TODO(solon): ditto above, names in n.members.
+ m := security.MakeSQLUsernameFromPreNormalizedString(string(n.members[j]))
+
+ if r.IsAdminRole() && m.IsRootUser() {
// We use CodeObjectInUseError which is what happens if you tried to delete the current user in pg.
return pgerror.Newf(pgcode.ObjectInUse,
"role/user %s cannot be removed from role %s or lose the ADMIN OPTION",
@@ -125,9 +152,9 @@ func (n *RevokeRoleNode) startExec(params runParams) error {
params.ctx,
opName,
params.p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
memberStmt,
- r, m,
+ r.Normalized(), m.Normalized(),
)
if err != nil {
return err
diff --git a/pkg/sql/rowenc/testutils.go b/pkg/sql/rowenc/testutils.go
index 7a879f2db915..b280a3c17f02 100644
--- a/pkg/sql/rowenc/testutils.go
+++ b/pkg/sql/rowenc/testutils.go
@@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/geo/geopb"
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -1039,7 +1040,9 @@ type Mutator interface {
}
// MakeSchemaName creates a CreateSchema definition
-func MakeSchemaName(ifNotExists bool, schema string, authRole string) *tree.CreateSchema {
+func MakeSchemaName(
+ ifNotExists bool, schema string, authRole security.SQLUsername,
+) *tree.CreateSchema {
return &tree.CreateSchema{
IfNotExists: ifNotExists,
Schema: tree.Name(schema),
diff --git a/pkg/sql/schema_changer.go b/pkg/sql/schema_changer.go
index ed27669826cb..c5f93b26bfb1 100644
--- a/pkg/sql/schema_changer.go
+++ b/pkg/sql/schema_changer.go
@@ -244,7 +244,7 @@ func (sc *SchemaChanger) backfillQueryIntoTable(
// Create an internal planner as the planner used to serve the user query
// would have committed by this point.
- p, cleanup := NewInternalPlanner(desc, txn, security.RootUser, &MemoryMetrics{}, sc.execCfg)
+ p, cleanup := NewInternalPlanner(desc, txn, security.RootUserName(), &MemoryMetrics{}, sc.execCfg)
defer cleanup()
localPlanner := p.(*planner)
stmt, err := parser.ParseOne(query)
@@ -464,7 +464,7 @@ func startGCJob(
ctx context.Context,
db *kv.DB,
jobRegistry *jobs.Registry,
- username string,
+ username security.SQLUsername,
schemaChangeDescription string,
details jobspb.SchemaChangeGCDetails,
) error {
@@ -640,7 +640,7 @@ func (sc *SchemaChanger) exec(ctx context.Context) error {
},
}
if err := startGCJob(
- ctx, sc.db, sc.jobRegistry, sc.job.Payload().Username, sc.job.Payload().Description, gcDetails,
+ ctx, sc.db, sc.jobRegistry, sc.job.Payload().UsernameProto.Decode(), sc.job.Payload().Description, gcDetails,
); err != nil {
return err
}
@@ -856,7 +856,7 @@ func (sc *SchemaChanger) rollbackSchemaChange(ctx context.Context, err error) er
// Queue a GC job.
jobRecord := CreateGCJobRecord(
"ROLLBACK OF "+sc.job.Payload().Description,
- sc.job.Payload().Description,
+ sc.job.Payload().UsernameProto.Decode(),
jobspb.SchemaChangeGCDetails{
Tables: []jobspb.SchemaChangeGCDetails_DroppedID{
{
@@ -981,7 +981,7 @@ func (sc *SchemaChanger) createIndexGCJob(
ParentID: sc.descID,
}
- gcJobRecord := CreateGCJobRecord(jobDesc, sc.job.Payload().Username, indexGCDetails)
+ gcJobRecord := CreateGCJobRecord(jobDesc, sc.job.Payload().UsernameProto.Decode(), indexGCDetails)
indexGCJob, err := sc.jobRegistry.CreateStartableJobWithTxn(ctx, gcJobRecord, txn, nil /* resultsCh */)
if err != nil {
return nil, err
@@ -1823,7 +1823,7 @@ func (sc *SchemaChanger) reverseMutation(
// CreateGCJobRecord creates the job record for a GC job, setting some
// properties which are common for all GC jobs.
func CreateGCJobRecord(
- originalDescription string, username string, details jobspb.SchemaChangeGCDetails,
+ originalDescription string, username security.SQLUsername, details jobspb.SchemaChangeGCDetails,
) jobs.Record {
descriptorIDs := make([]descpb.ID, 0)
if len(details.Indexes) > 0 {
@@ -2034,10 +2034,10 @@ func newFakeSessionData() *sessiondata.SessionData {
// And in fact it is used by `current_schemas()`, which, although is a pure
// function, takes arguments which might be impure (so it can't always be
// pre-evaluated).
- Database: "",
- User: security.NodeUser,
+ Database: "",
+ UserProto: security.NodeUserName().EncodeProto(),
},
- SearchPath: sessiondata.DefaultSearchPathForUser(security.NodeUser),
+ SearchPath: sessiondata.DefaultSearchPathForUser(security.NodeUserName()),
SequenceState: sessiondata.NewSequenceState(),
Location: time.UTC,
}
@@ -2203,7 +2203,7 @@ func (r schemaChangeResumer) Resume(
ctx,
p.ExecCfg().DB,
p.ExecCfg().JobRegistry,
- r.job.Payload().Username,
+ r.job.Payload().UsernameProto.Decode(),
r.job.Payload().Description,
multiTableGCDetails,
); err != nil {
@@ -2348,7 +2348,7 @@ func (sc *SchemaChanger) queueCleanupJobs(
if len(spanList) > 0 {
jobRecord := jobs.Record{
Description: fmt.Sprintf("CLEANUP JOB for '%s'", sc.job.Payload().Description),
- Username: sc.job.Payload().Username,
+ Username: sc.job.Payload().UsernameProto.Decode(),
DescriptorIDs: descpb.IDs{scDesc.GetID()},
Details: jobspb.SchemaChangeDetails{
DescID: sc.descID,
diff --git a/pkg/sql/schema_changer_test.go b/pkg/sql/schema_changer_test.go
index 66eff6752412..7e9e9ce701d8 100644
--- a/pkg/sql/schema_changer_test.go
+++ b/pkg/sql/schema_changer_test.go
@@ -1938,7 +1938,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT8);
status = jobs.StatusFailed
}
if err := jobutils.VerifySystemJob(t, &runner, migrationJobOffset+i, jobspb.TypeSchemaChange, status, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: tc.sql,
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -1953,7 +1953,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT8);
// Roll back job.
if err := jobutils.VerifySystemJob(t, &runner, len(testCases), jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf("ROLL BACK JOB %d: %s", jobID, testCases[jobRolledBack].sql),
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -3143,7 +3143,7 @@ CREATE TABLE t.test (k INT NOT NULL, v INT);
testutils.SucceedsSoon(t, func() error {
return jobutils.VerifySystemJob(t, sqlRun, 1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
Description: "CLEANUP JOB for 'ALTER TABLE t.public.test ALTER PRIMARY KEY USING COLUMNS (k)'",
- Username: security.RootUser,
+ Username: security.RootUserName(),
DescriptorIDs: descpb.IDs{tableDesc.ID},
})
})
@@ -4007,7 +4007,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT, pi DECIMAL DEFAULT (DECIMAL '3.14
testutils.SucceedsSoon(t, func() error {
return jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChangeGC, jobs.StatusRunning, jobs.Record{
Description: "GC for TRUNCATE TABLE t.public.test",
- Username: security.RootUser,
+ Username: security.RootUserName(),
DescriptorIDs: descpb.IDs{tableDesc.ID},
})
})
@@ -4136,7 +4136,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT, pi DECIMAL REFERENCES t.pi (d) DE
// most recent job instead.
schemaChangeJobOffset := 0
if err := jobutils.VerifySystemJob(t, sqlRun, schemaChangeJobOffset+2, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: "TRUNCATE TABLE t.public.test",
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -4311,7 +4311,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT);
sqlRun := sqlutils.MakeSQLRunner(sqlDB)
if err := jobutils.VerifySystemJob(t, sqlRun, 0, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: add_column,
DescriptorIDs: descpb.IDs{
oldID,
@@ -4320,7 +4320,7 @@ CREATE TABLE t.test (k INT PRIMARY KEY, v INT);
t.Fatal(err)
}
if err := jobutils.VerifySystemJob(t, sqlRun, 1, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: drop_column,
DescriptorIDs: descpb.IDs{
oldID,
@@ -4690,7 +4690,7 @@ func TestCancelSchemaChange(t *testing.T) {
t.Fatalf("unexpected %v", err)
}
if err := jobutils.VerifySystemJob(t, sqlDB, idx, jobspb.TypeSchemaChange, jobs.StatusCanceled, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: tc.sql,
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -4701,7 +4701,7 @@ func TestCancelSchemaChange(t *testing.T) {
jobID := jobutils.GetJobID(t, sqlDB, idx)
idx++
jobRecord := jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: fmt.Sprintf("ROLL BACK JOB %d: %s", jobID, tc.sql),
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -4719,7 +4719,7 @@ func TestCancelSchemaChange(t *testing.T) {
} else {
sqlDB.Exec(t, tc.sql)
if err := jobutils.VerifySystemJob(t, sqlDB, idx, jobspb.TypeSchemaChange, jobs.StatusSucceeded, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: tc.sql,
DescriptorIDs: descpb.IDs{
tableDesc.ID,
@@ -5762,7 +5762,7 @@ INSERT INTO t.test (k, v) VALUES (1, 99), (2, 100);
// TODO (lucy): Maybe this test API should use an offset starting
// from the most recent job instead.
return jobutils.VerifyRunningSystemJob(t, sqlRun, 0, jobspb.TypeSchemaChange, sql.RunningStatusValidation, jobs.Record{
- Username: security.RootUser,
+ Username: security.RootUserName(),
Description: "ALTER TABLE t.public.test ADD COLUMN a INT8 AS (v - 1) STORED, ADD CHECK ((a < v) AND (a IS NOT NULL))",
DescriptorIDs: descpb.IDs{
tableDesc.ID,
diff --git a/pkg/sql/sem/builtins/builtins.go b/pkg/sql/sem/builtins/builtins.go
index ebc0de64f08d..b1abd216c633 100644
--- a/pkg/sql/sem/builtins/builtins.go
+++ b/pkg/sql/sem/builtins/builtins.go
@@ -36,13 +36,13 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
- "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkeys"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catalogkv"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
@@ -1629,7 +1629,7 @@ var builtins = map[string]builtinDefinition{
stringOverload1(
func(evalCtx *tree.EvalContext, s string) (tree.Datum, error) {
var buf bytes.Buffer
- lex.EncodeRestrictedSQLIdent(&buf, s, lex.EncNoFlags)
+ lexbase.EncodeRestrictedSQLIdent(&buf, s, lexbase.EncNoFlags)
return tree.NewDString(buf.String()), nil
},
types.String,
@@ -3350,10 +3350,10 @@ may increase either contention or retry errors, or both.`,
Types: tree.ArgTypes{},
ReturnType: tree.FixedReturnType(types.String),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
- if len(ctx.SessionData.User) == 0 {
+ if ctx.SessionData.User().Undefined() {
return tree.DNull, nil
}
- return tree.NewDString(ctx.SessionData.User), nil
+ return tree.NewDString(ctx.SessionData.User().Normalized()), nil
},
Info: "Returns the current user. This function is provided for " +
"compatibility with PostgreSQL.",
@@ -6345,7 +6345,7 @@ var errInsufficientPriv = pgerror.New(
)
func checkPrivilegedUser(ctx *tree.EvalContext) error {
- if ctx.SessionData.User != security.RootUser {
+ if !ctx.SessionData.User().IsRootUser() {
return errInsufficientPriv
}
return nil
diff --git a/pkg/sql/sem/builtins/pg_builtins.go b/pkg/sql/sem/builtins/pg_builtins.go
index f24d021bd868..8013b4ba8333 100644
--- a/pkg/sql/sem/builtins/pg_builtins.go
+++ b/pkg/sql/sem/builtins/pg_builtins.go
@@ -16,6 +16,7 @@ import (
"time"
"github.com/cockroachdb/cockroach/pkg/keys"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
@@ -313,7 +314,7 @@ var strOrOidTypes = []*types.T{types.String, types.Oid}
func makePGPrivilegeInquiryDef(
infoDetail string,
objSpecArgs argTypeOpts,
- fn func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error),
+ fn func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error),
) builtinDefinition {
// Collect the different argument type variations.
//
@@ -359,16 +360,18 @@ func makePGPrivilegeInquiryDef(
Types: argType,
ReturnType: tree.FixedReturnType(types.Bool),
Fn: func(ctx *tree.EvalContext, args tree.Datums) (tree.Datum, error) {
- var user string
+ var user security.SQLUsername
if withUser {
- var err error
-
arg := tree.UnwrapDatum(ctx, args[0])
- user, err = getNameForArg(ctx, arg, "pg_roles", "rolname")
+ userS, err := getNameForArg(ctx, arg, "pg_roles", "rolname")
if err != nil {
return nil, err
}
- if user == "" {
+ // Note: the username in pg_roles is already normalized, so
+ // we can safely turn it into a SQLUsername without
+ // re-normalization.
+ user = security.MakeSQLUsernameFromPreNormalizedString(userS)
+ if user.Undefined() {
if _, ok := arg.(*tree.DOid); ok {
// Postgres returns falseifn no matching user is
// found when given an OID.
@@ -381,11 +384,11 @@ func makePGPrivilegeInquiryDef(
// Remove the first argument.
args = args[1:]
} else {
- if len(ctx.SessionData.User) == 0 {
+ if ctx.SessionData.User().Undefined() {
// Wut... is this possible?
return tree.DNull, nil
}
- user = ctx.SessionData.User
+ user = ctx.SessionData.User()
}
return fn(ctx, args, user)
},
@@ -517,7 +520,12 @@ func parsePrivilegeStr(arg tree.Datum, availOpts pgPrivList) (tree.Datum, error)
// privilege check should also test whether the privilege is held with grant
// option.
func evalPrivilegeCheck(
- ctx *tree.EvalContext, infoTable, user, pred string, priv privilege.Kind, withGrantOpt bool,
+ ctx *tree.EvalContext,
+ infoTable string,
+ user security.SQLUsername,
+ pred string,
+ priv privilege.Kind,
+ withGrantOpt bool,
) (tree.Datum, error) {
privChecks := []privilege.Kind{priv}
if withGrantOpt {
@@ -531,7 +539,7 @@ func evalPrivilegeCheck(
// TODO(mberhault): "public" is a constant defined in sql/sqlbase, but importing that
// would cause a dependency cycle sqlbase -> sem/transform -> sem/builtins -> sqlbase
r, err := ctx.InternalExecutor.QueryRow(
- ctx.Ctx(), "eval-privilege-check", ctx.Txn, query, "public", user,
+ ctx.Ctx(), "eval-privilege-check", ctx.Txn, query, "public", user.Normalized(),
)
if err != nil {
return nil, err
@@ -1155,7 +1163,7 @@ SELECT description
"has_any_column_privilege": makePGPrivilegeInquiryDef(
"any column of table",
argTypeOpts{{"table", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
tableArg := tree.UnwrapDatum(ctx, args[0])
tn, err := getTableNameForArg(ctx, tableArg)
if err != nil {
@@ -1209,7 +1217,7 @@ SELECT description
"has_column_privilege": makePGPrivilegeInquiryDef(
"column",
argTypeOpts{{"table", strOrOidTypes}, {"column", []*types.T{types.String, types.Int}}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
tableArg := tree.UnwrapDatum(ctx, args[0])
tn, err := getTableNameForArg(ctx, tableArg)
if err != nil {
@@ -1291,7 +1299,7 @@ SELECT description
"has_database_privilege": makePGPrivilegeInquiryDef(
"database",
argTypeOpts{{"database", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
dbArg := tree.UnwrapDatum(ctx, args[0])
db, err := getNameForArg(ctx, dbArg, "pg_database", "datname")
if err != nil {
@@ -1347,7 +1355,7 @@ SELECT description
"has_foreign_data_wrapper_privilege": makePGPrivilegeInquiryDef(
"foreign-data wrapper",
argTypeOpts{{"fdw", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
fdwArg := tree.UnwrapDatum(ctx, args[0])
fdw, err := getNameForArg(ctx, fdwArg, "pg_foreign_data_wrapper", "fdwname")
if err != nil {
@@ -1376,7 +1384,7 @@ SELECT description
"has_function_privilege": makePGPrivilegeInquiryDef(
"function",
argTypeOpts{{"function", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
oidArg := tree.UnwrapDatum(ctx, args[0])
// When specifying a function by a text string rather than by OID,
// the allowed input is the same as for the regprocedure data type.
@@ -1418,7 +1426,7 @@ SELECT description
"has_language_privilege": makePGPrivilegeInquiryDef(
"language",
argTypeOpts{{"language", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
langArg := tree.UnwrapDatum(ctx, args[0])
lang, err := getNameForArg(ctx, langArg, "pg_language", "lanname")
if err != nil {
@@ -1452,7 +1460,7 @@ SELECT description
"has_schema_privilege": makePGPrivilegeInquiryDef(
"schema",
argTypeOpts{{"schema", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
schemaArg := tree.UnwrapDatum(ctx, args[0])
schema, err := getNameForArg(ctx, schemaArg, "pg_namespace", "nspname")
if err != nil {
@@ -1499,7 +1507,7 @@ SELECT description
"has_sequence_privilege": makePGPrivilegeInquiryDef(
"sequence",
argTypeOpts{{"sequence", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
seqArg := tree.UnwrapDatum(ctx, args[0])
tn, err := getTableNameForArg(ctx, seqArg)
if err != nil {
@@ -1559,7 +1567,7 @@ SELECT description
"has_server_privilege": makePGPrivilegeInquiryDef(
"foreign server",
argTypeOpts{{"server", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
serverArg := tree.UnwrapDatum(ctx, args[0])
server, err := getNameForArg(ctx, serverArg, "pg_foreign_server", "srvname")
if err != nil {
@@ -1588,7 +1596,7 @@ SELECT description
"has_table_privilege": makePGPrivilegeInquiryDef(
"table",
argTypeOpts{{"table", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
tableArg := tree.UnwrapDatum(ctx, args[0])
tn, err := getTableNameForArg(ctx, tableArg)
if err != nil {
@@ -1663,7 +1671,7 @@ SELECT description
"has_tablespace_privilege": makePGPrivilegeInquiryDef(
"tablespace",
argTypeOpts{{"tablespace", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
tablespaceArg := tree.UnwrapDatum(ctx, args[0])
tablespace, err := getNameForArg(ctx, tablespaceArg, "pg_tablespace", "spcname")
if err != nil {
@@ -1692,7 +1700,7 @@ SELECT description
"has_type_privilege": makePGPrivilegeInquiryDef(
"type",
argTypeOpts{{"type", strOrOidTypes}},
- func(ctx *tree.EvalContext, args tree.Datums, user string) (tree.Datum, error) {
+ func(ctx *tree.EvalContext, args tree.Datums, user security.SQLUsername) (tree.Datum, error) {
oidArg := tree.UnwrapDatum(ctx, args[0])
// When specifying a type by a text string rather than by OID, the
// allowed input is the same as for the regtype data type.
diff --git a/pkg/sql/sem/tree/alter_database.go b/pkg/sql/sem/tree/alter_database.go
index 104cc1b10dee..c7e4747a2490 100644
--- a/pkg/sql/sem/tree/alter_database.go
+++ b/pkg/sql/sem/tree/alter_database.go
@@ -10,10 +10,14 @@
package tree
+import "github.com/cockroachdb/cockroach/pkg/security"
+
// AlterDatabaseOwner represents a ALTER DATABASE OWNER TO statement.
type AlterDatabaseOwner struct {
- Name Name
- Owner Name
+ Name Name
+ // TODO(solon): Adjust this, see
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ Owner security.SQLUsername
}
// Format implements the NodeFormatter interface.
@@ -21,7 +25,7 @@ func (node *AlterDatabaseOwner) Format(ctx *FmtCtx) {
ctx.WriteString("ALTER DATABASE ")
ctx.FormatNode(&node.Name)
ctx.WriteString(" OWNER TO ")
- ctx.FormatNode(&node.Owner)
+ ctx.FormatUsername(node.Owner)
}
// AlterDatabaseAddRegion represents a ALTER DATABASE ADD REGION(S) statement.
diff --git a/pkg/sql/sem/tree/alter_schema.go b/pkg/sql/sem/tree/alter_schema.go
index cb3a7bd5be37..4687eafd5dbf 100644
--- a/pkg/sql/sem/tree/alter_schema.go
+++ b/pkg/sql/sem/tree/alter_schema.go
@@ -10,6 +10,8 @@
package tree
+import "github.com/cockroachdb/cockroach/pkg/security"
+
// AlterSchema represents an ALTER SCHEMA statement.
type AlterSchema struct {
Schema Name
@@ -48,11 +50,13 @@ func (*AlterSchemaOwner) alterSchemaCmd() {}
// AlterSchemaOwner represents an ALTER SCHEMA OWNER TO command.
type AlterSchemaOwner struct {
- Owner Name
+ // TODO(solon): Adjust this, see
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ Owner security.SQLUsername
}
// Format implements the NodeFormatter interface.
func (node *AlterSchemaOwner) Format(ctx *FmtCtx) {
ctx.WriteString(" OWNER TO ")
- ctx.FormatNode(&node.Owner)
+ ctx.FormatUsername(node.Owner)
}
diff --git a/pkg/sql/sem/tree/alter_table.go b/pkg/sql/sem/tree/alter_table.go
index 3124775bd1d4..d997210d3600 100644
--- a/pkg/sql/sem/tree/alter_table.go
+++ b/pkg/sql/sem/tree/alter_table.go
@@ -13,6 +13,7 @@ package tree
import (
"strings"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
)
@@ -581,7 +582,9 @@ func (node *AlterTableSetSchema) Format(ctx *FmtCtx) {
// AlterTableOwner represents an ALTER TABLE OWNER TO command.
type AlterTableOwner struct {
- Owner Name
+ // TODO(solon): Adjust this, see
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ Owner security.SQLUsername
}
// TelemetryCounter implements the AlterTableCmd interface.
@@ -592,5 +595,5 @@ func (node *AlterTableOwner) TelemetryCounter() telemetry.Counter {
// Format implements the NodeFormatter interface.
func (node *AlterTableOwner) Format(ctx *FmtCtx) {
ctx.WriteString(" OWNER TO ")
- ctx.FormatNode(&node.Owner)
+ ctx.FormatUsername(node.Owner)
}
diff --git a/pkg/sql/sem/tree/alter_type.go b/pkg/sql/sem/tree/alter_type.go
index c022843cb0c3..98b2c4789d07 100644
--- a/pkg/sql/sem/tree/alter_type.go
+++ b/pkg/sql/sem/tree/alter_type.go
@@ -11,6 +11,7 @@
package tree
import (
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
)
@@ -138,13 +139,15 @@ func (node *AlterTypeSetSchema) TelemetryCounter() telemetry.Counter {
// AlterTypeOwner represents an ALTER TYPE OWNER TO command.
type AlterTypeOwner struct {
- Owner Name
+ // TODO(solon): Adjust this, see
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ Owner security.SQLUsername
}
// Format implements the NodeFormatter interface.
func (node *AlterTypeOwner) Format(ctx *FmtCtx) {
ctx.WriteString(" OWNER TO ")
- ctx.FormatNode(&node.Owner)
+ ctx.FormatUsername(node.Owner)
}
// TelemetryCounter implements the AlterTypeCmd interface.
diff --git a/pkg/sql/sem/tree/create.go b/pkg/sql/sem/tree/create.go
index 817f45e2ad32..a6e9b3084714 100644
--- a/pkg/sql/sem/tree/create.go
+++ b/pkg/sql/sem/tree/create.go
@@ -24,6 +24,7 @@ import (
"strconv"
"strings"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
@@ -1322,7 +1323,9 @@ func (node *CreateTable) HoistConstraints() {
type CreateSchema struct {
IfNotExists bool
Schema Name
- AuthRole string
+ // TODO(solon): Adjust this, see
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ AuthRole security.SQLUsername
}
// Format implements the NodeFormatter interface.
@@ -1338,9 +1341,9 @@ func (node *CreateSchema) Format(ctx *FmtCtx) {
ctx.FormatNode(&node.Schema)
}
- if node.AuthRole != "" {
+ if !node.AuthRole.Undefined() {
ctx.WriteString(" AUTHORIZATION ")
- ctx.WriteString(node.AuthRole)
+ ctx.FormatUsername(node.AuthRole)
}
}
diff --git a/pkg/sql/sem/tree/datum.go b/pkg/sql/sem/tree/datum.go
index 788851ea4002..ec755804cfcb 100644
--- a/pkg/sql/sem/tree/datum.go
+++ b/pkg/sql/sem/tree/datum.go
@@ -28,6 +28,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/geo"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/types"
@@ -609,7 +610,7 @@ func (d *DBitArray) Format(ctx *FmtCtx) {
if f.HasFlags(fmtPgwireFormat) {
d.BitArray.Format(&ctx.Buffer)
} else {
- withQuotes := !f.HasFlags(FmtFlags(lex.EncBareStrings))
+ withQuotes := !f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if withQuotes {
ctx.WriteString("B'")
}
@@ -1438,7 +1439,7 @@ func (d *DBytes) Format(ctx *FmtCtx) {
writeAsHexString(ctx, d)
ctx.WriteString(`"`)
} else {
- withQuotes := !f.HasFlags(FmtFlags(lex.EncBareStrings))
+ withQuotes := !f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if withQuotes {
if f.HasFlags(fmtFormatByteLiterals) {
ctx.WriteByte('b')
@@ -1538,7 +1539,7 @@ func (*DUuid) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DUuid) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -1700,7 +1701,7 @@ func (*DIPAddr) AmbiguousFormat() bool {
// Format implements the NodeFormatter interface.
func (d *DIPAddr) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -1881,7 +1882,7 @@ func (*DDate) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DDate) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -1997,7 +1998,7 @@ func (*DTime) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DTime) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -2122,7 +2123,7 @@ func (*DTimeTZ) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DTimeTZ) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -2354,7 +2355,7 @@ func (*DTimestamp) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DTimestamp) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -2511,7 +2512,7 @@ func (*DTimestampTZ) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DTimestampTZ) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -2723,7 +2724,7 @@ func (*DInterval) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DInterval) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -2831,7 +2832,7 @@ func (*DGeography) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DGeography) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -2939,7 +2940,7 @@ func (*DGeometry) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DGeometry) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -3048,7 +3049,7 @@ func (*DBox2D) AmbiguousFormat() bool { return true }
// Format implements the NodeFormatter interface.
func (d *DBox2D) Format(ctx *FmtCtx) {
f := ctx.flags
- bareStrings := f.HasFlags(FmtFlags(lex.EncBareStrings))
+ bareStrings := f.HasFlags(FmtFlags(lexbase.EncBareStrings))
if !bareStrings {
ctx.WriteByte('\'')
}
@@ -4298,12 +4299,12 @@ func (d *DOid) Format(ctx *FmtCtx) {
ctx.WriteByte('(')
d.DInt.Format(ctx)
ctx.WriteByte(',')
- lex.EncodeSQLStringWithFlags(&ctx.Buffer, d.name, lex.EncNoFlags)
+ lex.EncodeSQLStringWithFlags(&ctx.Buffer, d.name, lexbase.EncNoFlags)
ctx.WriteByte(')')
} else {
// This is used to print the name of pseudo-procedures in e.g.
// pg_catalog.pg_type.typinput
- lex.EncodeSQLStringWithFlags(&ctx.Buffer, d.name, lex.EncBareStrings)
+ lex.EncodeSQLStringWithFlags(&ctx.Buffer, d.name, lexbase.EncBareStrings)
}
}
diff --git a/pkg/sql/sem/tree/drop_owned_by.go b/pkg/sql/sem/tree/drop_owned_by.go
index 2ca9766dc986..8a5e99ed1d77 100644
--- a/pkg/sql/sem/tree/drop_owned_by.go
+++ b/pkg/sql/sem/tree/drop_owned_by.go
@@ -10,9 +10,11 @@
package tree
+import "github.com/cockroachdb/cockroach/pkg/security"
+
// DropOwnedBy represents a DROP OWNED BY command.
type DropOwnedBy struct {
- Roles []string
+ Roles []security.SQLUsername
DropBehavior DropBehavior
}
@@ -25,7 +27,7 @@ func (node *DropOwnedBy) Format(ctx *FmtCtx) {
if i > 0 {
ctx.WriteString(", ")
}
- ctx.FormatNameP(&node.Roles[i])
+ ctx.FormatUsername(node.Roles[i])
}
if node.DropBehavior != DropDefault {
ctx.WriteString(" ")
diff --git a/pkg/sql/sem/tree/format.go b/pkg/sql/sem/tree/format.go
index 6921195c0b0c..66e3023d8a59 100644
--- a/pkg/sql/sem/tree/format.go
+++ b/pkg/sql/sem/tree/format.go
@@ -15,7 +15,8 @@ import (
"fmt"
"sync"
- "github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/security"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/errors"
@@ -35,8 +36,8 @@ func (f FmtFlags) HasAnyFlags(subset FmtFlags) bool {
}
// EncodeFlags returns the subset of the flags that are also lex encode flags.
-func (f FmtFlags) EncodeFlags() lex.EncodeFlags {
- return lex.EncodeFlags(f) & (lex.EncFirstFreeFlagBit - 1)
+func (f FmtFlags) EncodeFlags() lexbase.EncodeFlags {
+ return lexbase.EncodeFlags(f) & (lexbase.EncFirstFreeFlagBit - 1)
}
// Basic bit definitions for the FmtFlags bitmask.
@@ -52,15 +53,15 @@ const (
// string will be escaped and enclosed in e'...' regardless of
// whether FmtBareStrings is specified. See FmtRawStrings below for
// an alternative.
- FmtBareStrings FmtFlags = FmtFlags(lex.EncBareStrings)
+ FmtBareStrings FmtFlags = FmtFlags(lexbase.EncBareStrings)
// FmtBareIdentifiers instructs the pretty-printer to print
// identifiers without wrapping quotes in any case.
- FmtBareIdentifiers FmtFlags = FmtFlags(lex.EncBareIdentifiers)
+ FmtBareIdentifiers FmtFlags = FmtFlags(lexbase.EncBareIdentifiers)
// FmtShowPasswords instructs the pretty-printer to not suppress passwords.
// If not set, passwords are replaced by *****.
- FmtShowPasswords FmtFlags = FmtFlags(lex.EncFirstFreeFlagBit) << iota
+ FmtShowPasswords FmtFlags = FmtFlags(lexbase.EncFirstFreeFlagBit) << iota
// FmtShowTypes instructs the pretty-printer to
// annotate expressions with their resolved types.
@@ -143,7 +144,7 @@ const (
// FmtPgwireText instructs the pretty-printer to use
// a pg-compatible conversion to strings. See comments
// in pgwire_encode.go.
- FmtPgwireText FmtFlags = fmtPgwireFormat | FmtFlags(lex.EncBareStrings)
+ FmtPgwireText FmtFlags = fmtPgwireFormat | FmtFlags(lexbase.EncBareStrings)
// FmtParsable instructs the pretty-printer to produce a representation that
// can be parsed into an equivalent expression. If there is a chance that the
@@ -335,6 +336,11 @@ func (ctx *FmtCtx) FormatNameP(s *string) {
ctx.FormatNode((*Name)(s))
}
+// FormatUsername formats a username safely.
+func (ctx *FmtCtx) FormatUsername(s security.SQLUsername) {
+ ctx.FormatName(s.Normalized())
+}
+
// FormatNode recurses into a node for pretty-printing.
// Flag-driven special cases can hook into this.
func (ctx *FmtCtx) FormatNode(n NodeFormatter) {
diff --git a/pkg/sql/sem/tree/name_part.go b/pkg/sql/sem/tree/name_part.go
index b7b4ca8eb5ab..c76a7c48eb72 100644
--- a/pkg/sql/sem/tree/name_part.go
+++ b/pkg/sql/sem/tree/name_part.go
@@ -11,7 +11,7 @@
package tree
import (
- "github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
)
@@ -34,7 +34,7 @@ func (n *Name) Format(ctx *FmtCtx) {
if f.HasFlags(FmtAnonymize) && !isArityIndicatorString(string(*n)) {
ctx.WriteByte('_')
} else {
- lex.EncodeRestrictedSQLIdent(&ctx.Buffer, string(*n), f.EncodeFlags())
+ lexbase.EncodeRestrictedSQLIdent(&ctx.Buffer, string(*n), f.EncodeFlags())
}
}
@@ -66,7 +66,7 @@ func ErrNameString(s string) string {
// Normalize normalizes to lowercase and Unicode Normalization Form C
// (NFC).
func (n Name) Normalize() string {
- return lex.NormalizeName(string(n))
+ return lexbase.NormalizeName(string(n))
}
// An UnrestrictedName is a Name that does not need to be escaped when it
@@ -89,7 +89,7 @@ func (u *UnrestrictedName) Format(ctx *FmtCtx) {
if f.HasFlags(FmtAnonymize) {
ctx.WriteByte('_')
} else {
- lex.EncodeUnrestrictedSQLIdent(&ctx.Buffer, string(*u), f.EncodeFlags())
+ lexbase.EncodeUnrestrictedSQLIdent(&ctx.Buffer, string(*u), f.EncodeFlags())
}
}
diff --git a/pkg/sql/sem/tree/reassign_owned_by.go b/pkg/sql/sem/tree/reassign_owned_by.go
index ff1e218545b9..2626e04fc0a5 100644
--- a/pkg/sql/sem/tree/reassign_owned_by.go
+++ b/pkg/sql/sem/tree/reassign_owned_by.go
@@ -10,10 +10,14 @@
package tree
+import "github.com/cockroachdb/cockroach/pkg/security"
+
// ReassignOwnedBy represents a REASSIGN OWNED BY TO statement.
type ReassignOwnedBy struct {
- OldRoles []string
- NewRole string
+ // TODO(solon): Adjust this, see
+ // https://github.com/cockroachdb/cockroach/issues/54696
+ OldRoles []security.SQLUsername
+ NewRole security.SQLUsername
}
var _ Statement = &ReassignOwnedBy{}
@@ -25,8 +29,8 @@ func (node *ReassignOwnedBy) Format(ctx *FmtCtx) {
if i > 0 {
ctx.WriteString(", ")
}
- ctx.FormatNameP(&node.OldRoles[i])
+ ctx.FormatUsername(node.OldRoles[i])
}
ctx.WriteString(" TO ")
- ctx.FormatNameP(&node.NewRole)
+ ctx.FormatUsername(node.NewRole)
}
diff --git a/pkg/sql/sem/tree/type_check.go b/pkg/sql/sem/tree/type_check.go
index e81f93bb2354..5be87aeeae1c 100644
--- a/pkg/sql/sem/tree/type_check.go
+++ b/pkg/sql/sem/tree/type_check.go
@@ -16,7 +16,7 @@ import (
"strings"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
- "github.com/cockroachdb/cockroach/pkg/sql/lex"
+ "github.com/cockroachdb/cockroach/pkg/sql/lexbase"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
@@ -1419,7 +1419,7 @@ func (expr *Tuple) TypeCheck(
if len(expr.Labels) > 0 {
labels = make([]string, len(expr.Labels))
for i := range expr.Labels {
- labels[i] = lex.NormalizeName(expr.Labels[i])
+ labels[i] = lexbase.NormalizeName(expr.Labels[i])
}
}
expr.typ = types.MakeLabeledTuple(contents, labels)
diff --git a/pkg/sql/sessiondata/internal.go b/pkg/sql/sessiondata/internal.go
index a4fa7d920c8a..086d7e892e6f 100644
--- a/pkg/sql/sessiondata/internal.go
+++ b/pkg/sql/sessiondata/internal.go
@@ -16,7 +16,7 @@ import "github.com/cockroachdb/cockroach/pkg/security"
// to allow control over some of the session data.
type InternalExecutorOverride struct {
// User represents the user that the query will run under.
- User string
+ User security.SQLUsername
// Database represents the default database for the query.
Database string
// ApplicationName represents the application that the query runs under.
@@ -34,4 +34,5 @@ var NoSessionDataOverride = InternalExecutorOverride{}
// NodeUserSessionDataOverride is an InternalExecutorOverride which overrides
// the users to the NodeUser.
-var NodeUserSessionDataOverride = InternalExecutorOverride{User: security.NodeUser}
+var NodeUserSessionDataOverride = InternalExecutorOverride{
+ User: security.MakeSQLUsernameFromPreNormalizedString(security.NodeUser)}
diff --git a/pkg/sql/sessiondata/search_path.go b/pkg/sql/sessiondata/search_path.go
index e4de19f1961d..79ed3cfd3b30 100644
--- a/pkg/sql/sessiondata/search_path.go
+++ b/pkg/sql/sessiondata/search_path.go
@@ -13,6 +13,7 @@ package sessiondata
import (
"strings"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
)
@@ -64,8 +65,8 @@ var EmptySearchPath = SearchPath{}
// DefaultSearchPathForUser returns the default search path with the user
// specific schema name set so that it can be expanded during resolution.
-func DefaultSearchPathForUser(username string) SearchPath {
- return DefaultSearchPath.WithUserSchemaName(username)
+func DefaultSearchPathForUser(username security.SQLUsername) SearchPath {
+ return DefaultSearchPath.WithUserSchemaName(username.Normalized())
}
// MakeSearchPath returns a new immutable SearchPath struct. The paths slice
diff --git a/pkg/sql/sessiondata/session_data.go b/pkg/sql/sessiondata/session_data.go
index 7bcae321daf6..7ebc6401f3a9 100644
--- a/pkg/sql/sessiondata/session_data.go
+++ b/pkg/sql/sessiondata/session_data.go
@@ -92,7 +92,7 @@ func UnmarshalNonLocal(proto sessiondatapb.SessionData) (*SessionData, error) {
proto.SearchPath,
).WithTemporarySchemaName(
proto.TemporarySchemaName,
- ).WithUserSchemaName(proto.User),
+ ).WithUserSchemaName(proto.UserProto.Decode().Normalized()),
SequenceState: seqState,
Location: location,
}, nil
diff --git a/pkg/sql/sessiondatapb/session_data.go b/pkg/sql/sessiondatapb/session_data.go
index b039d59efea1..122f2e6dbff4 100644
--- a/pkg/sql/sessiondatapb/session_data.go
+++ b/pkg/sql/sessiondatapb/session_data.go
@@ -13,6 +13,8 @@ package sessiondatapb
import (
"fmt"
"strings"
+
+ "github.com/cockroachdb/cockroach/pkg/security"
)
// GetFloatPrec computes a precision suitable for a call to
@@ -109,3 +111,8 @@ func VectorizeExecModeFromString(val string) (VectorizeExecMode, bool) {
}
return m, true
}
+
+// User retrieves the session user.
+func (s *SessionData) User() security.SQLUsername {
+ return s.UserProto.Decode()
+}
diff --git a/pkg/sql/sessiondatapb/session_data.pb.go b/pkg/sql/sessiondatapb/session_data.pb.go
index 18096ff59fd4..bb29a848ac3e 100644
--- a/pkg/sql/sessiondatapb/session_data.pb.go
+++ b/pkg/sql/sessiondatapb/session_data.pb.go
@@ -7,6 +7,8 @@ import proto "github.com/gogo/protobuf/proto"
import fmt "fmt"
import math "math"
+import github_com_cockroachdb_cockroach_pkg_security "github.com/cockroachdb/cockroach/pkg/security"
+
import io "io"
// Reference imports to suppress errors if they are not otherwise used.
@@ -45,7 +47,7 @@ var BytesEncodeFormat_value = map[string]int32{
}
func (BytesEncodeFormat) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_session_data_c24dfdb3837a3a66, []int{0}
+ return fileDescriptor_session_data_4d2720a764c15c66, []int{0}
}
// VectorizeExecMode controls if an when the Executor executes queries using
@@ -84,7 +86,7 @@ var VectorizeExecMode_value = map[string]int32{
}
func (VectorizeExecMode) EnumDescriptor() ([]byte, []int) {
- return fileDescriptor_session_data_c24dfdb3837a3a66, []int{1}
+ return fileDescriptor_session_data_4d2720a764c15c66, []int{1}
}
// SessionData contains session parameters that are easily serializable and are
@@ -98,7 +100,7 @@ type SessionData struct {
// session. This can be used for logging and per-application statistics.
ApplicationName string `protobuf:"bytes,2,opt,name=application_name,json=applicationName,proto3" json:"application_name,omitempty"`
// User is the name of the user logged into the session.
- User string `protobuf:"bytes,3,opt,name=user,proto3" json:"user,omitempty"`
+ UserProto github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto `protobuf:"bytes,3,opt,name=user_proto,json=userProto,proto3,casttype=github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto" json:"user_proto,omitempty"`
// DataConversion gives access to the data conversion configuration.
DataConversionConfig DataConversionConfig `protobuf:"bytes,4,opt,name=data_conversion_config,json=dataConversionConfig,proto3" json:"data_conversion_config"`
// VectorizeMode indicates which kinds of queries to use vectorized execution
@@ -135,7 +137,7 @@ func (m *SessionData) Reset() { *m = SessionData{} }
func (m *SessionData) String() string { return proto.CompactTextString(m) }
func (*SessionData) ProtoMessage() {}
func (*SessionData) Descriptor() ([]byte, []int) {
- return fileDescriptor_session_data_c24dfdb3837a3a66, []int{0}
+ return fileDescriptor_session_data_4d2720a764c15c66, []int{0}
}
func (m *SessionData) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -176,7 +178,7 @@ func (m *DataConversionConfig) Reset() { *m = DataConversionConfig{} }
func (m *DataConversionConfig) String() string { return proto.CompactTextString(m) }
func (*DataConversionConfig) ProtoMessage() {}
func (*DataConversionConfig) Descriptor() ([]byte, []int) {
- return fileDescriptor_session_data_c24dfdb3837a3a66, []int{1}
+ return fileDescriptor_session_data_4d2720a764c15c66, []int{1}
}
func (m *DataConversionConfig) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -213,7 +215,7 @@ func (m *SequenceState) Reset() { *m = SequenceState{} }
func (m *SequenceState) String() string { return proto.CompactTextString(m) }
func (*SequenceState) ProtoMessage() {}
func (*SequenceState) Descriptor() ([]byte, []int) {
- return fileDescriptor_session_data_c24dfdb3837a3a66, []int{2}
+ return fileDescriptor_session_data_4d2720a764c15c66, []int{2}
}
func (m *SequenceState) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -248,7 +250,7 @@ func (m *SequenceState_Seq) Reset() { *m = SequenceState_Seq{} }
func (m *SequenceState_Seq) String() string { return proto.CompactTextString(m) }
func (*SequenceState_Seq) ProtoMessage() {}
func (*SequenceState_Seq) Descriptor() ([]byte, []int) {
- return fileDescriptor_session_data_c24dfdb3837a3a66, []int{2, 0}
+ return fileDescriptor_session_data_4d2720a764c15c66, []int{2, 0}
}
func (m *SequenceState_Seq) XXX_Unmarshal(b []byte) error {
return m.Unmarshal(b)
@@ -308,11 +310,11 @@ func (m *SessionData) MarshalTo(dAtA []byte) (int, error) {
i = encodeVarintSessionData(dAtA, i, uint64(len(m.ApplicationName)))
i += copy(dAtA[i:], m.ApplicationName)
}
- if len(m.User) > 0 {
+ if len(m.UserProto) > 0 {
dAtA[i] = 0x1a
i++
- i = encodeVarintSessionData(dAtA, i, uint64(len(m.User)))
- i += copy(dAtA[i:], m.User)
+ i = encodeVarintSessionData(dAtA, i, uint64(len(m.UserProto)))
+ i += copy(dAtA[i:], m.UserProto)
}
dAtA[i] = 0x22
i++
@@ -494,7 +496,7 @@ func (m *SessionData) Size() (n int) {
if l > 0 {
n += 1 + l + sovSessionData(uint64(l))
}
- l = len(m.User)
+ l = len(m.UserProto)
if l > 0 {
n += 1 + l + sovSessionData(uint64(l))
}
@@ -678,7 +680,7 @@ func (m *SessionData) Unmarshal(dAtA []byte) error {
iNdEx = postIndex
case 3:
if wireType != 2 {
- return fmt.Errorf("proto: wrong wireType = %d for field User", wireType)
+ return fmt.Errorf("proto: wrong wireType = %d for field UserProto", wireType)
}
var stringLen uint64
for shift := uint(0); ; shift += 7 {
@@ -703,7 +705,7 @@ func (m *SessionData) Unmarshal(dAtA []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
- m.User = string(dAtA[iNdEx:postIndex])
+ m.UserProto = github_com_cockroachdb_cockroach_pkg_security.SQLUsernameProto(dAtA[iNdEx:postIndex])
iNdEx = postIndex
case 4:
if wireType != 2 {
@@ -1313,56 +1315,59 @@ var (
)
func init() {
- proto.RegisterFile("sql/sessiondatapb/session_data.proto", fileDescriptor_session_data_c24dfdb3837a3a66)
+ proto.RegisterFile("sql/sessiondatapb/session_data.proto", fileDescriptor_session_data_4d2720a764c15c66)
}
-var fileDescriptor_session_data_c24dfdb3837a3a66 = []byte{
- // 738 bytes of a gzipped FileDescriptorProto
- 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x54, 0xcd, 0x6e, 0xea, 0x46,
- 0x14, 0xb6, 0x03, 0xa4, 0x30, 0x14, 0xae, 0x33, 0x97, 0x5b, 0x59, 0xdc, 0xd6, 0x58, 0x51, 0x17,
- 0x6e, 0x54, 0x91, 0x7b, 0x69, 0xd5, 0x7d, 0x48, 0x88, 0xca, 0x22, 0x6d, 0x64, 0xd4, 0x2c, 0xba,
- 0xe8, 0x68, 0x18, 0x1f, 0xc0, 0xad, 0xed, 0xb1, 0x3d, 0x43, 0xfe, 0x9e, 0xa0, 0xcb, 0xbe, 0x43,
- 0xb3, 0xe8, 0xa3, 0x64, 0x53, 0x29, 0xcb, 0xac, 0xd2, 0x96, 0xbc, 0x48, 0x35, 0x43, 0x02, 0x21,
- 0x89, 0x92, 0xdd, 0x9c, 0xef, 0xe7, 0xcc, 0xe1, 0x7c, 0x83, 0xd1, 0x97, 0x22, 0x8b, 0xb6, 0x05,
- 0x08, 0x11, 0xf2, 0x24, 0xa0, 0x92, 0xa6, 0xc3, 0xfb, 0x8a, 0xa8, 0xb2, 0x9d, 0xe6, 0x5c, 0x72,
- 0xfc, 0x9e, 0x71, 0xf6, 0x5b, 0xce, 0x29, 0x9b, 0xb4, 0x45, 0x16, 0xb5, 0x57, 0xf4, 0xcd, 0xc6,
- 0x98, 0x8f, 0xb9, 0xd6, 0x6d, 0xab, 0xd3, 0xdc, 0xb2, 0xf9, 0x4f, 0x11, 0x55, 0x07, 0x73, 0xdd,
- 0x1e, 0x95, 0x14, 0x37, 0x51, 0x59, 0xe9, 0x87, 0x54, 0x80, 0x6d, 0xba, 0xa6, 0x57, 0xf1, 0x17,
- 0x35, 0xfe, 0x0a, 0x59, 0x34, 0x4d, 0xa3, 0x90, 0x51, 0xa9, 0x2e, 0x4e, 0x68, 0x0c, 0xf6, 0x9a,
- 0xd6, 0xbc, 0x79, 0x80, 0xff, 0x40, 0x63, 0xc0, 0x18, 0x15, 0xa7, 0x02, 0x72, 0xbb, 0xa0, 0x69,
- 0x7d, 0xc6, 0x31, 0xfa, 0x4c, 0xb5, 0x22, 0x8c, 0x27, 0xc7, 0x90, 0xeb, 0xd9, 0x19, 0x4f, 0x46,
- 0xe1, 0xd8, 0x2e, 0xba, 0xa6, 0x57, 0xed, 0x7c, 0x6c, 0xbf, 0x30, 0x7e, 0x5b, 0x4d, 0xb7, 0xbb,
- 0x70, 0xee, 0x6a, 0x63, 0xb7, 0x78, 0x79, 0xd3, 0x32, 0xfc, 0x46, 0xf0, 0x0c, 0x87, 0x7f, 0x42,
- 0xf5, 0x63, 0x60, 0x92, 0xe7, 0xe1, 0x39, 0x90, 0x98, 0x07, 0x60, 0x97, 0x5c, 0xd3, 0xab, 0x77,
- 0xda, 0x2f, 0x5e, 0x73, 0x74, 0x6f, 0xe9, 0x9d, 0x02, 0x3b, 0xe0, 0x01, 0xf8, 0xb5, 0x45, 0x17,
- 0x55, 0xe2, 0x1e, 0x6a, 0x49, 0x10, 0x32, 0x4c, 0xc6, 0x64, 0xd9, 0x3e, 0x4c, 0x7e, 0x05, 0x26,
- 0x49, 0x4a, 0x93, 0x90, 0x09, 0x7b, 0xdd, 0x35, 0xbd, 0xb2, 0xff, 0xf9, 0x9d, 0x6c, 0xd1, 0xb1,
- 0xaf, 0x45, 0x87, 0x5a, 0x83, 0x3d, 0x64, 0x05, 0x30, 0xa2, 0xd3, 0x48, 0x92, 0x30, 0x91, 0x44,
- 0x84, 0xe7, 0x60, 0x7f, 0xe2, 0x9a, 0x5e, 0xc9, 0xaf, 0xdf, 0xe1, 0xfd, 0x44, 0x0e, 0xc2, 0x73,
- 0x50, 0x89, 0x44, 0x7c, 0xbe, 0x5a, 0xbb, 0x3c, 0x4f, 0xe4, 0xbe, 0xc6, 0x2d, 0x54, 0x15, 0x40,
- 0x73, 0x36, 0x21, 0x29, 0x95, 0x13, 0xbb, 0xe2, 0x16, 0xbc, 0x8a, 0x8f, 0xe6, 0xd0, 0x21, 0x95,
- 0x13, 0xdc, 0x41, 0xef, 0x24, 0xc4, 0x29, 0xcf, 0x69, 0x7e, 0x46, 0x04, 0x9b, 0x40, 0x4c, 0xe7,
- 0xb9, 0x21, 0xdd, 0xe9, 0xed, 0x82, 0x1c, 0x68, 0x4e, 0x67, 0x77, 0x80, 0x2a, 0x02, 0x32, 0x22,
- 0x24, 0x95, 0x60, 0x57, 0x75, 0x34, 0x5b, 0x2f, 0xee, 0x6c, 0x00, 0xd9, 0x14, 0x12, 0x06, 0x03,
- 0xe5, 0xb8, 0xcb, 0xa4, 0x2c, 0x20, 0xd3, 0xf5, 0xe6, 0x85, 0x89, 0x1a, 0xcf, 0x85, 0x87, 0x7f,
- 0x41, 0x6f, 0x87, 0x67, 0x12, 0x04, 0x81, 0x84, 0xf1, 0x00, 0xc8, 0x88, 0xe7, 0x31, 0x95, 0xfa,
- 0xd5, 0xbd, 0x96, 0x52, 0x57, 0xf9, 0x7a, 0xda, 0xb6, 0xaf, 0x5d, 0xfe, 0xc6, 0xf0, 0x31, 0x84,
- 0xbf, 0x46, 0x18, 0x4e, 0x65, 0x4e, 0xc9, 0x28, 0xe2, 0x54, 0x92, 0x20, 0x1c, 0x87, 0x52, 0xe8,
- 0x07, 0x5b, 0xf2, 0x2d, 0xcd, 0xec, 0x2b, 0x62, 0x4f, 0xe3, 0x9b, 0x7f, 0x9b, 0xa8, 0xb6, 0xf2,
- 0x43, 0x70, 0x17, 0x15, 0x05, 0x64, 0xc2, 0x36, 0xdd, 0x82, 0x57, 0x7d, 0x65, 0xa0, 0x15, 0xa7,
- 0xaa, 0x7c, 0xed, 0xc5, 0x1f, 0x50, 0x23, 0xa2, 0x42, 0x12, 0xb5, 0xd0, 0x30, 0x61, 0x39, 0xc4,
- 0x90, 0x48, 0x08, 0xf4, 0x14, 0x35, 0x1f, 0x2b, 0x6e, 0x00, 0x59, 0x7f, 0xc9, 0x34, 0xf7, 0x51,
- 0x61, 0x00, 0x19, 0x76, 0xd1, 0xba, 0xf6, 0x04, 0x7a, 0x1f, 0xb5, 0x6e, 0x65, 0x76, 0xd3, 0x2a,
- 0x29, 0xe9, 0x9e, 0x5f, 0x12, 0x90, 0xf5, 0x03, 0xfc, 0x05, 0x42, 0x11, 0x55, 0x6f, 0x8c, 0x1c,
- 0xd3, 0x48, 0x37, 0x2c, 0xf8, 0x95, 0x39, 0x72, 0x44, 0xa3, 0x2d, 0x82, 0x36, 0x9e, 0x6c, 0x09,
- 0x63, 0x54, 0x7f, 0x00, 0x7e, 0x0f, 0xa7, 0x96, 0x81, 0xdf, 0xad, 0x08, 0x7b, 0x82, 0xd1, 0x14,
- 0x2c, 0xf3, 0x11, 0xdc, 0xa5, 0x02, 0xbe, 0xfb, 0xd6, 0x5a, 0x6b, 0x96, 0x7f, 0xff, 0xd3, 0x31,
- 0xfe, 0xba, 0x70, 0x8c, 0xad, 0x13, 0xb4, 0xf1, 0xe4, 0xcf, 0x82, 0x2d, 0xf4, 0xe9, 0x02, 0xfc,
- 0x71, 0x34, 0xb2, 0x0c, 0xdc, 0x40, 0xd6, 0x02, 0xe9, 0x7c, 0xf8, 0xb8, 0x33, 0x95, 0xdc, 0x32,
- 0xf1, 0x1b, 0x54, 0x5d, 0xea, 0x12, 0x6b, 0x0d, 0xb7, 0xd0, 0xfb, 0x07, 0xdd, 0x52, 0xc8, 0x43,
- 0xb5, 0x0f, 0x1a, 0xed, 0x44, 0x27, 0xf4, 0x4c, 0x58, 0x85, 0xe5, 0xc5, 0xdd, 0xed, 0xcb, 0xff,
- 0x1c, 0xe3, 0x72, 0xe6, 0x98, 0x57, 0x33, 0xc7, 0xbc, 0x9e, 0x39, 0xe6, 0xbf, 0x33, 0xc7, 0xfc,
- 0xe3, 0xd6, 0x31, 0xae, 0x6e, 0x1d, 0xe3, 0xfa, 0xd6, 0x31, 0x7e, 0xae, 0xad, 0x84, 0x33, 0x5c,
- 0xd7, 0x9f, 0xba, 0x6f, 0xfe, 0x0f, 0x00, 0x00, 0xff, 0xff, 0x35, 0xd6, 0x95, 0x08, 0x45, 0x05,
- 0x00, 0x00,
+var fileDescriptor_session_data_4d2720a764c15c66 = []byte{
+ // 794 bytes of a gzipped FileDescriptorProto
+ 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x84, 0x54, 0xc1, 0x6e, 0xdb, 0x46,
+ 0x10, 0x25, 0x2d, 0xcb, 0x95, 0x46, 0x95, 0x43, 0x6f, 0x94, 0x82, 0x70, 0x5a, 0x4a, 0x30, 0x7a,
+ 0x50, 0x8d, 0x42, 0x4a, 0xdc, 0xa2, 0xc7, 0x02, 0x51, 0x6c, 0xa3, 0x06, 0x9a, 0xd6, 0xa5, 0x90,
+ 0x1c, 0x7a, 0xe8, 0x62, 0xb5, 0x1c, 0x49, 0xdb, 0x90, 0x5c, 0x92, 0xbb, 0x72, 0x2c, 0x7f, 0x41,
+ 0x8f, 0xfd, 0x87, 0xe6, 0xd0, 0x53, 0xbf, 0xc3, 0x97, 0x02, 0x39, 0xe6, 0x64, 0xb4, 0xf2, 0x5f,
+ 0xf4, 0x54, 0xec, 0xca, 0x96, 0xac, 0x24, 0xb0, 0x6f, 0x3b, 0x6f, 0xde, 0x9b, 0x19, 0xce, 0x1b,
+ 0x09, 0x3e, 0x57, 0x79, 0xdc, 0x55, 0xa8, 0x94, 0x90, 0x69, 0xc4, 0x34, 0xcb, 0x06, 0xd7, 0x11,
+ 0x35, 0x61, 0x27, 0x2b, 0xa4, 0x96, 0xe4, 0x21, 0x97, 0xfc, 0x65, 0x21, 0x19, 0x1f, 0x77, 0x54,
+ 0x1e, 0x77, 0x56, 0xf8, 0xdb, 0x8d, 0x91, 0x1c, 0x49, 0xcb, 0xeb, 0x9a, 0xd7, 0x5c, 0xb2, 0xf3,
+ 0x57, 0x19, 0x6a, 0xfd, 0x39, 0x6f, 0x9f, 0x69, 0x46, 0xb6, 0xa1, 0x62, 0xf8, 0x03, 0xa6, 0xd0,
+ 0x77, 0x5b, 0x6e, 0xbb, 0x1a, 0x2e, 0x62, 0xf2, 0x05, 0x78, 0x2c, 0xcb, 0x62, 0xc1, 0x99, 0x36,
+ 0x8d, 0x53, 0x96, 0xa0, 0xbf, 0x66, 0x39, 0xf7, 0x6e, 0xe0, 0x3f, 0xb0, 0x04, 0x09, 0x03, 0x98,
+ 0x28, 0x2c, 0xa8, 0x6d, 0xe2, 0x97, 0x0c, 0xa9, 0xd7, 0xfb, 0xef, 0xa2, 0xf9, 0xed, 0x48, 0xe8,
+ 0xf1, 0x64, 0xd0, 0xe1, 0x32, 0xe9, 0x2e, 0x86, 0x8d, 0x06, 0xcb, 0x77, 0x37, 0x7b, 0x39, 0xea,
+ 0x2a, 0xe4, 0x93, 0x42, 0xe8, 0x69, 0xa7, 0xff, 0xd3, 0xf7, 0xcf, 0x15, 0x16, 0xa6, 0xd3, 0xb1,
+ 0xa9, 0x14, 0x56, 0x4d, 0x55, 0xfb, 0x24, 0x09, 0x7c, 0x62, 0x26, 0xa3, 0x5c, 0xa6, 0x27, 0x58,
+ 0xd8, 0x55, 0x70, 0x99, 0x0e, 0xc5, 0xc8, 0x5f, 0x6f, 0xb9, 0xed, 0xda, 0xde, 0xe3, 0xce, 0x2d,
+ 0xdb, 0xe8, 0x98, 0x8f, 0x7d, 0xba, 0x50, 0x3e, 0xb5, 0xc2, 0xde, 0xfa, 0xf9, 0x45, 0xd3, 0x09,
+ 0x1b, 0xd1, 0x07, 0x72, 0xe4, 0x39, 0x6c, 0x9e, 0x20, 0xd7, 0xb2, 0x10, 0x67, 0x48, 0x13, 0x19,
+ 0xa1, 0x5f, 0x6e, 0xb9, 0xed, 0xcd, 0xbd, 0xce, 0xad, 0x6d, 0x5e, 0x5c, 0x4b, 0x0e, 0x4e, 0x91,
+ 0x3f, 0x93, 0x11, 0x86, 0xf5, 0x45, 0x15, 0x13, 0x92, 0x03, 0x68, 0x6a, 0x54, 0x5a, 0xa4, 0x23,
+ 0xba, 0x2c, 0x2f, 0xd2, 0x5f, 0x91, 0x6b, 0x9a, 0xb1, 0x54, 0x70, 0xe5, 0x6f, 0xb4, 0xdc, 0x76,
+ 0x25, 0xfc, 0xf4, 0x8a, 0xb6, 0xa8, 0x78, 0x64, 0x49, 0xc7, 0x96, 0x43, 0xda, 0xe0, 0x45, 0x38,
+ 0x64, 0x93, 0x58, 0x53, 0x91, 0x6a, 0xaa, 0xc4, 0x19, 0xfa, 0x1f, 0xb5, 0xdc, 0x76, 0x39, 0xdc,
+ 0xbc, 0xc2, 0x8f, 0x52, 0xdd, 0x17, 0x67, 0x68, 0x0c, 0x8e, 0xe5, 0xdc, 0x29, 0xbf, 0x32, 0x37,
+ 0xf8, 0x3a, 0x26, 0x4d, 0xa8, 0x29, 0x64, 0x05, 0x1f, 0xd3, 0x8c, 0xe9, 0xb1, 0x5f, 0x6d, 0x95,
+ 0xda, 0xd5, 0x10, 0xe6, 0xd0, 0x31, 0xd3, 0x63, 0xb2, 0x07, 0x0f, 0x34, 0x26, 0x99, 0x2c, 0x58,
+ 0x31, 0xa5, 0x8a, 0x8f, 0x31, 0x61, 0xf3, 0x33, 0x00, 0x5b, 0xe9, 0xfe, 0x22, 0xd9, 0xb7, 0x39,
+ 0x7b, 0x0a, 0xcf, 0xa0, 0xaa, 0x30, 0xa7, 0x4a, 0x33, 0x8d, 0x7e, 0xcd, 0x5a, 0xb3, 0x7b, 0xeb,
+ 0xce, 0xfa, 0x98, 0x4f, 0x30, 0xe5, 0xd8, 0x37, 0x8a, 0x2b, 0x4f, 0x2a, 0x0a, 0x73, 0x1b, 0xef,
+ 0xbc, 0x76, 0xa1, 0xf1, 0x21, 0xf3, 0xc8, 0x2f, 0x70, 0x7f, 0x30, 0xd5, 0xa8, 0x28, 0xa6, 0x5c,
+ 0x46, 0x48, 0x87, 0xb2, 0x48, 0x98, 0xb6, 0x47, 0x7c, 0x97, 0x4b, 0x3d, 0xa3, 0x3b, 0xb0, 0xb2,
+ 0x43, 0xab, 0x0a, 0xb7, 0x06, 0xef, 0x42, 0xe4, 0x4b, 0x20, 0x78, 0xaa, 0x0b, 0x46, 0x87, 0xb1,
+ 0x64, 0x9a, 0x46, 0x62, 0x24, 0xb4, 0xb2, 0xf7, 0x5f, 0x0e, 0x3d, 0x9b, 0x39, 0x34, 0x89, 0x7d,
+ 0x8b, 0xef, 0xfc, 0xed, 0x42, 0x7d, 0xe5, 0x43, 0x48, 0x0f, 0xd6, 0x15, 0xe6, 0xca, 0x77, 0x5b,
+ 0xa5, 0x76, 0xed, 0x8e, 0x81, 0x56, 0x94, 0x26, 0x0a, 0xad, 0x96, 0x3c, 0x82, 0x46, 0xcc, 0x94,
+ 0xa6, 0x66, 0xa1, 0x22, 0xe5, 0x05, 0x26, 0x98, 0x6a, 0x8c, 0xec, 0x14, 0xf5, 0x90, 0x98, 0x5c,
+ 0x1f, 0xf3, 0xa3, 0x65, 0x66, 0xfb, 0x10, 0x4a, 0x7d, 0xcc, 0x49, 0x0b, 0x36, 0xac, 0x26, 0xb2,
+ 0xfb, 0xa8, 0xf7, 0xaa, 0xb3, 0x8b, 0x66, 0xd9, 0x50, 0xf7, 0xc3, 0xb2, 0xc2, 0xfc, 0x28, 0x22,
+ 0x9f, 0x01, 0xc4, 0xcc, 0xdc, 0x18, 0x3d, 0x61, 0xb1, 0x2d, 0x58, 0x0a, 0xab, 0x73, 0xe4, 0x05,
+ 0x8b, 0x77, 0x29, 0x6c, 0xbd, 0xb7, 0x25, 0x42, 0x60, 0xf3, 0x06, 0xf8, 0x1d, 0x9e, 0x7a, 0x0e,
+ 0x79, 0xb0, 0x42, 0x3c, 0x50, 0x9c, 0x65, 0xe8, 0xb9, 0xef, 0xc0, 0x3d, 0xa6, 0xf0, 0x9b, 0xaf,
+ 0xbd, 0xb5, 0xed, 0xca, 0x6f, 0x7f, 0x04, 0xce, 0x9f, 0xaf, 0x03, 0x67, 0xf7, 0x15, 0x6c, 0xbd,
+ 0xf7, 0x63, 0x21, 0x1e, 0x7c, 0xbc, 0x00, 0x7f, 0x1c, 0x0e, 0x3d, 0x87, 0x34, 0xc0, 0x5b, 0x20,
+ 0x7b, 0x8f, 0x1e, 0x3f, 0x99, 0x68, 0xe9, 0xb9, 0xe4, 0x1e, 0xd4, 0x96, 0xbc, 0xd4, 0x5b, 0x23,
+ 0x4d, 0x78, 0x78, 0xa3, 0x5a, 0x86, 0x85, 0x30, 0xfb, 0x60, 0xf1, 0x93, 0xf8, 0x15, 0x9b, 0x2a,
+ 0xaf, 0xb4, 0x6c, 0xdc, 0xeb, 0x9e, 0xff, 0x1b, 0x38, 0xe7, 0xb3, 0xc0, 0x7d, 0x33, 0x0b, 0xdc,
+ 0xb7, 0xb3, 0xc0, 0xfd, 0x67, 0x16, 0xb8, 0xbf, 0x5f, 0x06, 0xce, 0x9b, 0xcb, 0xc0, 0x79, 0x7b,
+ 0x19, 0x38, 0x3f, 0xd7, 0x57, 0xcc, 0x19, 0x6c, 0xd8, 0x3f, 0xb5, 0xaf, 0xfe, 0x0f, 0x00, 0x00,
+ 0xff, 0xff, 0xc5, 0xba, 0x62, 0xdc, 0x94, 0x05, 0x00, 0x00,
}
diff --git a/pkg/sql/sessiondatapb/session_data.proto b/pkg/sql/sessiondatapb/session_data.proto
index 22d056990eb9..e81ddb6e45c6 100644
--- a/pkg/sql/sessiondatapb/session_data.proto
+++ b/pkg/sql/sessiondatapb/session_data.proto
@@ -25,7 +25,7 @@ message SessionData {
// session. This can be used for logging and per-application statistics.
string application_name = 2;
// User is the name of the user logged into the session.
- string user = 3;
+ string user_proto = 3 [(gogoproto.casttype) = "github.com/cockroachdb/cockroach/pkg/security.SQLUsernameProto"];
// DataConversion gives access to the data conversion configuration.
DataConversionConfig data_conversion_config = 4 [(gogoproto.nullable) = false];
// VectorizeMode indicates which kinds of queries to use vectorized execution
diff --git a/pkg/sql/set_cluster_setting.go b/pkg/sql/set_cluster_setting.go
index 78838aa001b3..dcdb788f712e 100644
--- a/pkg/sql/set_cluster_setting.go
+++ b/pkg/sql/set_cluster_setting.go
@@ -196,7 +196,7 @@ func (n *setClusterSettingNode) startExec(params runParams) error {
expectedEncodedValue = n.setting.EncodedDefault()
if _, err := execCfg.InternalExecutor.ExecEx(
ctx, "reset-setting", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"DELETE FROM system.settings WHERE name = $1", n.name,
); err != nil {
return err
@@ -211,7 +211,7 @@ func (n *setClusterSettingNode) startExec(params runParams) error {
if _, ok := n.setting.(*settings.StateMachineSetting); ok {
datums, err := execCfg.InternalExecutor.QueryRowEx(
ctx, "retrieve-prev-setting", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SELECT value FROM system.settings WHERE name = $1", n.name,
)
if err != nil {
@@ -233,7 +233,7 @@ func (n *setClusterSettingNode) startExec(params runParams) error {
}
if _, err = execCfg.InternalExecutor.ExecEx(
ctx, "update-setting", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
`UPSERT INTO system.settings (name, value, "lastUpdated", "valueType") VALUES ($1, $2, now(), $3)`,
n.name, encoded, n.setting.Typ(),
); err != nil {
@@ -289,7 +289,7 @@ func (n *setClusterSettingNode) startExec(params runParams) error {
EventLogSetClusterSetting,
0, /* no target */
int32(params.extendedEvalCtx.NodeID.SQLInstanceID()),
- EventLogSetClusterSettingDetail{n.name, reportedValue, params.SessionData().User},
+ EventLogSetClusterSettingDetail{n.name, reportedValue, params.SessionData().User()},
)
}); err != nil {
return err
diff --git a/pkg/sql/set_zone_config.go b/pkg/sql/set_zone_config.go
index 42e91625c927..f4557fd4a004 100644
--- a/pkg/sql/set_zone_config.go
+++ b/pkg/sql/set_zone_config.go
@@ -21,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/serverpb"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
@@ -206,7 +207,7 @@ func checkPrivilegeForSetZoneConfig(ctx context.Context, p *planner, zs tree.Zon
return pgerror.Newf(pgcode.InsufficientPrivilege,
"user %s does not have %s or %s privilege on %s %s",
- p.SessionData().User, privilege.ZONECONFIG, privilege.CREATE, dbDesc.TypeName(), dbDesc.GetName())
+ p.SessionData().User(), privilege.ZONECONFIG, privilege.CREATE, dbDesc.TypeName(), dbDesc.GetName())
}
tableDesc, err := p.resolveTableForZone(ctx, &zs)
if err != nil {
@@ -229,7 +230,7 @@ func checkPrivilegeForSetZoneConfig(ctx context.Context, p *planner, zs tree.Zon
return pgerror.Newf(pgcode.InsufficientPrivilege,
"user %s does not have %s or %s privilege on %s %s",
- p.SessionData().User, privilege.ZONECONFIG, privilege.CREATE, tableDesc.TypeName(), tableDesc.GetName())
+ p.SessionData().User(), privilege.ZONECONFIG, privilege.CREATE, tableDesc.TypeName(), tableDesc.GetName())
}
// setZoneConfigRun contains the run-time state of setZoneConfigNode during local execution.
@@ -674,12 +675,12 @@ func (n *setZoneConfigNode) startExec(params runParams) error {
Target string
Config string `json:",omitempty"`
Options string `json:",omitempty"`
- User string
+ User security.SQLUsername
}{
Target: tree.AsStringWithFQNames(&zs, params.Ann()),
Config: strings.TrimSpace(yamlConfig),
Options: optionStr.String(),
- User: params.SessionData().User,
+ User: params.SessionData().User(),
}
if deleteZone {
eventLogType = EventLogRemoveZoneConfig
diff --git a/pkg/sql/show_cluster_setting.go b/pkg/sql/show_cluster_setting.go
index 8b6aec83b2dd..2a1e184c8341 100644
--- a/pkg/sql/show_cluster_setting.go
+++ b/pkg/sql/show_cluster_setting.go
@@ -53,7 +53,7 @@ func (p *planner) showStateMachineSetting(
datums, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryRowEx(
ctx, "read-setting",
txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SELECT value FROM system.settings WHERE name = $1", name,
)
if err != nil {
diff --git a/pkg/sql/show_fingerprints.go b/pkg/sql/show_fingerprints.go
index 96a5e3685725..166844aed263 100644
--- a/pkg/sql/show_fingerprints.go
+++ b/pkg/sql/show_fingerprints.go
@@ -146,7 +146,7 @@ func (n *showFingerprintsNode) Next(params runParams) (bool, error) {
fingerprintCols, err := params.extendedEvalCtx.ExecCfg.InternalExecutor.QueryRowEx(
params.ctx, "hash-fingerprint",
params.p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
sql,
)
if err != nil {
diff --git a/pkg/sql/show_histogram.go b/pkg/sql/show_histogram.go
index da0b9dbf703b..617f25c91876 100644
--- a/pkg/sql/show_histogram.go
+++ b/pkg/sql/show_histogram.go
@@ -48,7 +48,7 @@ func (p *planner) ShowHistogram(ctx context.Context, n *tree.ShowHistogram) (pla
ctx,
"read-histogram",
p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
`SELECT histogram
FROM system.table_statistics
WHERE "statisticID" = $1`,
diff --git a/pkg/sql/sqlliveness/slstorage/slstorage.go b/pkg/sql/sqlliveness/slstorage/slstorage.go
index fbb39c3b5edf..0139a99e58d1 100644
--- a/pkg/sql/sqlliveness/slstorage/slstorage.go
+++ b/pkg/sql/sqlliveness/slstorage/slstorage.go
@@ -113,7 +113,7 @@ func NewTestingStorage(
db: db,
ex: ie,
sd: sessiondata.InternalExecutorOverride{
- User: security.NodeUser,
+ User: security.NodeUserName(),
Database: database,
},
newTimer: newTimer,
diff --git a/pkg/sql/stmtdiagnostics/statement_diagnostics.go b/pkg/sql/stmtdiagnostics/statement_diagnostics.go
index 01ac7daafe34..16371ad58510 100644
--- a/pkg/sql/stmtdiagnostics/statement_diagnostics.go
+++ b/pkg/sql/stmtdiagnostics/statement_diagnostics.go
@@ -217,7 +217,7 @@ func (r *Registry) insertRequestInternal(ctx context.Context, fprint string) (re
// Check if there's already a pending request for this fingerprint.
row, err := r.ie.QueryRowEx(ctx, "stmt-diag-check-pending", txn,
sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
},
"SELECT count(1) FROM system.statement_diagnostics_requests "+
"WHERE completed = false AND statement_fingerprint = $1",
@@ -232,7 +232,7 @@ func (r *Registry) insertRequestInternal(ctx context.Context, fprint string) (re
row, err = r.ie.QueryRowEx(ctx, "stmt-diag-insert-request", txn,
sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
},
"INSERT INTO system.statement_diagnostics_requests (statement_fingerprint, requested_at) "+
"VALUES ($1, $2) RETURNING id",
@@ -385,7 +385,7 @@ func (r *Registry) insertStatementDiagnostics(
err := r.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
if requestID != 0 {
row, err := r.ie.QueryRowEx(ctx, "stmt-diag-check-completed", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"SELECT count(1) FROM system.statement_diagnostics_requests WHERE id = $1 AND completed = false",
requestID)
if err != nil {
@@ -418,7 +418,7 @@ func (r *Registry) insertStatementDiagnostics(
// Insert the chunk into system.statement_bundle_chunks.
row, err := r.ie.QueryRowEx(
ctx, "stmt-bundle-chunks-insert", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"INSERT INTO system.statement_bundle_chunks(description, data) VALUES ($1, $2) RETURNING id",
"statement diagnostics bundle",
tree.NewDBytes(tree.DBytes(chunk)),
@@ -437,7 +437,7 @@ func (r *Registry) insertStatementDiagnostics(
// Insert the trace into system.statement_diagnostics.
row, err := r.ie.QueryRowEx(
ctx, "stmt-diag-insert", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"INSERT INTO system.statement_diagnostics "+
"(statement_fingerprint, statement, collected_at, trace, bundle_chunks, error) "+
"VALUES ($1, $2, $3, $4, $5, $6) RETURNING id",
@@ -451,7 +451,7 @@ func (r *Registry) insertStatementDiagnostics(
if requestID != 0 {
// Mark the request from system.statement_diagnostics_request as completed.
_, err := r.ie.ExecEx(ctx, "stmt-diag-mark-completed", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"UPDATE system.statement_diagnostics_requests "+
"SET completed = true, statement_diagnostics_id = $1 WHERE id = $2",
diagID, requestID)
@@ -463,7 +463,7 @@ func (r *Registry) insertStatementDiagnostics(
// This is necessary because the UI uses this table to discover completed
// diagnostics.
_, err := r.ie.ExecEx(ctx, "stmt-diag-add-completed", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
"INSERT INTO system.statement_diagnostics_requests"+
" (completed, statement_fingerprint, statement_diagnostics_id, requested_at)"+
" VALUES (true, $1, $2, $3)",
@@ -493,7 +493,7 @@ func (r *Registry) pollRequests(ctx context.Context) error {
var err error
rows, err = r.ie.QueryEx(ctx, "stmt-diag-poll", nil, /* txn */
sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
},
"SELECT id, statement_fingerprint FROM system.statement_diagnostics_requests "+
"WHERE completed = false")
diff --git a/pkg/sql/table_test.go b/pkg/sql/table_test.go
index 4d517056cced..74fccc73ce04 100644
--- a/pkg/sql/table_test.go
+++ b/pkg/sql/table_test.go
@@ -182,7 +182,7 @@ func TestMakeTableDescColumns(t *testing.T) {
for i, d := range testData {
s := "CREATE TABLE foo.test (a " + d.sqlType + " PRIMARY KEY, b " + d.sqlType + ")"
schema, err := CreateTestTableDescriptor(context.Background(), 1, 100, s,
- descpb.NewDefaultPrivilegeDescriptor(security.AdminRole))
+ descpb.NewDefaultPrivilegeDescriptor(security.AdminRoleName()))
if err != nil {
t.Fatalf("%d: %v", i, err)
}
@@ -298,7 +298,7 @@ func TestMakeTableDescIndexes(t *testing.T) {
for i, d := range testData {
s := "CREATE TABLE foo.test (" + d.sql + ")"
schema, err := CreateTestTableDescriptor(context.Background(), 1, 100, s,
- descpb.NewDefaultPrivilegeDescriptor(security.AdminRole))
+ descpb.NewDefaultPrivilegeDescriptor(security.AdminRoleName()))
if err != nil {
t.Fatalf("%d (%s): %v", i, d.sql, err)
}
@@ -318,7 +318,7 @@ func TestPrimaryKeyUnspecified(t *testing.T) {
s := "CREATE TABLE foo.test (a INT, b INT, CONSTRAINT c UNIQUE (b))"
ctx := context.Background()
desc, err := CreateTestTableDescriptor(ctx, 1, 100, s,
- descpb.NewDefaultPrivilegeDescriptor(security.AdminRole))
+ descpb.NewDefaultPrivilegeDescriptor(security.AdminRoleName()))
if err != nil {
t.Fatal(err)
}
diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go
index d4cbe26a4202..b31a6a52f029 100644
--- a/pkg/sql/temporary_schema.go
+++ b/pkg/sql/temporary_schema.go
@@ -272,10 +272,10 @@ func cleanupSchemaObjects(
}
}
- searchPath := sessiondata.DefaultSearchPathForUser(security.RootUser).WithTemporarySchemaName(schemaName)
+ searchPath := sessiondata.DefaultSearchPathForUser(security.RootUserName()).WithTemporarySchemaName(schemaName)
override := sessiondata.InternalExecutorOverride{
SearchPath: &searchPath,
- User: security.RootUser,
+ User: security.RootUserName(),
DatabaseIDToTempSchemaID: databaseIDToTempSchemaID,
}
diff --git a/pkg/sql/tests/system_table_test.go b/pkg/sql/tests/system_table_test.go
index 7521bcb09d5d..ee9a36096c2c 100644
--- a/pkg/sql/tests/system_table_test.go
+++ b/pkg/sql/tests/system_table_test.go
@@ -66,7 +66,7 @@ func TestInitialKeys(t *testing.T) {
keys.SystemDatabaseID,
keys.MaxReservedDescID,
"CREATE TABLE system.x (val INTEGER PRIMARY KEY)",
- descpb.NewDefaultPrivilegeDescriptor(security.NodeUser),
+ descpb.NewDefaultPrivilegeDescriptor(security.NodeUserName()),
)
if err != nil {
t.Fatal(err)
diff --git a/pkg/sql/truncate.go b/pkg/sql/truncate.go
index 1dbf19b4e054..3f7d53ff0c7e 100644
--- a/pkg/sql/truncate.go
+++ b/pkg/sql/truncate.go
@@ -160,8 +160,8 @@ func (t *truncateNode) startExec(params runParams) error {
struct {
TableName string
Statement string
- User string
- }{name, n.String(), p.SessionData().User},
+ User security.SQLUsername
+ }{name, n.String(), p.SessionData().User()},
); err != nil {
return err
}
@@ -415,7 +415,7 @@ func (p *planner) reassignIndexComments(
ctx,
"update-table-comments",
p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
`SELECT count(*) FROM system.comments WHERE object_id = $1 AND type = $2`,
table.ID,
keys.IndexCommentType,
@@ -429,7 +429,7 @@ func (p *planner) reassignIndexComments(
ctx,
"update-table-comments",
p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
`UPDATE system.comments SET sub_id=$1 WHERE sub_id=$2 AND object_id=$3 AND type=$4`,
new,
old,
diff --git a/pkg/sql/user.go b/pkg/sql/user.go
index 170eea654e7b..d9bd7adc841a 100644
--- a/pkg/sql/user.go
+++ b/pkg/sql/user.go
@@ -55,7 +55,7 @@ import (
// via the cluster setting.
//
func GetUserHashedPassword(
- ctx context.Context, ie *InternalExecutor, username string,
+ ctx context.Context, ie *InternalExecutor, username security.SQLUsername,
) (
exists bool,
canLogin bool,
@@ -63,7 +63,7 @@ func GetUserHashedPassword(
validUntilFn func(ctx context.Context) (timestamp *tree.DTimestamp, err error),
err error,
) {
- isRoot := username == security.RootUser
+ isRoot := username.IsRootUser()
if isRoot {
// As explained above, for root we report that the user exists
@@ -91,7 +91,7 @@ func GetUserHashedPassword(
}
func retrieveUserAndPassword(
- ctx context.Context, ie *InternalExecutor, isRoot bool, normalizedUsername string,
+ ctx context.Context, ie *InternalExecutor, isRoot bool, normalizedUsername security.SQLUsername,
) (exists bool, canLogin bool, hashedPassword []byte, validUntil *tree.DTimestamp, err error) {
// We may be operating with a timeout.
timeout := userLoginTimeout.Get(&ie.s.cfg.Settings.SV)
@@ -115,7 +115,7 @@ func retrieveUserAndPassword(
`WHERE username=$1`
values, err := ie.QueryRowEx(
ctx, "get-hashed-pwd", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
getHashedPassword, normalizedUsername)
if err != nil {
return errors.Wrapf(err, "error looking up user %s", normalizedUsername)
@@ -136,7 +136,7 @@ func retrieveUserAndPassword(
loginDependencies, err := ie.QueryEx(
ctx, "get-login-dependencies", nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
getLoginDependencies,
normalizedUsername,
)
@@ -188,30 +188,31 @@ var userLoginTimeout = settings.RegisterPublicNonNegativeDurationSetting(
)
// GetAllRoles returns a "set" (map) of Roles -> true.
-func (p *planner) GetAllRoles(ctx context.Context) (map[string]bool, error) {
+func (p *planner) GetAllRoles(ctx context.Context) (map[security.SQLUsername]bool, error) {
query := `SELECT username FROM system.users`
rows, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryEx(
ctx, "read-users", p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
query)
if err != nil {
return nil, err
}
- users := make(map[string]bool)
+ users := make(map[security.SQLUsername]bool)
for _, row := range rows {
username := tree.MustBeDString(row[0])
- users[string(username)] = true
+ // The usernames in system.users are already normalized.
+ users[security.MakeSQLUsernameFromPreNormalizedString(string(username))] = true
}
return users, nil
}
// RoleExists returns true if the role exists.
-func (p *planner) RoleExists(ctx context.Context, role string) (bool, error) {
+func (p *planner) RoleExists(ctx context.Context, role security.SQLUsername) (bool, error) {
query := `SELECT username FROM system.users WHERE username = $1`
row, err := p.ExtendedEvalContext().ExecCfg.InternalExecutor.QueryRowEx(
ctx, "read-users", p.txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
query, role,
)
if err != nil {
diff --git a/pkg/sql/values_test.go b/pkg/sql/values_test.go
index e74c5d4961cf..26a2191fd205 100644
--- a/pkg/sql/values_test.go
+++ b/pkg/sql/values_test.go
@@ -48,7 +48,7 @@ func makeTestPlanner() *planner {
// TODO(andrei): pass the cleanup along to the caller.
p, _ /* cleanup */ := newInternalPlanner(
- "test", nil /* txn */, security.RootUser, &MemoryMetrics{}, &execCfg,
+ "test", nil /* txn */, security.RootUserName(), &MemoryMetrics{}, &execCfg,
)
return p
}
diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go
index 401c3623e53f..7deadc100727 100644
--- a/pkg/sql/vars.go
+++ b/pkg/sql/vars.go
@@ -888,14 +888,14 @@ var varGen = map[string]sessionVar{
// In PG this is a pseudo-function used with SELECT, not SHOW.
// See https://www.postgresql.org/docs/10/static/functions-info.html
`session_user`: {
- Get: func(evalCtx *extendedEvalContext) string { return evalCtx.SessionData.User },
+ Get: func(evalCtx *extendedEvalContext) string { return evalCtx.SessionData.User().Normalized() },
},
// See pg sources src/backend/utils/misc/guc.c. The variable is defined
// but is hidden from SHOW ALL.
`session_authorization`: {
Hidden: true,
- Get: func(evalCtx *extendedEvalContext) string { return evalCtx.SessionData.User },
+ Get: func(evalCtx *extendedEvalContext) string { return evalCtx.SessionData.User().Normalized() },
},
// Supported for PG compatibility only.
diff --git a/pkg/sql/virtual_schema.go b/pkg/sql/virtual_schema.go
index 1adce4da704c..16a14ab72f0b 100644
--- a/pkg/sql/virtual_schema.go
+++ b/pkg/sql/virtual_schema.go
@@ -669,7 +669,7 @@ func NewVirtualSchemaHolder(
// For instance, information_schema will only expose rows to a given user which that
// user has access to.
var publicSelectPrivileges = descpb.NewPrivilegeDescriptor(
- security.PublicRole, privilege.List{privilege.SELECT}, security.NodeUser,
+ security.PublicRoleName(), privilege.List{privilege.SELECT}, security.NodeUserName(),
)
func initVirtualDatabaseDesc(id descpb.ID, name string) *dbdesc.Immutable {
diff --git a/pkg/sqlmigrations/migrations.go b/pkg/sqlmigrations/migrations.go
index 4ca4d8e92970..57e3597e7354 100644
--- a/pkg/sqlmigrations/migrations.go
+++ b/pkg/sqlmigrations/migrations.go
@@ -460,7 +460,7 @@ type runner struct {
func (r runner) execAsRoot(ctx context.Context, opName, stmt string, qargs ...interface{}) error {
_, err := r.sqlExecutor.ExecEx(ctx, opName, nil, /* txn */
sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
},
stmt, qargs...)
return err
@@ -872,7 +872,7 @@ func (m *Manager) migrateSystemNamespace(
rows, err := r.sqlExecutor.QueryEx(
ctx, "read-deprecated-namespace-table", txn,
sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
},
q)
if err != nil {
@@ -1085,7 +1085,7 @@ func markDeprecatedSchemaChangeJobsFailed(ctx context.Context, r runner) error {
// Get jobs in a non-terminal state.
rows, err := r.sqlExecutor.QueryEx(
ctx, "get-deprecated-schema-change-jobs", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
`SELECT id, status, payload FROM system.jobs WHERE status NOT IN ($1, $2, $3) LIMIT $4`,
jobs.StatusSucceeded, jobs.StatusCanceled, jobs.StatusFailed, batchSize,
)
@@ -1125,7 +1125,7 @@ func markDeprecatedSchemaChangeJobsFailed(ctx context.Context, r runner) error {
}
if _, err := r.sqlExecutor.ExecEx(
ctx, "update-deprecated-schema-change-job", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
`UPDATE system.jobs SET status = $1, payload = $2 WHERE id = $3`,
jobs.StatusFailed, newPayloadBytes, id,
); err != nil {
@@ -1260,7 +1260,7 @@ func disallowPublicUserOrRole(ctx context.Context, r runner) error {
row, err := r.sqlExecutor.QueryRowEx(
ctx, "disallowPublicUserOrRole", nil, /* txn */
sessiondata.InternalExecutorOverride{
- User: security.RootUser,
+ User: security.RootUserName(),
},
selectPublicStmt, security.PublicRole,
)
@@ -1353,7 +1353,7 @@ func updateSystemLocationData(ctx context.Context, r runner) error {
// If so, we don't want to do anything.
row, err := r.sqlExecutor.QueryRowEx(ctx, "update-system-locations",
nil, /* txn */
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
`SELECT count(*) FROM system.locations`)
if err != nil {
return err
@@ -1405,7 +1405,7 @@ ON system.jobs (created_by_type, created_by_id)
STORING (status)
`
asNode := sessiondata.InternalExecutorOverride{
- User: security.NodeUser,
+ User: security.NodeUserName(),
}
if _, err := r.sqlExecutor.ExecEx(
@@ -1430,7 +1430,7 @@ ADD COLUMN IF NOT EXISTS claim_session_id BYTES CREATE FAMILY claim,
ADD COLUMN IF NOT EXISTS claim_instance_id INT8 FAMILY claim
`
asNode := sessiondata.InternalExecutorOverride{
- User: security.NodeUser,
+ User: security.NodeUserName(),
}
if _, err := r.sqlExecutor.ExecEx(ctx, "add-jobs-claim-cols", nil, asNode, addColsStmt); err != nil {
return err
@@ -1444,7 +1444,7 @@ func createTenantsTable(ctx context.Context, r runner) error {
func alterSystemScheduledJobsFixTableSchema(ctx context.Context, r runner) error {
setOwner := "UPDATE system.scheduled_jobs SET owner='root' WHERE owner IS NULL"
- asNode := sessiondata.InternalExecutorOverride{User: security.NodeUser}
+ asNode := sessiondata.InternalExecutorOverride{User: security.NodeUserName()}
if _, err := r.sqlExecutor.ExecEx(ctx, "set-schedule-owner", nil, asNode, setOwner); err != nil {
return err
diff --git a/pkg/storage/cloud/external_storage.go b/pkg/storage/cloud/external_storage.go
index 2ead548f817c..e77080ee1acb 100644
--- a/pkg/storage/cloud/external_storage.go
+++ b/pkg/storage/cloud/external_storage.go
@@ -17,6 +17,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
)
@@ -82,7 +83,7 @@ type ExternalStorageFactory func(ctx context.Context, dest roachpb.ExternalStora
// ExternalStorageFromURIFactory describes a factory function for ExternalStorage given a URI.
type ExternalStorageFromURIFactory func(ctx context.Context, uri string,
- user string) (ExternalStorage, error)
+ user security.SQLUsername) (ExternalStorage, error)
// SQLConnI encapsulates the interfaces which will be implemented by the network
// backed SQLConn which is used to interact with the userfile tables.
diff --git a/pkg/storage/cloudimpl/cloudimpltests/external_storage_test.go b/pkg/storage/cloudimpl/cloudimpltests/external_storage_test.go
index ee016d5a2893..bc7ad6ff6ade 100644
--- a/pkg/storage/cloudimpl/cloudimpltests/external_storage_test.go
+++ b/pkg/storage/cloudimpl/cloudimpltests/external_storage_test.go
@@ -30,6 +30,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
+ "github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
"github.com/cockroachdb/cockroach/pkg/storage/cloudimpl"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
@@ -302,7 +303,10 @@ func testListFiles(t *testing.T, storeURI, user string, ie *sql.InternalExecutor
for i := range in {
u := *uri
if u.Scheme == "userfile" && u.Host == "" {
- u.Host = cloudimpl.DefaultQualifiedNamePrefix + user
+ composedTableName := tree.Name(cloudimpl.DefaultQualifiedNamePrefix + user)
+ u.Host = cloudimpl.DefaultQualifiedNamespace +
+ // Escape special identifiers as needed.
+ composedTableName.String()
}
u.Path = u.Path + "/" + in[i]
out[i] = u.String()
diff --git a/pkg/storage/cloudimpl/external_storage.go b/pkg/storage/cloudimpl/external_storage.go
index 929fdbe27120..a0f535aceb1c 100644
--- a/pkg/storage/cloudimpl/external_storage.go
+++ b/pkg/storage/cloudimpl/external_storage.go
@@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/blobs"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
@@ -113,7 +114,9 @@ var ErrListingUnsupported = errors.New("listing is not supported")
var ErrFileDoesNotExist = errors.New("external_storage: file doesn't exist")
// ExternalStorageConfFromURI generates an ExternalStorage config from a URI string.
-func ExternalStorageConfFromURI(path, user string) (roachpb.ExternalStorage, error) {
+func ExternalStorageConfFromURI(
+ path string, user security.SQLUsername,
+) (roachpb.ExternalStorage, error) {
conf := roachpb.ExternalStorage{}
uri, err := url.Parse(path)
if err != nil {
@@ -197,18 +200,22 @@ func ExternalStorageConfFromURI(path, user string) (roachpb.ExternalStorage, err
}
case "userfile":
qualifiedTableName := uri.Host
- if user == "" {
+ if user.Undefined() {
return conf, errors.Errorf("user creating the FileTable ExternalStorage must be specified")
}
// If the import statement does not specify a qualified table name then use
// the default to attempt to locate the file(s).
if qualifiedTableName == "" {
- qualifiedTableName = DefaultQualifiedNamePrefix + user
+ composedTableName := security.MakeSQLUsernameFromPreNormalizedString(
+ DefaultQualifiedNamePrefix + user.Normalized())
+ qualifiedTableName = DefaultQualifiedNamespace +
+ // Escape special identifiers as needed.
+ composedTableName.SQLIdentifier()
}
conf.Provider = roachpb.ExternalStorageProvider_FileTable
- conf.FileTableConfig.User = user
+ conf.FileTableConfig.User = user.Normalized()
conf.FileTableConfig.QualifiedTableName = qualifiedTableName
conf.FileTableConfig.Path = uri.Path
default:
@@ -226,7 +233,7 @@ func ExternalStorageFromURI(
externalConfig base.ExternalIODirConfig,
settings *cluster.Settings,
blobClientFactory blobs.BlobClientFactory,
- user string,
+ user security.SQLUsername,
ie *sql.InternalExecutor,
kvDB *kv.DB,
) (cloud.ExternalStorage, error) {
diff --git a/pkg/storage/cloudimpl/file_table_storage.go b/pkg/storage/cloudimpl/file_table_storage.go
index 9d787ed3daed..d52744572302 100644
--- a/pkg/storage/cloudimpl/file_table_storage.go
+++ b/pkg/storage/cloudimpl/file_table_storage.go
@@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql"
"github.com/cockroachdb/cockroach/pkg/storage/cloud"
@@ -32,9 +33,11 @@ import (
const (
// DefaultUserfileScheme is the default scheme used in a userfile URI.
DefaultUserfileScheme = "userfile"
- // DefaultQualifiedNamePrefix is the default FQN prefix used when referencing
- // tables in userfile.
- DefaultQualifiedNamePrefix = "defaultdb.public.userfiles_"
+ // DefaultQualifiedNamespace is the default FQN namespace prefix
+ // used when referencing tables in userfile.
+ DefaultQualifiedNamespace = "defaultdb.public."
+ // DefaultQualifiedNamePrefix is the defualt FQN table name prefix.
+ DefaultQualifiedNamePrefix = "userfiles_"
)
type fileTableStorage struct {
@@ -81,9 +84,12 @@ func makeFileTableStorage(
trimmedPath, path.Clean(trimmedPath))
}
+ // cfg.User is already a normalized SQL username.
+ username := security.MakeSQLUsernameFromPreNormalizedString(cfg.User)
+
executor := filetable.MakeInternalFileToTableExecutor(ie, db)
- fileToTableSystem, err := filetable.NewFileToTableSystem(ctx, cfg.QualifiedTableName, executor,
- cfg.User)
+ fileToTableSystem, err := filetable.NewFileToTableSystem(ctx,
+ cfg.QualifiedTableName, executor, username)
if err != nil {
return nil, err
}
@@ -106,8 +112,12 @@ func MakeSQLConnFileTableStorage(
ctx context.Context, cfg roachpb.ExternalStorage_FileTable, conn cloud.SQLConnI,
) (cloud.ExternalStorage, error) {
executor := filetable.MakeSQLConnFileToTableExecutor(conn)
- fileToTableSystem, err := filetable.NewFileToTableSystem(ctx, cfg.QualifiedTableName, executor,
- cfg.User)
+
+ // cfg.User is already a normalized username,
+ username := security.MakeSQLUsernameFromPreNormalizedString(cfg.User)
+
+ fileToTableSystem, err := filetable.NewFileToTableSystem(ctx,
+ cfg.QualifiedTableName, executor, username)
if err != nil {
return nil, err
}
diff --git a/pkg/storage/cloudimpl/filetable/file_table_read_writer.go b/pkg/storage/cloudimpl/filetable/file_table_read_writer.go
index 333c4c8d9bda..34f65067ec68 100644
--- a/pkg/storage/cloudimpl/filetable/file_table_read_writer.go
+++ b/pkg/storage/cloudimpl/filetable/file_table_read_writer.go
@@ -46,9 +46,11 @@ type FileToTableExecutorRows struct {
// FileToTableSystemExecutor is the interface which defines the methods for the
// SQL query executor used by the FileToTableSystem
type FileToTableSystemExecutor interface {
- Query(ctx context.Context, opName, query, username string,
+ Query(ctx context.Context, opName, query string,
+ username security.SQLUsername,
qargs ...interface{}) (*FileToTableExecutorRows, error)
- Exec(ctx context.Context, opName, query, username string,
+ Exec(ctx context.Context, opName, query string,
+ username security.SQLUsername,
qargs ...interface{}) error
}
@@ -71,7 +73,7 @@ func MakeInternalFileToTableExecutor(
// Query implements the FileToTableSystemExecutor interface.
func (i *InternalFileToTableExecutor) Query(
- ctx context.Context, opName, query, username string, qargs ...interface{},
+ ctx context.Context, opName, query string, username security.SQLUsername, qargs ...interface{},
) (*FileToTableExecutorRows, error) {
result := FileToTableExecutorRows{}
var err error
@@ -85,7 +87,7 @@ func (i *InternalFileToTableExecutor) Query(
// Exec implements the FileToTableSystemExecutor interface.
func (i *InternalFileToTableExecutor) Exec(
- ctx context.Context, opName, query, username string, qargs ...interface{},
+ ctx context.Context, opName, query string, username security.SQLUsername, qargs ...interface{},
) error {
_, err := i.ie.ExecEx(ctx, opName, nil,
sessiondata.InternalExecutorOverride{User: username}, query, qargs...)
@@ -108,7 +110,7 @@ func MakeSQLConnFileToTableExecutor(executor cloud.SQLConnI) *SQLConnFileToTable
// Query implements the FileToTableSystemExecutor interface.
func (i *SQLConnFileToTableExecutor) Query(
- ctx context.Context, _, query, _ string, qargs ...interface{},
+ ctx context.Context, _, query string, _ security.SQLUsername, qargs ...interface{},
) (*FileToTableExecutorRows, error) {
result := FileToTableExecutorRows{}
@@ -132,7 +134,7 @@ func (i *SQLConnFileToTableExecutor) Query(
// Exec implements the FileToTableSystemExecutor interface.
func (i *SQLConnFileToTableExecutor) Exec(
- ctx context.Context, _, query, _ string, qargs ...interface{},
+ ctx context.Context, _, query string, _ security.SQLUsername, qargs ...interface{},
) error {
argVals := make([]driver.NamedValue, len(qargs))
for i, qarg := range qargs {
@@ -158,7 +160,7 @@ func (i *SQLConnFileToTableExecutor) Exec(
type FileToTableSystem struct {
qualifiedTableName string
executor FileToTableSystemExecutor
- username string
+ username security.SQLUsername
}
// FileTable which contains records for every uploaded file.
@@ -235,7 +237,7 @@ func NewFileToTableSystem(
ctx context.Context,
qualifiedTableName string,
executor FileToTableSystemExecutor,
- username string,
+ username security.SQLUsername,
) (*FileToTableSystem, error) {
// Check the qualifiedTableName is parseable, so that we can return a useful
// error pre-emptively.
@@ -500,7 +502,9 @@ var _ io.WriteCloser = &chunkWriter{}
func newChunkWriter(
ctx context.Context,
chunkSize int,
- filename, username, fileTableName, payloadTableName string,
+ filename string,
+ username security.SQLUsername,
+ fileTableName, payloadTableName string,
ie *sql.InternalExecutor,
db *kv.DB,
) (*chunkWriter, error) {
@@ -634,7 +638,9 @@ func (f *fileReader) Close() error {
func newFileReader(
ctx context.Context,
- filename, username, fileTableName, payloadTableName string,
+ filename string,
+ username security.SQLUsername,
+ fileTableName, payloadTableName string,
ie *sql.InternalExecutor,
) (io.ReadCloser, error) {
fileTableReader, err := newFileTableReader(ctx, filename, username, fileTableName,
@@ -647,7 +653,9 @@ func newFileReader(
func newFileTableReader(
ctx context.Context,
- filename, username, fileTableName, payloadTableName string,
+ filename string,
+ username security.SQLUsername,
+ fileTableName, payloadTableName string,
ie *sql.InternalExecutor,
) (io.Reader, error) {
// Get file_id from metadata entry in File table.
@@ -729,7 +737,7 @@ func (f *FileToTableSystem) checkIfFileAndPayloadTableExist(
`SELECT table_name FROM [SHOW TABLES FROM %s] WHERE table_name=$1 OR table_name=$2`,
databaseSchema)
rows, err := ie.QueryEx(ctx, "tables-exist", nil,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
tableExistenceQuery, fileTableName, payloadTableName)
if err != nil {
return false, err
@@ -782,7 +790,7 @@ func (f *FileToTableSystem) grantCurrentUserTablePrivileges(
grantQuery := fmt.Sprintf(`GRANT SELECT, INSERT, DROP, DELETE ON TABLE %s, %s TO %s`,
f.GetFQFileTableName(), f.GetFQPayloadTableName(), f.username)
_, err := ie.QueryEx(ctx, "grant-user-file-payload-table-access", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
grantQuery)
if err != nil {
return errors.Wrap(err, "failed to grant access privileges to file and payload tables")
@@ -800,7 +808,7 @@ func (f *FileToTableSystem) revokeOtherUserTablePrivileges(
users WHERE NOT "username" = 'root' AND NOT "username" = 'admin' AND NOT "username" = $1`)
rows, err := ie.QueryEx(
ctx, "get-users", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
getUsersQuery, f.username,
)
if err != nil {
@@ -816,7 +824,7 @@ users WHERE NOT "username" = 'root' AND NOT "username" = 'admin' AND NOT "userna
revokeQuery := fmt.Sprintf(`REVOKE ALL ON TABLE %s, %s FROM %s`,
f.GetFQFileTableName(), f.GetFQPayloadTableName(), user)
_, err = ie.QueryEx(ctx, "revoke-user-privileges", txn,
- sessiondata.InternalExecutorOverride{User: security.RootUser},
+ sessiondata.InternalExecutorOverride{User: security.RootUserName()},
revokeQuery)
if err != nil {
return errors.Wrap(err, "failed to revoke privileges")
diff --git a/pkg/storage/cloudimpl/filetable/filetabletest/file_table_read_writer_test.go b/pkg/storage/cloudimpl/filetable/filetabletest/file_table_read_writer_test.go
index d096192692ca..c822e68f73d1 100644
--- a/pkg/storage/cloudimpl/filetable/filetabletest/file_table_read_writer_test.go
+++ b/pkg/storage/cloudimpl/filetable/filetabletest/file_table_read_writer_test.go
@@ -109,7 +109,7 @@ func TestListAndDeleteFiles(t *testing.T) {
executor := filetable.MakeInternalFileToTableExecutor(s.InternalExecutor().(*sql.
InternalExecutor), kvDB)
fileTableReadWriter, err := filetable.NewFileToTableSystem(ctx, qualifiedTableName,
- executor, security.RootUser)
+ executor, security.RootUserName())
require.NoError(t, err)
// Create first test file with multiple chunks.
@@ -160,7 +160,7 @@ func TestReadWriteFile(t *testing.T) {
executor := filetable.MakeInternalFileToTableExecutor(s.InternalExecutor().(*sql.
InternalExecutor), kvDB)
fileTableReadWriter, err := filetable.NewFileToTableSystem(ctx, qualifiedTableName,
- executor, security.RootUser)
+ executor, security.RootUserName())
require.NoError(t, err)
testFileName := "testfile"
@@ -311,7 +311,7 @@ func TestReadWriteFile(t *testing.T) {
// FileTable creation.
t.Run("no-db-or-schema-qualified-table-name", func(t *testing.T) {
_, err := filetable.NewFileToTableSystem(ctx, "foo",
- executor, security.RootUser)
+ executor, security.RootUserName())
testutils.IsError(err, "could not resolve db or schema name")
})
}
@@ -511,7 +511,7 @@ func TestDatabaseScope(t *testing.T) {
executor := filetable.MakeInternalFileToTableExecutor(s.InternalExecutor().(*sql.
InternalExecutor), kvDB)
fileTableReadWriter, err := filetable.NewFileToTableSystem(ctx, qualifiedTableName,
- executor, security.RootUser)
+ executor, security.RootUserName())
require.NoError(t, err)
// Verify defaultdb has the file we wrote.
@@ -528,7 +528,7 @@ func TestDatabaseScope(t *testing.T) {
_, err = sqlDB.Exec(`CREATE DATABASE newdb`)
require.NoError(t, err)
newFileTableReadWriter, err := filetable.NewFileToTableSystem(ctx,
- "newdb.file_table_read_writer", executor, security.RootUser)
+ "newdb.file_table_read_writer", executor, security.RootUserName())
require.NoError(t, err)
_, err = newFileTableReadWriter.ReadFile(ctx, "file1")
require.True(t, os.IsNotExist(err))
diff --git a/pkg/testutils/base.go b/pkg/testutils/base.go
index b468c167df5c..17d26026160f 100644
--- a/pkg/testutils/base.go
+++ b/pkg/testutils/base.go
@@ -19,11 +19,11 @@ import (
// embedded certs and the default node user. The default node user has both
// server and client certificates.
func NewNodeTestBaseContext() *base.Config {
- return NewTestBaseContext(security.NodeUser)
+ return NewTestBaseContext(security.NodeUserName())
}
// NewTestBaseContext creates a secure base context for user.
-func NewTestBaseContext(user string) *base.Config {
+func NewTestBaseContext(user security.SQLUsername) *base.Config {
cfg := &base.Config{
Insecure: false,
User: user,
diff --git a/pkg/testutils/jobutils/jobs_verification.go b/pkg/testutils/jobutils/jobs_verification.go
index 52e9c839a83f..6112e0b0a9e9 100644
--- a/pkg/testutils/jobutils/jobs_verification.go
+++ b/pkg/testutils/jobutils/jobs_verification.go
@@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/protoutil"
@@ -128,6 +129,7 @@ func verifySystemJob(
var statusString string
var runningStatus gosql.NullString
var runningStatusString string
+ var usernameString string
// We have to query for the nth job created rather than filtering by ID,
// because job-generating SQL queries (e.g. BACKUP) do not currently return
// the job ID.
@@ -137,9 +139,10 @@ func verifySystemJob(
filterType.String(),
offset,
).Scan(
- &actual.Description, &actual.Username, &rawDescriptorIDs,
+ &actual.Description, &usernameString, &rawDescriptorIDs,
&statusString, &runningStatus,
)
+ actual.Username = security.MakeSQLUsernameFromPreNormalizedString(usernameString)
if runningStatus.Valid {
runningStatusString = runningStatus.String
}
diff --git a/pkg/workload/dep_test.go b/pkg/workload/dep_test.go
index 36fdf458a3cf..3634ff9f4b32 100644
--- a/pkg/workload/dep_test.go
+++ b/pkg/workload/dep_test.go
@@ -28,6 +28,8 @@ func TestDepAllowlist(t *testing.T) {
`github.com/cockroachdb/cockroach/pkg/col/typeconv`,
`github.com/cockroachdb/cockroach/pkg/docs`,
`github.com/cockroachdb/cockroach/pkg/geo/geopb`,
+ `github.com/cockroachdb/cockroach/pkg/security`,
+ `github.com/cockroachdb/cockroach/pkg/sql/lexbase`,
`github.com/cockroachdb/cockroach/pkg/sql/lex`,
`github.com/cockroachdb/cockroach/pkg/sql/oidext`,
`github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb`,
diff --git a/pkg/workload/schemachange/schemachange.go b/pkg/workload/schemachange/schemachange.go
index c5a9603b9129..d47611162ac1 100644
--- a/pkg/workload/schemachange/schemachange.go
+++ b/pkg/workload/schemachange/schemachange.go
@@ -19,6 +19,7 @@ import (
"strings"
"sync/atomic"
+ "github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
@@ -1184,7 +1185,7 @@ func (w *schemaChangeWorker) createSchema(tx *pgx.Tx) (string, error) {
}
// TODO(jayshrivastava): Support authorization
- stmt := rowenc.MakeSchemaName(w.rng.Intn(2) == 0, schemaName, "")
+ stmt := rowenc.MakeSchemaName(w.rng.Intn(2) == 0, schemaName, security.RootUserName())
return tree.Serialize(stmt), nil
}