Skip to content

Commit

Permalink
Merge branch 'master' into fix-write-temporary-table
Browse files Browse the repository at this point in the history
  • Loading branch information
tiancaiamao authored Jun 21, 2021
2 parents e768188 + bd8d3b1 commit 0dd1714
Show file tree
Hide file tree
Showing 84 changed files with 667 additions and 6,475 deletions.
3 changes: 2 additions & 1 deletion config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -494,7 +494,8 @@ type Binlog struct {
// PessimisticTxn is the config for pessimistic transaction.
type PessimisticTxn struct {
// The max count of retry for a single statement in a pessimistic transaction.
MaxRetryCount uint `toml:"max-retry-count" json:"max-retry-count"`
MaxRetryCount uint `toml:"max-retry-count" json:"max-retry-count"`
// The max count of deadlock events that will be recorded in the information_schema.deadlocks table.
DeadlockHistoryCapacity uint `toml:"deadlock-history-capacity" json:"deadlock-history-capacity"`
}

Expand Down
3 changes: 3 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -443,6 +443,9 @@ strategy = "range"
# max retry count for a statement in a pessimistic transaction.
max-retry-count = 256

# The max count of deadlock events that will be recorded in the information_schema.deadlocks table.
deadlock-history-capacity = 10

[stmt-summary]
# enable statement summary.
enable = true
Expand Down
3 changes: 3 additions & 0 deletions config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,8 @@ group= "abc"
zone= "dc-1"
[security]
spilled-file-encryption-method = "plaintext"
[pessimistic-txn]
deadlock-history-capacity = 123
`)

c.Assert(err, IsNil)
Expand Down Expand Up @@ -284,6 +286,7 @@ spilled-file-encryption-method = "plaintext"
c.Assert(conf.EnableEnumLengthLimit, Equals, false)
c.Assert(conf.EnableForwarding, Equals, true)
c.Assert(conf.StoresRefreshInterval, Equals, uint64(30))
c.Assert(conf.PessimisticTxn.DeadlockHistoryCapacity, Equals, uint(123))

_, err = f.WriteString(`
[log.file]
Expand Down
23 changes: 23 additions & 0 deletions ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2106,3 +2106,26 @@ func (s *testColumnTypeChangeSuite) TestChangePrefixedIndexColumnToNonPrefixOne(
tk.MustExec("create table t(a varchar(700), key(a(700)));")
tk.MustGetErrCode("alter table t change column a a tinytext;", mysql.ErrBlobKeyWithoutLength)
}

// Fix issue https://github.com/pingcap/tidb/issues/25469
func (s *testColumnTypeChangeSuite) TestCastToTimeStampDecodeError(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")

tk.MustExec("CREATE TABLE `t` (" +
" `a` datetime DEFAULT '1764-06-11 02:46:14'" +
") ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin COMMENT='7b84832e-f857-4116-8872-82fc9dcc4ab3'")
tk.MustExec("insert into `t` values();")
tk.MustGetErrCode("alter table `t` change column `a` `b` TIMESTAMP NULL DEFAULT '2015-11-14 07:12:24';", mysql.ErrTruncatedWrongValue)

tk.MustExec("drop table if exists t")
tk.MustExec("CREATE TABLE `t` (" +
" `a` date DEFAULT '1764-06-11 02:46:14'" +
") ENGINE=InnoDB DEFAULT CHARSET=latin1 COLLATE=latin1_bin COMMENT='7b84832e-f857-4116-8872-82fc9dcc4ab3'")
tk.MustExec("insert into `t` values();")
tk.MustGetErrCode("alter table `t` change column `a` `b` TIMESTAMP NULL DEFAULT '2015-11-14 07:12:24';", mysql.ErrTruncatedWrongValue)
tk.MustExec("drop table if exists t")

// Normal cast datetime to timestamp can succeed.
tk.MustQuery("select timestamp(cast('1000-11-11 12-3-1' as date));").Check(testkit.Rows("1000-11-11 00:00:00"))
}
5 changes: 3 additions & 2 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -310,6 +310,7 @@ func (s *testDBSuite8) TestAddPrimaryKeyRollback2(c *C) {
}

func (s *testDBSuite2) TestAddUniqueIndexRollback(c *C) {
c.Skip("unstable, skip it and fix it before 20210702")
hasNullValsInKey := false
idxName := "c3_index"
addIdxSQL := "create unique index c3_index on t1 (c3)"
Expand Down Expand Up @@ -5109,8 +5110,8 @@ func (s *testDBSuite1) TestModifyColumnTime_DatetimeToTimestamp(c *C) {
{"datetime", `20060102150405`, "timestamp", "2006-01-02 15:04:05", 0},
{"datetime", `060102150405`, "timestamp", "2006-01-02 15:04:05", 0},
{"datetime", `"2006-01-02 23:59:59.506"`, "timestamp", "2006-01-03 00:00:00", 0},
{"datetime", `"1000-01-02 23:59:59"`, "timestamp", "1000-01-02 23:59:59", 0},
{"datetime", `"9999-01-02 23:59:59"`, "timestamp", "9999-01-02 23:59:59", 0},
{"datetime", `"1971-01-02 23:59:59"`, "timestamp", "1971-01-02 23:59:59", 0},
{"datetime", `"2009-01-02 23:59:59"`, "timestamp", "2009-01-02 23:59:59", 0},
}
testModifyColumnTime(c, s.store, tests)
}
Expand Down
14 changes: 8 additions & 6 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1590,15 +1590,17 @@ func checkTableInfoValidWithStmt(ctx sessionctx.Context, tbInfo *model.TableInfo
if err := checkGeneratedColumn(s.Cols); err != nil {
return errors.Trace(err)
}
if tbInfo.Partition != nil && s.Partition != nil {
if tbInfo.Partition != nil {
if err := checkPartitionDefinitionConstraints(ctx, tbInfo); err != nil {
return errors.Trace(err)
}
if err := checkPartitionFuncType(ctx, s.Partition.Expr, tbInfo); err != nil {
return errors.Trace(err)
}
if err := checkPartitioningKeysConstraints(ctx, s, tbInfo); err != nil {
return errors.Trace(err)
if s.Partition != nil {
if err := checkPartitionFuncType(ctx, s.Partition.Expr, tbInfo); err != nil {
return errors.Trace(err)
}
if err := checkPartitioningKeysConstraints(ctx, s, tbInfo); err != nil {
return errors.Trace(err)
}
}
}
return nil
Expand Down
5 changes: 3 additions & 2 deletions ddl/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -524,7 +524,7 @@ func checkPartitionValuesIsInt(ctx sessionctx.Context, def *ast.PartitionDefinit
switch val.Kind() {
case types.KindUint64, types.KindNull:
case types.KindInt64:
if mysql.HasUnsignedFlag(tp.Flag) && val.GetInt64() < 0 {
if !ctx.GetSessionVars().SQLMode.HasNoUnsignedSubtractionMode() && mysql.HasUnsignedFlag(tp.Flag) && val.GetInt64() < 0 {
return ErrPartitionConstDomain.GenWithStackByArgs()
}
default:
Expand Down Expand Up @@ -666,7 +666,8 @@ func checkRangePartitionValue(ctx sessionctx.Context, tblInfo *model.TableInfo)
if strings.EqualFold(defs[len(defs)-1].LessThan[0], partitionMaxValue) {
defs = defs[:len(defs)-1]
}
isUnsigned := isColUnsigned(cols, pi)
// treat partition value under NoUnsignedSubtractionMode as signed
isUnsigned := isColUnsigned(cols, pi) && !ctx.GetSessionVars().SQLMode.HasNoUnsignedSubtractionMode()
var prevRangeValue interface{}
for i := 0; i < len(defs); i++ {
if strings.EqualFold(defs[i].LessThan[0], partitionMaxValue) {
Expand Down
61 changes: 41 additions & 20 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"sort"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/kvproto/pkg/metapb"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/ddl/placement"
Expand All @@ -33,20 +34,38 @@ import (
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tipb/go-tipb"
"github.com/tikv/client-go/v2/oracle"
)

// RequestBuilder is used to build a "kv.Request".
// It is called before we issue a kv request by "Select".
type RequestBuilder struct {
kv.Request
// txnScope indicates the value of txn_scope
txnScope string
is infoschema.InfoSchema
err error
is infoschema.InfoSchema
err error
}

// Build builds a "kv.Request".
func (builder *RequestBuilder) Build() (*kv.Request, error) {
if builder.TxnScope == "" {
builder.TxnScope = oracle.GlobalTxnScope
}
if builder.IsStaleness && builder.TxnScope != kv.GlobalTxnScope {
builder.MatchStoreLabels = []*metapb.StoreLabel{
{
Key: placement.DCLabelKey,
Value: builder.TxnScope,
},
}
}
failpoint.Inject("assertRequestBuilderStalenessOption", func(val failpoint.Value) {
assertScope := val.(string)
if len(assertScope) > 0 {
if builder.IsStaleness && assertScope != builder.TxnScope {
panic("batch point get staleness option fail")
}
}
})
err := builder.verifyTxnScope()
if err != nil {
builder.err = err
Expand Down Expand Up @@ -229,16 +248,6 @@ func (builder *RequestBuilder) SetFromSessionVars(sv *variable.SessionVars) *Req
builder.Request.TaskID = sv.StmtCtx.TaskID
builder.Request.Priority = builder.getKVPriority(sv)
builder.Request.ReplicaRead = sv.GetReplicaRead()
builder.txnScope = sv.TxnCtx.TxnScope
builder.IsStaleness = sv.TxnCtx.IsStaleness
if builder.IsStaleness && builder.txnScope != kv.GlobalTxnScope {
builder.MatchStoreLabels = []*metapb.StoreLabel{
{
Key: placement.DCLabelKey,
Value: builder.txnScope,
},
}
}
builder.SetResourceGroupTag(sv.StmtCtx)
return builder
}
Expand Down Expand Up @@ -284,10 +293,10 @@ func (builder *RequestBuilder) SetResourceGroupTag(sc *stmtctx.StatementContext)
}

func (builder *RequestBuilder) verifyTxnScope() error {
if builder.txnScope == "" {
builder.txnScope = kv.GlobalTxnScope
if builder.TxnScope == "" {
builder.TxnScope = kv.GlobalTxnScope
}
if builder.txnScope == kv.GlobalTxnScope || builder.is == nil {
if builder.TxnScope == kv.GlobalTxnScope || builder.is == nil {
return nil
}
visitPhysicalTableID := make(map[int64]struct{})
Expand All @@ -301,7 +310,7 @@ func (builder *RequestBuilder) verifyTxnScope() error {
}

for phyTableID := range visitPhysicalTableID {
valid := VerifyTxnScope(builder.txnScope, phyTableID, builder.is)
valid := VerifyTxnScope(builder.TxnScope, phyTableID, builder.is)
if !valid {
var tblName string
var partName string
Expand All @@ -313,17 +322,29 @@ func (builder *RequestBuilder) verifyTxnScope() error {
tblInfo, _ = builder.is.TableByID(phyTableID)
tblName = tblInfo.Meta().Name.String()
}
err := fmt.Errorf("table %v can not be read by %v txn_scope", tblName, builder.txnScope)
err := fmt.Errorf("table %v can not be read by %v txn_scope", tblName, builder.TxnScope)
if len(partName) > 0 {
err = fmt.Errorf("table %v's partition %v can not be read by %v txn_scope",
tblName, partName, builder.txnScope)
tblName, partName, builder.TxnScope)
}
return err
}
}
return nil
}

// SetTxnScope sets request TxnScope
func (builder *RequestBuilder) SetTxnScope(scope string) *RequestBuilder {
builder.TxnScope = scope
return builder
}

// SetIsStaleness sets request IsStaleness
func (builder *RequestBuilder) SetIsStaleness(is bool) *RequestBuilder {
builder.IsStaleness = is
return builder
}

// TableHandleRangesToKVRanges convert table handle ranges to "KeyRanges" for multiple tables.
func TableHandleRangesToKVRanges(sc *stmtctx.StatementContext, tid []int64, isCommonHandle bool, ranges []*ranger.Range, fb *statistics.QueryFeedback) ([]kv.KeyRange, error) {
if !isCommonHandle {
Expand Down
9 changes: 9 additions & 0 deletions distsql/request_builder_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tidb/util/testleak"
"github.com/pingcap/tipb/go-tipb"
"github.com/tikv/client-go/v2/oracle"
)

var _ = Suite(&testSuite{})
Expand Down Expand Up @@ -323,6 +324,7 @@ func (s *testSuite) TestRequestBuilder1(c *C) {
SyncLog: false,
Streaming: false,
ReplicaRead: kv.ReplicaReadLeader,
TxnScope: oracle.GlobalTxnScope,
}
c.Assert(actual, DeepEquals, expect)
}
Expand Down Expand Up @@ -399,6 +401,7 @@ func (s *testSuite) TestRequestBuilder2(c *C) {
SyncLog: false,
Streaming: false,
ReplicaRead: kv.ReplicaReadLeader,
TxnScope: oracle.GlobalTxnScope,
}
c.Assert(actual, DeepEquals, expect)
}
Expand Down Expand Up @@ -446,6 +449,7 @@ func (s *testSuite) TestRequestBuilder3(c *C) {
SyncLog: false,
Streaming: false,
ReplicaRead: kv.ReplicaReadLeader,
TxnScope: oracle.GlobalTxnScope,
}
c.Assert(actual, DeepEquals, expect)
}
Expand Down Expand Up @@ -493,6 +497,7 @@ func (s *testSuite) TestRequestBuilder4(c *C) {
NotFillCache: false,
SyncLog: false,
ReplicaRead: kv.ReplicaReadLeader,
TxnScope: oracle.GlobalTxnScope,
}
c.Assert(actual, DeepEquals, expect)
}
Expand Down Expand Up @@ -536,6 +541,7 @@ func (s *testSuite) TestRequestBuilder5(c *C) {
NotFillCache: true,
SyncLog: false,
Streaming: false,
TxnScope: oracle.GlobalTxnScope,
}
c.Assert(actual, DeepEquals, expect)
}
Expand Down Expand Up @@ -569,6 +575,7 @@ func (s *testSuite) TestRequestBuilder6(c *C) {
NotFillCache: true,
SyncLog: false,
Streaming: false,
TxnScope: oracle.GlobalTxnScope,
}

c.Assert(actual, DeepEquals, expect)
Expand Down Expand Up @@ -603,6 +610,7 @@ func (s *testSuite) TestRequestBuilder7(c *C) {
SyncLog: false,
Streaming: false,
ReplicaRead: replicaRead,
TxnScope: oracle.GlobalTxnScope,
}

c.Assert(actual, DeepEquals, expect)
Expand All @@ -624,6 +632,7 @@ func (s *testSuite) TestRequestBuilder8(c *C) {
Priority: 0,
MemTracker: (*memory.Tracker)(nil),
SchemaVar: 0,
TxnScope: oracle.GlobalTxnScope,
}
c.Assert(actual, DeepEquals, expect)
}
Expand Down
6 changes: 4 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: May 04, 2021
- Last updated: June 16, 2021
- Discussion at: N/A

## Table of Contents
Expand Down Expand Up @@ -233,14 +233,16 @@ No change
| `SYSTEM_VARIABLES_ADMIN` | Allows changing any GLOBAL system variable. | Currently this required `SUPER`. It will now require `SYSTEM_VARIABLES_ADMIN` or `SUPER`. |
| `ROLE_ADMIN` | Allows granting and revoking roles. | Won’t allow revoking on restricted_users (see below). |
| `CONNECTION_ADMIN` | Allows killing connections. | Like `PROCESS` static privilege, but slightly more restrictive (no show processlist). |
| `SYSTEM_USER` | The user can't be altered or dropped by only the `CREATE USER` privilege. | Helps prevent security escalations. |
#### TiDB Extensions
| Privilege Name | Description | Notes |
| --------------- | --------------- | --------------- |
| `RESTORE_ADMIN` | Restoring should require more permissions than backup because it is a higher risk operation. | It is inspired by MySQL's BACKUP_ADMIN/CLONE_ADMIN but not applicable to MySQL due to no online restore. |
| `RESTRICTED_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_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_CONNECTION_ADMIN` | Killing connections that belong to your own user is always permitted. Killing connections that belong to other users requires `CONNECTION_ADMIN` or `SUPER`; but there is an exception for `RESTRICTED_USER_ADMIN` users. To kill these connections also requires `RESTRICTED_CONNECTION_ADMIN`. This 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
7 changes: 5 additions & 2 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,8 @@ type ExecStmt struct {
SnapshotTS uint64
// ExplicitStaleness means whether the 'SELECT' clause are using 'AS OF TIMESTAMP' to perform stale read explicitly.
ExplicitStaleness bool
// TxnScope indicates the scope the store selector scope the request visited
TxnScope string
// InfoSchema stores a reference to the schema information.
InfoSchema infoschema.InfoSchema
// Plan stores a reference to the final physical plan.
Expand Down Expand Up @@ -245,7 +247,7 @@ func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*rec
}
}
if a.PsStmt.Executor == nil {
b := newExecutorBuilder(a.Ctx, is, a.Ti, a.SnapshotTS, a.ExplicitStaleness)
b := newExecutorBuilder(a.Ctx, is, a.Ti, a.SnapshotTS, a.ExplicitStaleness, a.TxnScope)
newExecutor := b.build(a.Plan)
if b.err != nil {
return nil, b.err
Expand Down Expand Up @@ -291,6 +293,7 @@ func (a *ExecStmt) RebuildPlan(ctx context.Context) (int64, error) {
a.InfoSchema = ret.InfoSchema
a.SnapshotTS = ret.LastSnapshotTS
a.ExplicitStaleness = ret.ExplicitStaleness
a.TxnScope = ret.TxnScope
p, names, err := planner.Optimize(ctx, a.Ctx, a.StmtNode, a.InfoSchema)
if err != nil {
return 0, err
Expand Down Expand Up @@ -792,7 +795,7 @@ func (a *ExecStmt) buildExecutor() (Executor, error) {
ctx.GetSessionVars().StmtCtx.Priority = kv.PriorityLow
}

b := newExecutorBuilder(ctx, a.InfoSchema, a.Ti, a.SnapshotTS, a.ExplicitStaleness)
b := newExecutorBuilder(ctx, a.InfoSchema, a.Ti, a.SnapshotTS, a.ExplicitStaleness, a.TxnScope)
e := b.build(a.Plan)
if b.err != nil {
return nil, errors.Trace(b.err)
Expand Down
2 changes: 1 addition & 1 deletion executor/analyze_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1038,7 +1038,7 @@ func (s *testSuite1) TestAnalyzeFullSamplingOnIndexWithVirtualColumnOrPrefixColu
tk.MustQuery("show stats_topn where table_name = 'sampling_index_prefix_col' and column_name = 'idx'").Check(testkit.Rows("test sampling_index_prefix_col idx 1 a 3"))
}

func (s *testSuite2) TestAnalyzeSamplingWorkPanic(c *C) {
func (s *testSerialSuite2) TestAnalyzeSamplingWorkPanic(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("set @@session.tidb_analyze_version = 2")
Expand Down
Loading

0 comments on commit 0dd1714

Please sign in to comment.