Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
  • Loading branch information
Howie59 committed May 2, 2021
2 parents 6a86dcd + 27cacd8 commit ce12242
Show file tree
Hide file tree
Showing 96 changed files with 1,388 additions and 424 deletions.
2 changes: 1 addition & 1 deletion CHANGELOG.md
Original file line number Diff line number Diff line change
@@ -1,5 +1,5 @@
# TiDB Changelog
All notable changes to this project will be documented in this file. See also [Release Notes](https://github.com/pingcap/docs/blob/master/releases/rn.md), [TiKV Changelog](https://github.com/tikv/tikv/blob/master/CHANGELOG.md) and [PD Changelog](https://github.com/tikv/pd/blob/master/CHANGELOG.md).
All notable changes to this project will be documented in this file. See also [Release Notes](https://github.com/pingcap/docs/blob/master/releases/release-notes.md), [TiKV Changelog](https://github.com/tikv/tikv/blob/master/CHANGELOG.md) and [PD Changelog](https://github.com/tikv/pd/blob/master/CHANGELOG.md).

## [3.0.4] 2019-10-08
## New features
Expand Down
2 changes: 1 addition & 1 deletion ddl/backfilling.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,7 +159,7 @@ func newBackfillWorker(sessCtx sessionctx.Context, worker *worker, id int, t tab
sessCtx: sessCtx,
taskCh: make(chan *reorgBackfillTask, 1),
resultCh: make(chan *backfillResult, 1),
priority: tikvstore.PriorityLow,
priority: kv.PriorityLow,
}
}

Expand Down
2 changes: 1 addition & 1 deletion ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -233,7 +233,7 @@ func (s *testColumnTypeChangeSuite) TestRollbackColumnTypeChangeBetweenInteger(c
SQL := "alter table t modify column c2 int not null"
_, err := tk.Exec(SQL)
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[ddl:1]MockRollingBackInCallBack-none")
c.Assert(err.Error(), Equals, "[ddl:1]MockRollingBackInCallBack-queueing")
assertRollBackedColUnchanged(c, tk)

// Mock roll back at model.StateDeleteOnly.
Expand Down
31 changes: 31 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ var _ = Suite(&testDBSuite5{&testDBSuite{}})
var _ = Suite(&testDBSuite6{&testDBSuite{}})
var _ = Suite(&testDBSuite7{&testDBSuite{}})
var _ = SerialSuites(&testSerialDBSuite{&testDBSuite{}})
var _ = Suite(&testDBSuite8{&testDBSuite{}})

const defaultBatchSize = 1024
const defaultReorgBatchSize = 256
Expand Down Expand Up @@ -145,6 +146,7 @@ type testDBSuite5 struct{ *testDBSuite }
type testDBSuite6 struct{ *testDBSuite }
type testDBSuite7 struct{ *testDBSuite }
type testSerialDBSuite struct{ *testDBSuite }
type testDBSuite8 struct{ *testDBSuite }

func testAddIndexWithPK(tk *testkit.TestKit) {
tk.MustExec("drop table if exists test_add_index_with_pk")
Expand Down Expand Up @@ -6700,3 +6702,32 @@ func (s *testSerialDBSuite) TestJsonUnmarshalErrWhenPanicInCancellingPath(c *C)
_, err := tk.Exec("alter table test_add_index_after_add_col add unique index cc(c);")
c.Assert(err.Error(), Equals, "[kv:1062]Duplicate entry '0' for key 'cc'")
}

// For Close issue #24288
// see https://github.com/pingcap/tidb/issues/24288
func (s *testDBSuite8) TestDdlMaxLimitOfIdentifier(c *C) {
tk := testkit.NewTestKit(c, s.store)

// create/drop database test
longDbName := strings.Repeat("库", mysql.MaxDatabaseNameLength-1)
tk.MustExec(fmt.Sprintf("create database %s", longDbName))
defer func() {
tk.MustExec(fmt.Sprintf("drop database %s", longDbName))
}()
tk.MustExec(fmt.Sprintf("use %s", longDbName))

// create/drop table,index test
longTblName := strings.Repeat("表", mysql.MaxTableNameLength-1)
longColName := strings.Repeat("三", mysql.MaxColumnNameLength-1)
longIdxName := strings.Repeat("索", mysql.MaxIndexIdentifierLen-1)
tk.MustExec(fmt.Sprintf("create table %s(f1 int primary key,f2 int, %s varchar(50))", longTblName, longColName))
tk.MustExec(fmt.Sprintf("create index %s on %s(%s)", longIdxName, longTblName, longColName))
defer func() {
tk.MustExec(fmt.Sprintf("drop index %s on %s", longIdxName, longTblName))
tk.MustExec(fmt.Sprintf("drop table %s", longTblName))
}()

// alter table
tk.MustExec(fmt.Sprintf("alter table %s change f2 %s int", longTblName, strings.Repeat("二", mysql.MaxColumnNameLength-1)))

}
21 changes: 15 additions & 6 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"strings"
"sync/atomic"
"time"
"unicode/utf8"

"github.com/cznic/mathutil"
"github.com/go-yaml/yaml"
Expand Down Expand Up @@ -225,21 +226,21 @@ func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error)
}

