Skip to content

Commit

Permalink
*: add security enhanced mode part 2 (#24279)
Browse files Browse the repository at this point in the history
  • Loading branch information
morgo authored Apr 29, 2021
1 parent af6cece commit c5ca2ea
Show file tree
Hide file tree
Showing 10 changed files with 134 additions and 20 deletions.
4 changes: 2 additions & 2 deletions docs/design/2021-03-09-dynamic-privileges.md
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
# Proposal:

- Author(s): [morgo](https://github.com/morgo)
- Last updated: April 12, 2021
- Last updated: April 25, 2021
- Discussion at: N/A

## Table of Contents
Expand Down Expand Up @@ -239,7 +239,7 @@ No change
| Privilege Name | Description | Notes |
| --------------- | --------------- | --------------- |
| `RESTRICTED_SYSTEM_VARIABLES_ADMIN` | Allows changing a restricted `GLOBAL` system variable. | Currently in SEM all high risk variables are unloaded. TBD, it might be required in future that they are only visible/settable to those with this privilege and not SUPER. |
| `RESTRICTED_STATUS_VARIABLES_ADMIN` | Allows observing restricted status variables. | i.e. `SHOW GLOBAL STATUS` by default hides some statistics when `SEM` is enabled. |
| `RESTRICTED_STATUS_ADMIN` | Allows observing restricted status variables. | i.e. `SHOW GLOBAL STATUS` by default hides some statistics when `SEM` is enabled. |
| `RESTRICTED_CONNECTION_ADMIN` | A special privilege to say that their connections, etc. can’t be killed by SUPER users AND they can kill connections by all other users. Affects `KILL`, `KILL TIDB` commands. | It is intended for the CloudAdmin user in DBaaS. |
| `RESTRICTED_USER_ADMIN` | A special privilege to say that their access can’t be changed by `SUPER` users. Statements `DROP USER`, `SET PASSWORD`, `ALTER USER`, `REVOKE` are all limited. | It is intended for the CloudAdmin user in DbaaS. |
| `RESTRICTED_TABLES_ADMIN` | A special privilege which means that the SEM hidden table semantic doesn’t apply. | It is intended for the CloudAdmin user in DbaaS. |
Expand Down
18 changes: 9 additions & 9 deletions docs/design/2021-03-09-security-enhanced-mode.md
Original file line number Diff line number Diff line change
@@ -1,7 +1,7 @@
# Proposal:

- Author(s): [morgo](https://github.com/morgo)
- Last updated: April 12, 2021
- Last updated: April 25, 2021
- Discussion at: N/A

## Table of Contents
Expand Down Expand Up @@ -49,7 +49,7 @@ A boolean option called `EnableEnhancedSecurity` (default `FALSE`) will be added

### System Variables

The following system variables will be hidden:
The following system variables will be hidden unless the user has the `RESTRICTED_SYSTEM_VARIABLES_ADMIN` privilege:

* variable.TiDBDDLSlowOprThreshold,
* variable.TiDBAllowRemoveAutoInc,
Expand Down Expand Up @@ -78,13 +78,13 @@ The following system variables will be reset to defaults:

### Status Variables

The following status variables will be hidden:
The following status variables will be hidden unless the user has the `RESTRICTED_STATUS_ADMIN` privilege:

* tidb_gc_leader_desc

### Information Schema Tables

The following tables will be hidden:
The following tables will be hidden unless the user has the `RESTRICTED_TABLES_ADMIN` privilege:

* cluster_config
* cluster_hardware
Expand All @@ -99,7 +99,7 @@ The following tables will be hidden:
* metrics_tables
* tidb_hot_regions

The following tables will be modified to hide columns:
The following tables will be modified to hide columns unless the user has the `RESTRICTED_TABLES_ADMIN` privilege:

* tikv_store_status
* The address, capacity, available, start_ts and uptime columns will return NULL.
Expand All @@ -110,7 +110,7 @@ The following tables will be modified to hide columns:

### Performance Schema Tables

The following tables will be hidden:
The following tables will be hidden unless the user has the `RESTRICTED_TABLES_ADMIN` privilege:

* pd_profile_allocs
* pd_profile_block
Expand All @@ -128,7 +128,7 @@ The following tables will be hidden:

### System (mysql) Tables

The following tables will be hidden:
The following tables will be hidden unless the user has the `RESTRICTED_TABLES_ADMIN` privilege:

* expr_pushdown_blacklist
* gc_delete_range
Expand All @@ -137,11 +137,11 @@ The following tables will be hidden:
* tidb
* global_variables

The remaining system tables will be limited to read-only operations.
The remaining system tables will be limited to read-only operations and can not create new tables.

### Metrics Schema

All tables will be hidden, including the schema itself.
All tables will be hidden, including the schema itself unless the user has the `RESTRICTED_TABLES_ADMIN` privilege.

### Commands

Expand Down
36 changes: 32 additions & 4 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,7 @@ import (
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/pdapi"
"github.com/pingcap/tidb/util/sem"
"github.com/pingcap/tidb/util/set"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/stmtsummary"
Expand Down Expand Up @@ -132,7 +133,7 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex
case infoschema.TableSessionVar:
err = e.setDataFromSessionVar(sctx)
case infoschema.TableTiDBServersInfo:
err = e.setDataForServersInfo()
err = e.setDataForServersInfo(sctx)
case infoschema.TableTiFlashReplica:
e.dataForTableTiFlashReplica(sctx, dbs)
case infoschema.TableTiKVStoreStatus:
Expand Down Expand Up @@ -978,6 +979,18 @@ func (e *memtableRetriever) dataForTiKVStoreStatus(ctx sessionctx.Context) (err
lastHeartbeatTs := types.NewTime(types.FromGoTime(storeStat.Status.LastHeartbeatTs), mysql.TypeDatetime, types.DefaultFsp)
row[17].SetMysqlTime(lastHeartbeatTs)
row[18].SetString(storeStat.Status.Uptime, mysql.DefaultCollationName)
if sem.IsEnabled() {
// Patch out IP addresses etc if the user does not have the RESTRICTED_TABLES_ADMIN privilege
checker := privilege.GetPrivilegeManager(ctx)
if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) {
row[1].SetString(strconv.FormatInt(storeStat.Store.ID, 10), mysql.DefaultCollationName)
row[1].SetNull()
row[6].SetNull()
row[7].SetNull()
row[16].SetNull()
row[18].SetNull()
}
}
e.rows = append(e.rows, row)
}
return nil
Expand Down Expand Up @@ -1120,6 +1133,15 @@ func (e *memtableRetriever) dataForTiDBClusterInfo(ctx sessionctx.Context) error
upTimeStr,
server.ServerID,
)
if sem.IsEnabled() {
checker := privilege.GetPrivilegeManager(ctx)
if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) {
row[1].SetString(strconv.FormatUint(server.ServerID, 10), mysql.DefaultCollationName)
row[2].SetNull()
row[5].SetNull()
row[6].SetNull()
}
}
rows = append(rows, row)
}
e.rows = rows
Expand All @@ -1143,7 +1165,7 @@ func (e *memtableRetriever) setDataFromKeyColumnUsage(ctx sessionctx.Context, sc

func (e *memtableRetriever) setDataForClusterProcessList(ctx sessionctx.Context) error {
e.setDataForProcessList(ctx)
rows, err := infoschema.AppendHostInfoToRows(e.rows)
rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows)
if err != nil {
return err
}
Expand Down Expand Up @@ -1729,7 +1751,7 @@ func (e *memtableRetriever) setDataForPseudoProfiling(sctx sessionctx.Context) {
}
}

