Skip to content

Commit

Permalink
*: make conversions between string and usernames more explicit
Browse files Browse the repository at this point in the history
tldr: the conversions between "external" strings and internal
usernames was unprincipled, and it turns out, incorrect in some
cases. This patch cleans this up by introducing a strict conversion
API.

**Background**

CockroachDB currently performs case folding and unicode NFC
normalization upon receiving usernames, specifically usernames received
from as SQL login principals.

Internally, usernames are often—but not always—considered
pre-normalized for the purpose of comparisons, privilege checks, role
membership checks and the like.

Finally, sometimes usernames are reported "back to outside".
In error messages, log files etc, but also:

- in the SQL syntax produced by SHOW CREATE, SHOW SYNTAX etc.
- to generate OS-level file names for exported files.

**New API**

This patch introduces a new data type `security.SQLUsername`.
It is incomparable and non-convertible with the Go `string`.

Engineers must now declare intent about how to do the conversion:

- `security.MakeSQLUsernameFromUserInput` converts an "outside" string
  to a username.
- `security.MakeSQLUsernameFromPreNormalizedString` promises that
  its argument has already been previously normalized.

To output usernames, the following APIs are also available.

- `username.Normalized()` produces the username itself, without
  decorations. These corresponds to the raw string (after
  normalization).

- `username.SQLIdentifier()` produces the username in valid
  SQL syntax, so that it can be injected safely in a SQL
  statement.

- `(*tree.FmtCtx).FormatUsername()` takes a username and properly
  handles quoting and anonymization, like `FormatName()` does for
  `tree.Name` already.

Likewise, conversion from/to protobuf is now regulated, via the new
APIs `username.EncodeProto()` and `usernameproto.Decode()`.

**Problems being solved**

- the usernames "from outside" were normalized sometimes, *but not
  consistently*:

  1. they were in the arguments of CREATE/DROP/ALTER ROLE. This was
     not changed.
  2. they were not consistently converted in `cockroach cert`. This was
     corrected.
  3. they were not in the `cockroach userfile` commands. This
     has been adjusted with a reference to issue cockroachdb#55389.
  4. they are *not* in GRANT/REVOKE. This patch does not change
     this behavior, but highlights it by spelling out
     `MakeSQLUsernameFromPreNormalizedString()` in the implementation.
  5. ditto for CREATE SCHEMA ... AUTHORIZATION and ALTER ... OWNER TO
  6. they were in the argument to `LoginRequest`. This was not
     changed.
  7. they were not in the argument of the other API requests that
     allow usernames, for example `ListSessions` or `CancelQuery`.
     This was not changed, but is now documented in the API.

- the usernames "inside" were incorrectly directly injected
  in SQL statements, even though they may contain special
  characters that create SQL syntax errors.

  This has been corrected by suitable uses of the new
  `SQLIdentifier()` method.

- There was an outright bug in a call to `CreateGCJobRec` (something
  about GCing jobs), where a `Description` field was passed in lieu
  of a username for a `User` field. The implications of this are unclear.

**Status after this change**

The new API makes it possible to audit exactly where "sensitive"
username/string conversion occurs. After this patch, we find the
following uses:

- `MakeSQLUsernameFromUserInput`:

  - pgwire user auth
  - CLI URL parsing
  - `cockroach userfile`
  - `cockroach cert`
  - `(*rpc.SecurityContext).PGURL()` (unsure whether that's a good thing)
  - CREATE/DROP/ALTER ROLE
  - when using literal strings as `role_spec` in the SQL grammar

- `MakeSQLUsernameFromPreNormalizedString`:

  - when intepreting the username in API query parameters, for
    those API documented as using pre-normalized usernames.
  - role membership checks inside SQL based on data read
    from `system` tables.
  - in GRANT/REVOKE (this is surprising, see above)

Release note: None
  • Loading branch information
knz committed Oct 25, 2020
1 parent 9fd5351 commit ca1a0c8
Show file tree
Hide file tree
Showing 259 changed files with 3,294 additions and 2,300 deletions.
4 changes: 2 additions & 2 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand Down Expand Up @@ -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 $@
Expand Down
8 changes: 4 additions & 4 deletions docs/generated/http/full.md
Original file line number Diff line number Diff line change
Expand Up @@ -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). |



Expand Down Expand Up @@ -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). |



Expand Down Expand Up @@ -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.<br><br>TODO(itsbilal): use [(gogoproto.customname) = "NodeID"] below. Need to figure out how to teach grpc-gateway about custom names.<br><br>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. |



Expand Down Expand Up @@ -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.<br><br>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. |



Expand Down
8 changes: 6 additions & 2 deletions docs/generated/sql/bnf/stmt_block.bnf
Original file line number Diff line number Diff line change
Expand Up @@ -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'

Expand Down Expand Up @@ -1174,7 +1174,7 @@ type_name_list ::=
type_list ::=
( typename ) ( ( ',' typename ) )*

non_reserved_word_or_sconst ::=
username_or_sconst ::=
non_reserved_word
| 'SCONST'

Expand Down Expand Up @@ -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 ) )*

Expand Down
4 changes: 2 additions & 2 deletions pkg/acceptance/cluster/certs.go
Original file line number Diff line number Diff line change
Expand Up @@ -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(
Expand Down
2 changes: 1 addition & 1 deletion pkg/acceptance/localcluster/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -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{
Expand Down
4 changes: 2 additions & 2 deletions pkg/base/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
11 changes: 6 additions & 5 deletions pkg/ccl/backupccl/backup_destination.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -31,7 +32,7 @@ import (
// chain.
func fetchPreviousBackups(
ctx context.Context,
user string,
user security.SQLUsername,
makeCloudStorage cloud.ExternalStorageFromURIFactory,
prevBackupURIs []string,
encryptionParams backupEncryptionParams,
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down
12 changes: 6 additions & 6 deletions pkg/ccl/backupccl/backup_destination_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand All @@ -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))
Expand All @@ -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))))
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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,
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()),
Expand Down Expand Up @@ -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 {
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/backupccl/backup_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
}
Expand Down
7 changes: 4 additions & 3 deletions pkg/ccl/backupccl/backup_processor_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -191,7 +192,7 @@ func makeBackupDataProcessorSpecs(
PKIDs: pkIDs,
BackupStartTime: startTime,
BackupEndTime: endTime,
User: user,
UserProto: user.EncodeProto(),
}
nodeToSpec[partition.Node] = spec
}
Expand All @@ -212,7 +213,7 @@ func makeBackupDataProcessorSpecs(
PKIDs: pkIDs,
BackupStartTime: startTime,
BackupEndTime: endTime,
User: user,
UserProto: user.EncodeProto(),
}
nodeToSpec[partition.Node] = spec
}
Expand Down
13 changes: 7 additions & 6 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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()
Expand Down Expand Up @@ -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",
Expand All @@ -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",
Expand Down Expand Up @@ -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),
Expand All @@ -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,
Expand Down Expand Up @@ -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),
Expand Down
3 changes: 2 additions & 1 deletion pkg/ccl/backupccl/create_scheduled_backup.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -408,7 +409,7 @@ func checkForExistingBackupsInCollection(

func makeBackupSchedule(
env scheduledjobs.JobSchedulerEnv,
owner string,
owner security.SQLUsername,
label string,
recurrence *scheduleRecurrence,
details jobspb.ScheduleDetails,
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/backupccl/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
)
Expand Down
Loading

0 comments on commit ca1a0c8

Please sign in to comment.