Skip to content

Commit

Permalink
privilege: add restricted read only (#25340)
Browse files Browse the repository at this point in the history
  • Loading branch information
ichn-hu authored Jul 16, 2021
1 parent bae5024 commit 4f26765
Show file tree
Hide file tree
Showing 16 changed files with 421 additions and 9 deletions.
56 changes: 56 additions & 0 deletions docs/design/2021-06-23-restricted-read-only.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
# Technical Design Document for Restrcted Read Only

> Author: [huzhifeng@pingcap.com](mailto:huzhifeng@pingcap.com)
This document is open to the web, and it will appear as a design doc PR to the TiDB repository.

# Introduction

In this document, we introduced the `TIDB_RESTRICTED_READ_ONLY` global variable. Turning it on will make the cluster read-only eventually for all users, including users with SUPER or CONNECTION_ADMIN privilege.

# Motivation or Background

See the [original specification](https://docs.google.com/document/d/1LtmW4fIohTjPcupD9smQGQpN2B2tRSFNyH8PFbg-L2c/edit).

# **Detailed Design**

## read-only handling

Currently, TiDB supports `read_only` and `super_read_only` global variables with no actual operation. Since TIDB_RESTRICTED_READ_ONLY does not comply with the behavior of MySQL read-only and super-read-only (which will block if there is table locked or ongoing committing of a transaction), we will not build TIDB_RESTRICTED_READ_ONLY upon read-only and super-read-only. These two variables will be remained untouched.

We will create a new global variable TIDB_RESTRICTED_READ_ONLY, and turning it to on and off will only be allowed in SEM mode, and only users with RESTRICTED_VARIABLES_ADMIN will be allowed to modify this variable.

To allow replication service to write, we introduced a new dynamic privilege level called `RESTRICTED_REPLICA_WRITER_ADMIN`, user with this privilege will be ignored for read-only checks.

Upon the change of the variable on a TiDB sever, the change will be broadcasted to all other TiDB servers through PD. Normally, the other TiDB servers will receive the update immediately, however under certain circumstances (such as TiDB server lose connection to PD), the lag can be up to 30 seconds.

### How to restrict SQLs

If read-only is turned on, we will reject SQLs that might change data during planning. The rules are:

- we won't restrict internal SQLs
- we rely on an allow list to determine whether SQL is allowed to execute
- we would allow set variables (otherwise can't unset them)
- we would allow `analyze table`
- we allow `show`
- we allow create and drop SQL bindings
- we allow prepare SQLs
- we allow `begin` and `rollback`
- we only allow `commit` if there is no change in the transaction, otherwise the transaction will abort
- finally, we resort to `planner/optimizer.go:IsReadOnly` for testing if the query is read-only for cases like explain, prepare and execute, etc.

## Privilege management

- add `tidb_restricted_read_only` to hidden variables, i.e., only users with `RESTRICTED_VARIABLES_ADMIN` privilege can modify it.
- alternative: add `RESTRICTED_READ_ONLY_ADMIN` privilege, only user with this privilege can change the setting, and user with this privilege can write to the cluster without being affected by the read-only setting. However it might be redundant for introducing a new privilege level.
- add `RESTRICTED_REPLICA_WRITER_ADMIN` to allow replication writer to surpass read-only checks. Note this privilege check is required even if SEM is not enabled, i.e., SUPER user still needs to be explicitly granted this privilege to be able to write.

## Difference with MySQL read-only

MySQL has support for read-only and super-read-only, which is currently not supported in TiDB. Restricted read only shares similar functionality as them, but differs in:

* When turning on read-only or super-read-only, MySQL might [fail or block when enabling read-only under some circumstances](https://dev.mysql.com/doc/refman/8.0/en/server-system-variables.html#sysvar_read_only). However turning on TIDB_RESTRICTED_READ_ONLY will return success immediately. Some other TiDB servers might not yet received the update of this variable, and will remain in updatable status until eventually the change on the global variable is broadcasted to all TiDB servers in the cluster.

## Alternative

- we will implement read-only and super-read-only that complies with MySQL's, and build TIDB_RESTRICED_READ_ONLY upon them, but we only need eventual read-only for TIDB_RESTRICTED_READ_ONLY. It is more relaxed, and it should always return success.
- we support read-only and super-read-only in a similar manner as TIDB_RESTRICED_READ_ONLY, this however will make its behavior incompatible with MySQL.
2 changes: 2 additions & 0 deletions domain/sysvar_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,6 +199,8 @@ func checkEnableServerGlobalVar(name, sVal string) {
break
}
variable.TopSQLVariable.ReportIntervalSeconds.Store(val)
case variable.TiDBRestrictedReadOnly:
variable.RestrictedReadOnly.Store(variable.TiDBOptOn(sVal))
}
if err != nil {
logutil.BgLogger().Error(fmt.Sprintf("load global variable %s error", name), zap.Error(err))
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1021,6 +1021,11 @@ error = '''
Internal : %s
'''

["planner:1836"]
error = '''
Running in read-only mode
'''

["planner:3029"]
error = '''
Expression #%d of ORDER BY contains aggregate function and applies to the result of a non-aggregated query
Expand Down
1 change: 1 addition & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -358,6 +358,7 @@ func (s *testSuiteP1) TestShow(c *C) {
"RESTRICTED_VARIABLES_ADMIN Server Admin ",
"RESTRICTED_USER_ADMIN Server Admin ",
"RESTRICTED_CONNECTION_ADMIN Server Admin ",
"RESTRICTED_REPLICA_WRITER_ADMIN Server Admin ",
))
c.Assert(len(tk.MustQuery("show table status").Rows()), Equals, 1)
}
Expand Down
1 change: 1 addition & 0 deletions planner/core/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,6 +93,7 @@ var (
ErrCTERecursiveForbidsAggregation = dbterror.ClassOptimizer.NewStd(mysql.ErrCTERecursiveForbidsAggregation)
ErrCTERecursiveForbiddenJoinOrder = dbterror.ClassOptimizer.NewStd(mysql.ErrCTERecursiveForbiddenJoinOrder)
ErrInvalidRequiresSingleReference = dbterror.ClassOptimizer.NewStd(mysql.ErrInvalidRequiresSingleReference)
ErrSQLInReadOnlyMode = dbterror.ClassOptimizer.NewStd(mysql.ErrReadOnlyMode)
// Since we cannot know if user logged in with a password, use message of ErrAccessDeniedNoPassword instead
ErrAccessDenied = dbterror.ClassOptimizer.NewStdErr(mysql.ErrAccessDenied, mysql.MySQLErrName[mysql.ErrAccessDeniedNoPassword])
ErrBadNull = dbterror.ClassOptimizer.NewStd(mysql.ErrBadNull)
Expand Down
50 changes: 50 additions & 0 deletions planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -221,6 +221,46 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
return bestPlan, names, nil
}

func allowInReadOnlyMode(sctx sessionctx.Context, node ast.Node) (bool, error) {
pm := privilege.GetPrivilegeManager(sctx)
if pm == nil {
return true, nil
}
roles := sctx.GetSessionVars().ActiveRoles
// allow replication thread
// NOTE: it is required, whether SEM is enabled or not, only user with explicit RESTRICTED_REPLICA_WRITER_ADMIN granted can ignore the restriction, so we need to surpass the case that if SEM is not enabled, SUPER will has all privileges
if pm.HasExplicitlyGrantedDynamicPrivilege(roles, "RESTRICTED_REPLICA_WRITER_ADMIN", false) {
return true, nil
}

switch node.(type) {
// allow change variables (otherwise can't unset read-only mode)
case *ast.SetStmt,
// allow analyze table
*ast.AnalyzeTableStmt,
*ast.UseStmt,
*ast.ShowStmt,
*ast.CreateBindingStmt,
*ast.DropBindingStmt,
*ast.PrepareStmt,
*ast.BeginStmt,
*ast.RollbackStmt:
return true, nil
case *ast.CommitStmt:
txn, err := sctx.Txn(true)
if err != nil {
return false, err
}
if !txn.IsReadOnly() {
return false, txn.Rollback()
}
return true, nil
}

vars := sctx.GetSessionVars()
return IsReadOnly(node, vars), nil
}

var planBuilderPool = sync.Pool{
New: func() interface{} {
return plannercore.NewPlanBuilder()
Expand Down Expand Up @@ -275,6 +315,16 @@ func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
return nil, nil, 0, err
}

if !sctx.GetSessionVars().InRestrictedSQL && variable.RestrictedReadOnly.Load() {
allowed, err := allowInReadOnlyMode(sctx, node)
if err != nil {
return nil, nil, 0, err
}
if !allowed {
return nil, nil, 0, errors.Trace(core.ErrSQLInReadOnlyMode)
}
}

// Handle the execute statement.
if execPlan, ok := p.(*plannercore.Execute); ok {
err := execPlan.OptimizePreparedPlan(ctx, sctx, is)
Expand Down
4 changes: 4 additions & 0 deletions privilege/privilege.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,6 +46,10 @@ type Manager interface {
// RequestVerificationWithUser verifies specific user privilege for the request.
RequestVerificationWithUser(db, table, column string, priv mysql.PrivilegeType, user *auth.UserIdentity) bool

// HasExplicitlyGrantedDynamicPrivilege verifies is a user has a dynamic privilege granted
// without using the SUPER privilege as a fallback.
HasExplicitlyGrantedDynamicPrivilege(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool

// RequestDynamicVerification verifies user privilege for a DYNAMIC privilege.
// Dynamic privileges are only assignable globally, and have their own grantable attribute.
RequestDynamicVerification(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool
Expand Down
14 changes: 12 additions & 2 deletions privilege/privileges/cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -963,8 +963,9 @@ func (p *MySQLPrivilege) matchColumns(user, host, db, table, column string) *col
return nil
}

// RequestDynamicVerification checks all roles for a specific DYNAMIC privilege.
func (p *MySQLPrivilege) RequestDynamicVerification(activeRoles []*auth.RoleIdentity, user, host, privName string, withGrant bool) bool {
// HasExplicitlyGrantedDynamicPrivilege checks if a user has a DYNAMIC privilege
// without accepting SUPER privilege as a fallback.
func (p *MySQLPrivilege) HasExplicitlyGrantedDynamicPrivilege(activeRoles []*auth.RoleIdentity, user, host, privName string, withGrant bool) bool {
privName = strings.ToUpper(privName)
roleList := p.FindAllRole(activeRoles)
roleList = append(roleList, &auth.RoleIdentity{Username: user, Hostname: host})
Expand All @@ -984,6 +985,15 @@ func (p *MySQLPrivilege) RequestDynamicVerification(activeRoles []*auth.RoleIden
}
}
}
return false
}

// RequestDynamicVerification checks all roles for a specific DYNAMIC privilege.
func (p *MySQLPrivilege) RequestDynamicVerification(activeRoles []*auth.RoleIdentity, user, host, privName string, withGrant bool) bool {
privName = strings.ToUpper(privName)
if p.HasExplicitlyGrantedDynamicPrivilege(activeRoles, user, host, privName, withGrant) {
return true
}
// If SEM is enabled, and the privilege is of type restricted, do not fall through
// To using SUPER as a replacement privilege.
if sem.IsEnabled() && sem.IsRestrictedPrivilege(privName) {
Expand Down
25 changes: 20 additions & 5 deletions privilege/privileges/privileges.go
Original file line number Diff line number Diff line change
Expand Up @@ -45,11 +45,12 @@ var dynamicPrivs = []string{
"SYSTEM_VARIABLES_ADMIN",
"ROLE_ADMIN",
"CONNECTION_ADMIN",
"RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled
"RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled.
"RESTRICTED_VARIABLES_ADMIN", // Can see all variables when SEM is enabled
"RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users.
"RESTRICTED_CONNECTION_ADMIN", // Can not be killed by PROCESS/CONNECTION_ADMIN privilege
"RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled
"RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled.
"RESTRICTED_VARIABLES_ADMIN", // Can see all variables when SEM is enabled
"RESTRICTED_USER_ADMIN", // User can not have their access revoked by SUPER users.
"RESTRICTED_CONNECTION_ADMIN", // Can not be killed by PROCESS/CONNECTION_ADMIN privilege
"RESTRICTED_REPLICA_WRITER_ADMIN", // Can write to the sever even when tidb_restriced_read_only is turned on.
}
var dynamicPrivLock sync.Mutex

Expand All @@ -76,6 +77,20 @@ func (p *UserPrivileges) RequestDynamicVerificationWithUser(privName string, gra
return mysqlPriv.RequestDynamicVerification(roles, user.Username, user.Hostname, privName, grantable)
}

// HasExplicitlyGrantedDynamicPrivilege checks if a user has a DYNAMIC privilege
// without accepting SUPER privilege as a fallback.
func (p *UserPrivileges) HasExplicitlyGrantedDynamicPrivilege(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool {
if SkipWithGrant {
return true
}
if p.user == "" && p.host == "" {
return true
}

mysqlPriv := p.Handle.Get()
return mysqlPriv.HasExplicitlyGrantedDynamicPrivilege(activeRoles, p.user, p.host, privName, grantable)
}

// RequestDynamicVerification implements the Manager interface.
func (p *UserPrivileges) RequestDynamicVerification(activeRoles []*auth.RoleIdentity, privName string, grantable bool) bool {
if SkipWithGrant {
Expand Down
3 changes: 2 additions & 1 deletion sessionctx/variable/sysvar.go
Original file line number Diff line number Diff line change
Expand Up @@ -1619,7 +1619,8 @@ var defaultSysVars = []*SysVar{
errors.RedactLogEnabled.Store(s.EnableRedactLog)
return nil
}},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBShardAllocateStep, Value: strconv.Itoa(DefTiDBShardAllocateStep), Type: TypeInt, MinValue: 1, MaxValue: math.MaxInt64, AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error {
{Scope: ScopeGlobal, Name: TiDBRestrictedReadOnly, Value: BoolToOnOff(DefTiDBRestrictedReadOnly), Type: TypeBool},
{Scope: ScopeGlobal | ScopeSession, Name: TiDBShardAllocateStep, Value: strconv.Itoa(DefTiDBShardAllocateStep), Type: TypeInt, MinValue: 1, MaxValue: uint64(math.MaxInt64), AutoConvertOutOfRange: true, SetSession: func(s *SessionVars, val string) error {
s.ShardAllocateStep = tidbOptInt64(val, DefTiDBShardAllocateStep)
return nil
}},
Expand Down
7 changes: 6 additions & 1 deletion sessionctx/variable/tidb_vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -504,6 +504,9 @@ const (
// TiDBRedactLog indicates that whether redact log.
TiDBRedactLog = "tidb_redact_log"

// TiDBRestrictedReadOnly is meant for the cloud admin to toggle the cluster read only
TiDBRestrictedReadOnly = "tidb_restricted_read_only"

// TiDBShardAllocateStep indicates the max size of continuous rowid shard in one transaction.
TiDBShardAllocateStep = "tidb_shard_allocate_step"
// TiDBEnableTelemetry indicates that whether usage data report to PingCAP is enabled.
Expand Down Expand Up @@ -698,6 +701,7 @@ const (
DefTiDBAllowAutoRandExplicitInsert = false
DefTiDBEnableClusteredIndex = ClusteredIndexDefModeIntOnly
DefTiDBRedactLog = false
DefTiDBRestrictedReadOnly = false
DefTiDBShardAllocateStep = math.MaxInt64
DefTiDBEnableTelemetry = true
DefTiDBEnableParallelApply = false
Expand Down Expand Up @@ -754,7 +758,8 @@ var (
MaxCollect: atomic.NewInt64(DefTiDBTopSQLMaxCollect),
ReportIntervalSeconds: atomic.NewInt64(DefTiDBTopSQLReportIntervalSeconds),
}
EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn)
EnableLocalTxn = atomic.NewBool(DefTiDBEnableLocalTxn)
RestrictedReadOnly = atomic.NewBool(DefTiDBRestrictedReadOnly)
)

// TopSQL is the variable for control top sql feature.
Expand Down
15 changes: 15 additions & 0 deletions tests/readonlytest/README.md
Original file line number Diff line number Diff line change
@@ -0,0 +1,15 @@
# E2E test for TiDB restricted read only

The test is not yet automated, so you need to set up an environment manually to test it.

Create a TiDB cluster with 2 TiDB servers on localhost. One server listen on port 4001 and another on 4002, then execute `go test` in this folder to run the test.

You are expected to see 2 tests passed.

```
$ go test
OK: 2 passed
PASS
ok github.com/pingcap/tidb/tests/readonlytest 2.150s
```

8 changes: 8 additions & 0 deletions tests/readonlytest/go.mod
Original file line number Diff line number Diff line change
@@ -0,0 +1,8 @@
module github.com/pingcap/tidb/tests/readonlytest

go 1.16

require (
github.com/go-sql-driver/mysql v1.6.0
github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712
)
Loading

0 comments on commit 4f26765

Please sign in to comment.