func (e *memtableRetriever) setDataForServersInfo() error {
func (e *memtableRetriever) setDataForServersInfo(ctx sessionctx.Context) error {
serversInfo, err := infosync.GetAllServerInfo(context.Background())
if err != nil {
return err
Expand All @@ -1747,6 +1769,12 @@ func (e *memtableRetriever) setDataForServersInfo() error {
info.BinlogStatus, // BINLOG_STATUS
stringutil.BuildStringFromLabels(info.Labels), // LABELS
)
if sem.IsEnabled() {
checker := privilege.GetPrivilegeManager(ctx)
if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) {
row[1].SetNull() // clear IP
}
}
rows = append(rows, row)
}
e.rows = rows
Expand Down Expand Up @@ -1844,7 +1872,7 @@ func (e *memtableRetriever) setDataForStatementsSummary(ctx sessionctx.Context,
switch tableName {
case infoschema.ClusterTableStatementsSummary,
infoschema.ClusterTableStatementsSummaryHistory:
rows, err := infoschema.AppendHostInfoToRows(e.rows)
rows, err := infoschema.AppendHostInfoToRows(ctx, e.rows)
if err != nil {
return err
}
Expand Down
8 changes: 8 additions & 0 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ import (
"github.com/pingcap/tidb/util/format"
"github.com/pingcap/tidb/util/hack"
"github.com/pingcap/tidb/util/hint"
"github.com/pingcap/tidb/util/sem"
"github.com/pingcap/tidb/util/set"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/stringutil"
Expand Down Expand Up @@ -707,10 +708,17 @@ func (e *ShowExec) fetchShowStatus() error {
if err != nil {
return errors.Trace(err)
}
checker := privilege.GetPrivilegeManager(e.ctx)
for status, v := range statusVars {
if e.GlobalScope && v.Scope == variable.ScopeSession {
continue
}
// Skip invisible status vars if permission fails.
if sem.IsEnabled() && sem.IsInvisibleStatusVar(status) {
if checker == nil || !checker.RequestDynamicVerification(sessionVars.ActiveRoles, "RESTRICTED_STATUS_ADMIN", false) {
continue
}
}
switch v.Value.(type) {
case []interface{}, nil:
v.Value = fmt.Sprintf("%v", v.Value)
Expand Down
6 changes: 3 additions & 3 deletions executor/slow_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@ func (e *slowQueryRetriever) retrieve(ctx context.Context, sctx sessionctx.Conte
}
e.initializeAsyncParsing(ctx, sctx)
}
rows, retrieved, err := e.dataForSlowLog(ctx)
rows, retrieved, err := e.dataForSlowLog(ctx, sctx)
if err != nil {
return nil, err
}
Expand Down Expand Up @@ -193,7 +193,7 @@ func (e *slowQueryRetriever) parseDataForSlowLog(ctx context.Context, sctx sessi
e.parseSlowLog(ctx, sctx, reader, ParseSlowLogBatchSize)
}

func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context) ([][]types.Datum, bool, error) {
func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, bool, error) {
var (
task slowLogTask
ok bool
Expand All @@ -216,7 +216,7 @@ func (e *slowQueryRetriever) dataForSlowLog(ctx context.Context) ([][]types.Datu
continue
}
if e.table.Name.L == strings.ToLower(infoschema.ClusterTableSlowLog) {
rows, err := infoschema.AppendHostInfoToRows(rows)
rows, err := infoschema.AppendHostInfoToRows(sctx, rows)
return rows, false, err
}
return rows, false, nil
Expand Down
11 changes: 10 additions & 1 deletion infoschema/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,11 @@ import (

"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/domain/infosync"
"github.com/pingcap/tidb/privilege"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/sem"
)

// Cluster table list, attention:
Expand Down Expand Up @@ -78,12 +81,18 @@ func isClusterTableByName(dbName, tableName string) bool {
}

// AppendHostInfoToRows appends host info to the rows.
func AppendHostInfoToRows(rows [][]types.Datum) ([][]types.Datum, error) {
func AppendHostInfoToRows(ctx sessionctx.Context, rows [][]types.Datum) ([][]types.Datum, error) {
serverInfo, err := infosync.GetServerInfo()
if err != nil {
return nil, err
}
addr := serverInfo.IP + ":" + strconv.FormatUint(uint64(serverInfo.StatusPort), 10)
if sem.IsEnabled() {
checker := privilege.GetPrivilegeManager(ctx)
if checker == nil || !checker.RequestDynamicVerification(ctx.GetSessionVars().ActiveRoles, "RESTRICTED_TABLES_ADMIN", false) {
addr = serverInfo.ID
}
}
for i := range rows {
row := make([]types.Datum, 0, len(rows[i])+1)
row = append(row, types.NewStringDatum(addr))
Expand Down
3 changes: 2 additions & 1 deletion privilege/privileges/privileges.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,7 +43,8 @@ var dynamicPrivs = []string{
"SYSTEM_VARIABLES_ADMIN",
"ROLE_ADMIN",
"CONNECTION_ADMIN",
"RESTRICTED_TABLES_ADMIN",
"RESTRICTED_TABLES_ADMIN", // Can see system tables when SEM is enabled
"RESTRICTED_STATUS_ADMIN", // Can see all status vars when SEM is enabled.
}
var dynamicPrivLock sync.Mutex

Expand Down
52 changes: 52 additions & 0 deletions privilege/privileges/privileges_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1339,5 +1339,57 @@ func (s *testPrivilegeSuite) TestSecurityEnhancedModeRestrictedTables(c *C) {
mustExec(c, cloudAdminSe, "USE metrics_schema")
mustExec(c, cloudAdminSe, "SELECT * FROM metrics_schema.uptime")
mustExec(c, cloudAdminSe, "CREATE TABLE mysql.abcd (a int)")
}

func (s *testPrivilegeSuite) TestSecurityEnhancedModeInfoschema(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("CREATE USER uroot1, uroot2, uroot3")
tk.MustExec("GRANT SUPER ON *.* to uroot1 WITH GRANT OPTION") // super not process
tk.MustExec("SET tidb_enable_dynamic_privileges=1")
tk.MustExec("GRANT SUPER, PROCESS, RESTRICTED_TABLES_ADMIN ON *.* to uroot2 WITH GRANT OPTION")
tk.Se.Auth(&auth.UserIdentity{
Username: "uroot1",
Hostname: "localhost",
AuthUsername: "uroot",
AuthHostname: "%",
}, nil, nil)

sem.Enable()
defer sem.Disable()

// Even though we have super, we still can't read protected information from tidb_servers_info, cluster_* tables
tk.MustQuery(`SELECT COUNT(*) FROM information_schema.tidb_servers_info WHERE ip IS NOT NULL`).Check(testkit.Rows("0"))
tk.MustQuery(`SELECT COUNT(*) FROM information_schema.cluster_info WHERE status_address IS NOT NULL`).Check(testkit.Rows("0"))
// 36 = a UUID. Normally it is an IP address.
tk.MustQuery(`SELECT COUNT(*) FROM information_schema.CLUSTER_STATEMENTS_SUMMARY WHERE length(instance) != 36`).Check(testkit.Rows("0"))

// That is unless we have the RESTRICTED_TABLES_ADMIN privilege
tk.Se.Auth(&auth.UserIdentity{
Username: "uroot2",
Hostname: "localhost",
AuthUsername: "uroot",
AuthHostname: "%",
}, nil, nil)

// flip from is NOT NULL etc
tk.MustQuery(`SELECT COUNT(*) FROM information_schema.tidb_servers_info WHERE ip IS NULL`).Check(testkit.Rows("0"))
tk.MustQuery(`SELECT COUNT(*) FROM information_schema.cluster_info WHERE status_address IS NULL`).Check(testkit.Rows("0"))
tk.MustQuery(`SELECT COUNT(*) FROM information_schema.CLUSTER_STATEMENTS_SUMMARY WHERE length(instance) = 36`).Check(testkit.Rows("0"))
}

func (s *testPrivilegeSuite) TestSecurityEnhancedModeStatusVars(c *C) {
// Without TiKV the status var list does not include tidb_gc_leader_desc
// So we can only test that the dynamic privilege is grantable.
// We will have to use an integration test to run SHOW STATUS LIKE 'tidb_gc_leader_desc'
// and verify if it appears.
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("CREATE USER unostatus, ustatus")
tk.MustExec("SET tidb_enable_dynamic_privileges=1")
tk.MustExec("GRANT RESTRICTED_STATUS_ADMIN ON *.* to ustatus")
tk.Se.Auth(&auth.UserIdentity{
Username: "unostatus",
Hostname: "localhost",
AuthUsername: "uroot",
AuthHostname: "%",
}, nil, nil)
}
9 changes: 9 additions & 0 deletions util/sem/sem.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (

"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/logutil"
)

const (
Expand Down Expand Up @@ -56,6 +57,7 @@ const (
tidbProfileMemory = "tidb_profile_memory"
tidbProfileMutex = "tidb_profile_mutex"
tikvProfileCPU = "tikv_profile_cpu"
tidbGCLeaderDesc = "tidb_gc_leader_desc"
restrictedPriv = "RESTRICTED_"
)

Expand All @@ -68,6 +70,8 @@ var (
func Enable() {
atomic.StoreInt32(&semEnabled, 1)
variable.SetSysVar(variable.TiDBEnableEnhancedSecurity, variable.On)
// write to log so users understand why some operations are weird.
logutil.BgLogger().Info("tidb-server is operating with security enhanced mode (SEM) enabled")
}

// Disable disables SEM. This is intended to be used by the test-suite.
Expand Down Expand Up @@ -116,6 +120,11 @@ func IsInvisibleTable(dbLowerName, tblLowerName string) bool {
return false
}

// IsInvisibleStatusVar returns true if the status var needs to be hidden
func IsInvisibleStatusVar(varName string) bool {
return varName == tidbGCLeaderDesc
}

// IsRestrictedPrivilege returns true if the privilege shuld not be satisfied by SUPER
// As most dynamic privileges are.
func IsRestrictedPrivilege(privNameInUpper string) bool {
Expand Down
7 changes: 7 additions & 0 deletions util/sem/sem_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -67,3 +67,10 @@ func (s *testSecurity) TestIsRestrictedPrivilege(c *C) {
c.Assert(IsRestrictedPrivilege("BACKUP_ADMIN"), IsFalse)
c.Assert(IsRestrictedPrivilege("aa"), IsFalse)
}

func (s *testSecurity) TestIsInvisibleStatusVar(c *C) {
c.Assert(IsInvisibleStatusVar(tidbGCLeaderDesc), IsTrue)
c.Assert(IsInvisibleStatusVar("server_id"), IsFalse)
c.Assert(IsInvisibleStatusVar("ddl_schema_version"), IsFalse)
c.Assert(IsInvisibleStatusVar("Ssl_version"), IsFalse)
}

0 comments on commit c5ca2ea

Please sign in to comment.