Skip to content

Commit

Permalink
sql: allow non-admins to perform some RESTOREs
Browse files Browse the repository at this point in the history
Release justification: Low risk, high reward change to existing
functionality

Release note (sql change): Non-admin users are now permitted to execute
RESTORE statements as long as the restore does not depend on implicit
credentials and the user has the appropriate privileges to create all of
the resulting database objects. For database restores, this means the
user must have the CREATEDB role privilege. For table restores, the user
must have CREATE privileges on the parent database. Full cluster
restores still require admin privileges.
  • Loading branch information
solongordon committed Sep 1, 2020
1 parent a5628ba commit cffe8b0
Show file tree
Hide file tree
Showing 6 changed files with 217 additions and 28 deletions.
1 change: 1 addition & 0 deletions pkg/base/testing_knobs.go
Original file line number Diff line number Diff line change
Expand Up @@ -34,4 +34,5 @@ type TestingKnobs struct {
Server ModuleTestingKnobs
TenantTestingKnobs ModuleTestingKnobs
JobsTestingKnobs ModuleTestingKnobs
BackupRestore ModuleTestingKnobs
}
74 changes: 50 additions & 24 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -83,14 +83,22 @@ func init() {
cloud.RegisterKMSFromURIFactory(MakeTestKMS, "testkms")
}

type sqlDBKey struct {
server string
user string
}

type datadrivenTestState struct {
servers map[string]serverutils.TestServerInterface
dataDirs map[string]string
sqlDBs map[string]*gosql.DB
sqlDBs map[sqlDBKey]*gosql.DB
cleanupFns []func()
}

func (d *datadrivenTestState) cleanup(ctx context.Context) {
for _, db := range d.sqlDBs {
db.Close()
}
for _, s := range d.servers {
s.Stopper().Stop(ctx)
}
Expand All @@ -99,16 +107,23 @@ func (d *datadrivenTestState) cleanup(ctx context.Context) {
}
}