func checkTooLongSchema(schema model.CIStr) error {
if len(schema.L) > mysql.MaxDatabaseNameLength {
if utf8.RuneCountInString(schema.L) > mysql.MaxDatabaseNameLength {
return ErrTooLongIdent.GenWithStackByArgs(schema)
}
return nil
}

func checkTooLongTable(table model.CIStr) error {
if len(table.L) > mysql.MaxTableNameLength {
if utf8.RuneCountInString(table.L) > mysql.MaxTableNameLength {
return ErrTooLongIdent.GenWithStackByArgs(table)
}
return nil
}

func checkTooLongIndex(index model.CIStr) error {
if len(index.L) > mysql.MaxIndexIdentifierLen {
if utf8.RuneCountInString(index.L) > mysql.MaxIndexIdentifierLen {
return ErrTooLongIdent.GenWithStackByArgs(index)
}
return nil
Expand Down Expand Up @@ -1107,7 +1108,7 @@ func checkGeneratedColumn(colDefs []*ast.ColumnDef) error {
func checkTooLongColumn(cols []*model.ColumnInfo) error {
for _, col := range cols {
colName := col.Name.O
if len(colName) > mysql.MaxColumnNameLength {
if utf8.RuneCountInString(colName) > mysql.MaxColumnNameLength {
return ErrTooLongIdent.GenWithStackByArgs(colName)
}
}
Expand Down Expand Up @@ -1751,6 +1752,14 @@ func buildTableInfoWithStmt(ctx sessionctx.Context, s *ast.CreateTableStmt, dbCh
if err != nil {
return nil, errors.Trace(err)
}
switch s.TemporaryKeyword {
case ast.TemporaryGlobal:
tbInfo.TempTableType = model.TempTableGlobal
case ast.TemporaryLocal:
tbInfo.TempTableType = model.TempTableLocal
case ast.TemporaryNone:
tbInfo.TempTableType = model.TempTableNone
}

if err = setTableAutoRandomBits(ctx, tbInfo, colDefs); err != nil {
return nil, errors.Trace(err)
Expand Down Expand Up @@ -2714,7 +2723,7 @@ func checkAndCreateNewColumn(ctx sessionctx.Context, ti ast.Ident, schema *model
if err = checkColumnAttributes(colName, specNewColumn.Tp); err != nil {
return nil, errors.Trace(err)
}
if len(colName) > mysql.MaxColumnNameLength {
if utf8.RuneCountInString(colName) > mysql.MaxColumnNameLength {
return nil, ErrTooLongIdent.GenWithStackByArgs(colName)
}

Expand Down Expand Up @@ -4980,7 +4989,7 @@ func buildHiddenColumnInfo(ctx sessionctx.Context, indexPartSpecifications []*as
}
idxPart.Length = types.UnspecifiedLength
// The index part is an expression, prepare a hidden column for it.
if len(idxPart.Column.Name.L) > mysql.MaxColumnNameLength {
if utf8.RuneCountInString(idxPart.Column.Name.L) > mysql.MaxColumnNameLength {
// TODO: Refine the error message.
return nil, ErrTooLongIdent.GenWithStackByArgs("hidden column")
}
Expand Down
3 changes: 1 addition & 2 deletions ddl/reorg.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,6 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/statistics"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
Expand Down Expand Up @@ -425,7 +424,7 @@ func (dc *ddlCtx) buildDescTableScan(ctx context.Context, startTS uint64, tbl ta
SetConcurrency(1).SetDesc(true)

builder.Request.NotFillCache = true
builder.Request.Priority = tikvstore.PriorityLow
builder.Request.Priority = kv.PriorityLow

kvReq, err := builder.Build()
if err != nil {
Expand Down
19 changes: 9 additions & 10 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
tikvstore "github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
Expand Down Expand Up @@ -145,8 +144,8 @@ func (builder *RequestBuilder) SetAnalyzeRequest(ana *tipb.AnalyzeReq) *RequestB
builder.Request.Tp = kv.ReqTypeAnalyze
builder.Request.Data, builder.err = ana.Marshal()
builder.Request.NotFillCache = true
builder.Request.IsolationLevel = tikvstore.RC
builder.Request.Priority = tikvstore.PriorityLow
builder.Request.IsolationLevel = kv.RC
builder.Request.Priority = kv.PriorityLow
}

return builder
Expand Down Expand Up @@ -199,24 +198,24 @@ func (builder *RequestBuilder) SetAllowBatchCop(batchCop bool) *RequestBuilder {
return builder
}

func (builder *RequestBuilder) getIsolationLevel() tikvstore.IsoLevel {
func (builder *RequestBuilder) getIsolationLevel() kv.IsoLevel {
switch builder.Tp {
case kv.ReqTypeAnalyze:
return tikvstore.RC
return kv.RC
}
return tikvstore.SI
return kv.SI
}

func (builder *RequestBuilder) getKVPriority(sv *variable.SessionVars) int {
switch sv.StmtCtx.Priority {
case mysql.NoPriority, mysql.DelayedPriority:
return tikvstore.PriorityNormal
return kv.PriorityNormal
case mysql.LowPriority:
return tikvstore.PriorityLow
return kv.PriorityLow
case mysql.HighPriority:
return tikvstore.PriorityHigh
return kv.PriorityHigh
}
return tikvstore.PriorityNormal
return kv.PriorityNormal
}

// SetFromSessionVars sets the following fields for "kv.Request" from session variables:
Expand Down
2 changes: 1 addition & 1 deletion distsql/request_builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -533,7 +533,7 @@ func (s *testSuite) TestRequestBuilder5(c *C) {
KeepOrder: true,
Desc: false,
Concurrency: 15,
IsolationLevel: tikvstore.RC,
IsolationLevel: kv.RC,
Priority: 1,
NotFillCache: true,
SyncLog: false,
Expand Down
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
10 changes: 5 additions & 5 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -213,7 +213,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec
if err != nil {
return nil, err
}
a.Ctx.GetSessionVars().StmtCtx.Priority = tikvstore.PriorityHigh
a.Ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityHigh

// try to reuse point get executor
if a.PsStmt.Executor != nil {
Expand Down Expand Up @@ -730,15 +730,15 @@ func (a *ExecStmt) buildExecutor() (Executor, error) {
if stmtPri := stmtCtx.Priority; stmtPri == mysql.NoPriority {
switch {
case useMaxTS:
stmtCtx.Priority = tikvstore.PriorityHigh
stmtCtx.Priority = kv.PriorityHigh
case a.LowerPriority:
stmtCtx.Priority = tikvstore.PriorityLow
stmtCtx.Priority = kv.PriorityLow
}
}
}
}
if _, ok := a.Plan.(*plannercore.Analyze); ok && ctx.GetSessionVars().InRestrictedSQL {
ctx.GetSessionVars().StmtCtx.Priority = tikvstore.PriorityLow
ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityLow
}

b := newExecutorBuilder(ctx, a.InfoSchema)
Expand All @@ -758,7 +758,7 @@ func (a *ExecStmt) buildExecutor() (Executor, error) {
a.isPreparedStmt = true
a.Plan = executorExec.plan
if executorExec.lowerPriority {
ctx.GetSessionVars().StmtCtx.Priority = tikvstore.PriorityLow
ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityLow
}
e = executorExec.stmtExec
}
Expand Down
Loading

0 comments on commit ce12242

Please sign in to comment.