func (d *datadrivenTestState) addServer(t *testing.T, name, iodir string) {
func (d *datadrivenTestState) addServer(
t *testing.T, name, iodir string, allowImplicitAccess bool,
) {
var tc serverutils.TestClusterInterface
var cleanup func()
params := base.TestClusterArgs{}
if allowImplicitAccess {
params.ServerArgs.Knobs.BackupRestore = &sql.BackupRestoreTestingKnobs{
AllowImplicitAccess: true,
}
}
if iodir == "" {
_, tc, _, iodir, cleanup = BackupRestoreTestSetup(t, singleNode, 0, InitNone)
_, tc, _, iodir, cleanup = backupRestoreTestSetupWithParams(t, singleNode, 0, InitNone, params)
} else {
_, tc, _, cleanup = backupRestoreTestSetupEmpty(t, singleNode, iodir, InitNone)
_, tc, _, cleanup = backupRestoreTestSetupEmptyWithParams(t, singleNode, iodir, InitNone, params)
}
d.servers[name] = tc.Server(0)
d.sqlDBs[name] = tc.ServerConn(0)
d.dataDirs[name] = iodir
d.cleanupFns = append(d.cleanupFns, cleanup)
}
Expand All @@ -121,19 +136,27 @@ func (d *datadrivenTestState) getIODir(t *testing.T, server string) string {
return dir
}

func (d *datadrivenTestState) getSQLDB(t *testing.T, server string) *gosql.DB {
db, ok := d.sqlDBs[server]
if !ok {
t.Fatalf("server %s does not exist", server)
func (d *datadrivenTestState) getSQLDB(t *testing.T, server string, user string) *gosql.DB {
key := sqlDBKey{server, user}
if db, ok := d.sqlDBs[key]; ok {
return db
}
addr := d.servers[server].ServingSQLAddr()
pgURL, cleanup := sqlutils.PGUrl(t, addr, "TestBackupRestoreDataDriven", url.User(user))
d.cleanupFns = append(d.cleanupFns, cleanup)
db, err := gosql.Open("postgres", pgURL.String())
if err != nil {
t.Fatal(err)
}
d.sqlDBs[key] = db
return db
}

func newDatadrivenTestState() datadrivenTestState {
return datadrivenTestState{
servers: make(map[string]serverutils.TestServerInterface),
dataDirs: make(map[string]string),
sqlDBs: make(map[string]*gosql.DB),
sqlDBs: make(map[sqlDBKey]*gosql.DB),
}
}

Expand Down Expand Up @@ -169,36 +192,44 @@ func TestBackupRestoreDataDriven(t *testing.T) {
return ""
case "new-server":
var name, shareDirWith, iodir string
var allowImplicitAccess bool
d.ScanArgs(t, "name", &name)
if d.HasArg("share-io-dir") {
d.ScanArgs(t, "share-io-dir", &shareDirWith)
}
if shareDirWith != "" {
iodir = ds.getIODir(t, shareDirWith)
}
if d.HasArg("allow-implicit-access") {
allowImplicitAccess = true
}
lastCreatedServer = name
ds.addServer(t, name, iodir)
ds.addServer(t, name, iodir, allowImplicitAccess)
return ""
case "exec-sql":
var server string
server := lastCreatedServer
user := "root"
if d.HasArg("server") {
d.ScanArgs(t, "server", &server)
} else {
server = lastCreatedServer
}
_, err := ds.getSQLDB(t, server).Exec(d.Input)
if d.HasArg("user") {
d.ScanArgs(t, "user", &user)
}
_, err := ds.getSQLDB(t, server, user).Exec(d.Input)
if err == nil {
return ""
}
return err.Error()
case "query-sql":
var server string
server := lastCreatedServer
user := "root"
if d.HasArg("server") {
d.ScanArgs(t, "server", &server)
} else {
server = lastCreatedServer
}
rows, err := ds.getSQLDB(t, server).Query(d.Input)
if d.HasArg("user") {
d.ScanArgs(t, "user", &user)
}
rows, err := ds.getSQLDB(t, server, user).Query(d.Input)
if err != nil {
return err.Error()
}
Expand Down Expand Up @@ -4462,11 +4493,6 @@ func TestBackupRestorePermissions(t *testing.T) {
) {
t.Fatal(err)
}
if _, err := testuser.Exec(`RESTORE blah FROM 'blah'`); !testutils.IsError(
err, "only users with the admin role are allowed to RESTORE",
) {
t.Fatal(err)
}
})

t.Run("privs-required", func(t *testing.T) {
Expand Down
66 changes: 62 additions & 4 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
"github.com/cockroachdb/cockroach/pkg/sql/roleoption"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlerrors"
"github.com/cockroachdb/cockroach/pkg/sql/types"
Expand Down Expand Up @@ -1150,10 +1151,6 @@ func restorePlanHook(
ctx, span := tracing.ChildSpan(ctx, stmt.StatementTag())
defer tracing.FinishSpan(span)

if err := p.RequireAdminRole(ctx, "RESTORE"); err != nil {
return err
}

if !(p.ExtendedEvalContext().TxnImplicit || restoreStmt.Options.Detached) {
return errors.Errorf("RESTORE cannot be used inside a transaction without DETACHED option")
}
Expand Down Expand Up @@ -1183,6 +1180,11 @@ func restorePlanHook(
from[0][i] = parsed.String()
}
}

if err := checkPrivilegesForRestore(ctx, restoreStmt, p, from); err != nil {
return err
}

var endTime hlc.Timestamp
if restoreStmt.AsOf.Expr != nil {
var err error
Expand Down Expand Up @@ -1225,6 +1227,62 @@ func restorePlanHook(
return fn, utilccl.BulkJobExecutionResultHeader, nil, false, nil
}

func checkPrivilegesForRestore(
ctx context.Context, restoreStmt *tree.Restore, p sql.PlanHookState, from [][]string,
) error {
hasAdmin, err := p.HasAdminRole(ctx)
if err != nil {
return err
}
if hasAdmin {
return nil
}
// Do not allow full cluster restores.
if restoreStmt.DescriptorCoverage == tree.AllDescriptors {
return pgerror.Newf(
pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to restore full cluster backups")
}
// Do not allow tenant restores.
if restoreStmt.Targets.Tenant != (roachpb.TenantID{}) {
return pgerror.Newf(
pgcode.InsufficientPrivilege,
"only users with the admin role can perform RESTORE TENANT")
}
// Database restores require the CREATEDB privileges.
if len(restoreStmt.Targets.Databases) > 0 {
hasCreateDB, err := p.HasRoleOption(ctx, roleoption.CREATEDB)
if err != nil {
return err
}
if !hasCreateDB {
return pgerror.Newf(
pgcode.InsufficientPrivilege,
"only users with the CREATEDB privilege can restore databases")
}
}
knobs := p.ExecCfg().BackupRestoreTestingKnobs
if knobs == nil || !knobs.AllowImplicitAccess {
// Check that none of the sources rely on implicit access.
for i := range from {
for j := range from[i] {
uri := from[i][j]
hasExplicitAuth, uriScheme, err := cloudimpl.AccessIsWithExplicitAuth(uri)
if err != nil {
return err
}
if !hasExplicitAuth {
return pgerror.Newf(
pgcode.InsufficientPrivilege,
"only users with the admin role are allowed to RESTORE from the specified %s URI",
uriScheme)
}
}
}
}
return nil
}

func doRestorePlan(
ctx context.Context,
restoreStmt *tree.Restore,
Expand Down
88 changes: 88 additions & 0 deletions pkg/ccl/backupccl/testdata/backup-restore/permissions
Original file line number Diff line number Diff line change
@@ -0,0 +1,88 @@
# Test permissions checks for non-admin users running RESTORE.
new-server name=s1
----

exec-sql
CREATE DATABASE d;
CREATE TABLE d.t (x INT);
INSERT INTO d.t VALUES (1), (2), (3);
----

exec-sql
BACKUP TO 'nodelocal://0/test/'
----

# Start a new cluster with the same IO dir.
new-server name=s2 share-io-dir=s1 allow-implicit-access
----

exec-sql server=s2
CREATE USER testuser
----

# Restore into the new cluster.
exec-sql server=s2 user=testuser
RESTORE FROM 'nodelocal://0/test/'
----
pq: only users with the admin role are allowed to restore full cluster backups

exec-sql server=s2 user=testuser
RESTORE DATABASE d FROM 'nodelocal://0/test/'
----
pq: only users with the CREATEDB privilege can restore databases

exec-sql server=s2
CREATE DATABASE d
----

exec-sql server=s2 user=testuser
RESTORE TABLE d.t FROM 'nodelocal://0/test/'
----
pq: user testuser does not have CREATE privilege on database d

exec-sql server=s2
GRANT CREATE ON DATABASE d TO testuser
----

exec-sql server=s2 user=testuser
RESTORE TABLE d.t FROM 'nodelocal://0/test/'
----

query-sql server=s2
SELECT x FROM d.t ORDER BY x
----
1
2
3

exec-sql server=s2
DROP DATABASE d
----

exec-sql server=s2
ALTER USER testuser CREATEDB
----

exec-sql server=s2 user=testuser
RESTORE DATABASE d FROM 'nodelocal://0/test/'
----

query-sql server=s2
SELECT x FROM d.t ORDER BY x
----
1
2
3

# Test that implicit access is disallowed when the testing knob isn't set.
new-server name=s3 share-io-dir=s1
----

exec-sql server=s3
CREATE USER testuser
----

exec-sql server=s3 user=testuser
RESTORE TABLE d.t FROM 'nodelocal://0/test/'
----
pq: only users with the admin role are allowed to RESTORE from the specified nodelocal URI
3 changes: 3 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -549,6 +549,9 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
if tenantKnobs := cfg.TestingKnobs.TenantTestingKnobs; tenantKnobs != nil {
execCfg.TenantTestingKnobs = tenantKnobs.(*sql.TenantTestingKnobs)
}
if backupRestoreKnobs := cfg.TestingKnobs.BackupRestore; backupRestoreKnobs != nil {
execCfg.BackupRestoreTestingKnobs = backupRestoreKnobs.(*sql.BackupRestoreTestingKnobs)
}

statsRefresher := stats.MakeRefresher(
cfg.Settings,
Expand Down
13 changes: 13 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -673,6 +673,7 @@ type ExecutorConfig struct {
DistSQLRunTestingKnobs *execinfra.TestingKnobs
EvalContextTestingKnobs tree.EvalContextTestingKnobs
TenantTestingKnobs *TenantTestingKnobs
BackupRestoreTestingKnobs *BackupRestoreTestingKnobs
// HistogramWindowInterval is (server.Config).HistogramWindowInterval.
HistogramWindowInterval time.Duration

Expand Down Expand Up @@ -845,6 +846,18 @@ func (k *TenantTestingKnobs) CanSetClusterSettings() bool {
return k != nil && k.ClusterSettingsUpdater != nil
}

// BackupRestoreTestingKnobs contains knobs for backup and restore behavior.
type BackupRestoreTestingKnobs struct {
// AllowImplicitAccess allows implicit access to data sources for non-admin
// users. This enables using nodelocal for testing RESTORE permissions.
AllowImplicitAccess bool
}

var _ base.ModuleTestingKnobs = &BackupRestoreTestingKnobs{}

// ModuleTestingKnobs implements the base.ModuleTestingKnobs interface.
func (*BackupRestoreTestingKnobs) ModuleTestingKnobs() {}

// databaseCacheHolder is a thread-safe container for a *Cache.
// It also allows clients to block until the cache is updated to a desired
// state.
Expand Down

0 comments on commit cffe8b0

Please sign in to comment.