Skip to content

Commit

Permalink
Merge branch 'master' into zimuxia/dist-reorg
Browse files Browse the repository at this point in the history
  • Loading branch information
zimulala committed Jan 13, 2023
2 parents c64b193 + 9a917c8 commit 0fff15a
Show file tree
Hide file tree
Showing 50 changed files with 7,096 additions and 7,144 deletions.
2 changes: 0 additions & 2 deletions ddl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,6 @@ go_library(
"//parser",
"//parser/ast",
"//parser/charset",
"//parser/duration",
"//parser/format",
"//parser/model",
"//parser/mysql",
Expand Down Expand Up @@ -231,7 +230,6 @@ go_test(
"//parser/ast",
"//parser/auth",
"//parser/charset",
"//parser/duration",
"//parser/model",
"//parser/mysql",
"//parser/terror",
Expand Down
5 changes: 2 additions & 3 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,6 @@ import (
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/duration"
"github.com/pingcap/tidb/parser/format"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
Expand Down Expand Up @@ -3452,7 +3451,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast
case ast.TableOptionTTL, ast.TableOptionTTLEnable, ast.TableOptionTTLJobInterval:
var ttlInfo *model.TTLInfo
var ttlEnable *bool
var ttlJobInterval *duration.Duration
var ttlJobInterval *string

if ttlOptionsHandled {
continue
Expand Down Expand Up @@ -5431,7 +5430,7 @@ func (d *ddl) AlterTableSetTiFlashReplica(ctx sessionctx.Context, ident ast.Iden
// When `ttlInfo` is nil, and `ttlCronJobSchedule` is not, it will use the original `.TTLInfo` in the table info and modify the
// `.JobInterval`. If the `.TTLInfo` in the table info is empty, this function will return an error.
// When `ttlInfo` is not nil, it simply submits the job with the `ttlInfo` and ignore the `ttlEnable`.
func (d *ddl) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.Ident, ttlInfo *model.TTLInfo, ttlEnable *bool, ttlCronJobSchedule *duration.Duration) error {
func (d *ddl) AlterTableTTLInfoOrEnable(ctx sessionctx.Context, ident ast.Ident, ttlInfo *model.TTLInfo, ttlEnable *bool, ttlCronJobSchedule *string) error {
is := d.infoCache.GetLatest()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
Expand Down
2 changes: 2 additions & 0 deletions ddl/generated_column.go
Original file line number Diff line number Diff line change
Expand Up @@ -325,6 +325,8 @@ func (c *illegalFunctionChecker) Enter(inNode ast.Node) (outNode ast.Node, skipC
c.otherErr = expression.ErrNotSupportedYet.GenWithStackByArgs("Use of CAST( .. AS .. ARRAY) outside of functional index in CREATE(non-SELECT)/ALTER TABLE or in general expressions")
return inNode, true
}
case *ast.ParenthesesExpr:
return inNode, false
}
c.disallowCastArrayFunc = true
return inNode, false
Expand Down
2 changes: 1 addition & 1 deletion ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,7 @@ func buildIndexColumns(ctx sessionctx.Context, columns []*model.ColumnInfo, inde
}
if col.FieldType.IsArray() {
if mvIndex {
return nil, false, dbterror.ErrNotSupportedYet.GenWithStack("'more than one multi-valued key part per index'")
return nil, false, dbterror.ErrNotSupportedYet.GenWithStackByArgs("more than one multi-valued key part per index")
}
mvIndex = true
}
Expand Down
18 changes: 18 additions & 0 deletions ddl/resource_group_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,16 @@ func TestResourceGroupBaisc(t *testing.T) {
hook.OnJobUpdatedExported.Store(&onJobUpdatedExportedFunc)
dom.DDL().SetHook(hook)

tk.MustExec("set global tidb_enable_resource_control = 'off'")
tk.MustGetErrCode("create user usr1 resource group rg1", mysql.ErrResourceGroupSupportDisabled)
tk.MustExec("create user usr1")
tk.MustGetErrCode("alter user usr1 resource group rg1", mysql.ErrResourceGroupSupportDisabled)
tk.MustGetErrCode("create resource group x "+
"RRU_PER_SEC=1000 "+
"WRU_PER_SEC=2000", mysql.ErrResourceGroupSupportDisabled)

tk.MustExec("set global tidb_enable_resource_control = 'on'")

tk.MustExec("create resource group x " +
"RRU_PER_SEC=1000 " +
"WRU_PER_SEC=2000")
Expand All @@ -61,6 +71,14 @@ func TestResourceGroupBaisc(t *testing.T) {
g := testResourceGroupNameFromIS(t, tk.Session(), "x")
checkFunc(g)

tk.MustExec("set global tidb_enable_resource_control = DEFAULT")
tk.MustGetErrCode("alter resource group x "+
"RRU_PER_SEC=2000 "+
"WRU_PER_SEC=3000", mysql.ErrResourceGroupSupportDisabled)
tk.MustGetErrCode("drop resource group x ", mysql.ErrResourceGroupSupportDisabled)

tk.MustExec("set global tidb_enable_resource_control = 'on'")

tk.MustGetErrCode("create resource group x "+
"RRU_PER_SEC=1000 "+
"WRU_PER_SEC=2000", mysql.ErrResourceGroupExists)
Expand Down
14 changes: 4 additions & 10 deletions ddl/ttl.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@ import (
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/duration"
"github.com/pingcap/tidb/parser/format"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
Expand Down Expand Up @@ -52,7 +51,7 @@ func onTTLInfoChange(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err er
// at least one for them is not nil
var ttlInfo *model.TTLInfo
var ttlInfoEnable *bool
var ttlInfoJobInterval *duration.Duration
var ttlInfoJobInterval *string

if err := job.DecodeArgs(&ttlInfo, &ttlInfoEnable, &ttlInfoJobInterval); err != nil {
job.State = model.JobStateCancelled
Expand Down Expand Up @@ -196,7 +195,7 @@ func checkPrimaryKeyForTTLTable(tblInfo *model.TableInfo) error {
// if TTL, TTL_ENABLE or TTL_JOB_INTERVAL is not set in the config, the corresponding return value will be nil.
// if both of TTL and TTL_ENABLE are set, the `ttlInfo.Enable` will be equal with `ttlEnable`.
// if both of TTL and TTL_JOB_INTERVAL are set, the `ttlInfo.JobInterval` will be equal with `ttlCronJobSchedule`.
func getTTLInfoInOptions(options []*ast.TableOption) (ttlInfo *model.TTLInfo, ttlEnable *bool, ttlCronJobSchedule *duration.Duration, err error) {
func getTTLInfoInOptions(options []*ast.TableOption) (ttlInfo *model.TTLInfo, ttlEnable *bool, ttlCronJobSchedule *string, err error) {
for _, op := range options {
switch op.Tp {
case ast.TableOptionTTL:
Expand All @@ -214,17 +213,12 @@ func getTTLInfoInOptions(options []*ast.TableOption) (ttlInfo *model.TTLInfo, tt
IntervalExprStr: intervalExpr,
IntervalTimeUnit: int(op.TimeUnitValue.Unit),
Enable: true,
JobInterval: duration.Duration{Hour: 1},
JobInterval: "1h",
}
case ast.TableOptionTTLEnable:
ttlEnable = &op.BoolValue
case ast.TableOptionTTLJobInterval:
schedule, err := duration.ParseDuration(op.StrValue)
if err != nil {
// this branch is actually unreachable, as the value has been validated in parser
return nil, nil, nil, err
}
ttlCronJobSchedule = &schedule
ttlCronJobSchedule = &op.StrValue
}
}

Expand Down
14 changes: 7 additions & 7 deletions ddl/ttl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,20 +18,20 @@ import (
"testing"

"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/duration"
"github.com/pingcap/tidb/parser/model"
"github.com/stretchr/testify/assert"
)

func Test_getTTLInfoInOptions(t *testing.T) {
falseValue := false
trueValue := true
twentyFourHours := "24h"

cases := []struct {
options []*ast.TableOption
ttlInfo *model.TTLInfo
ttlEnable *bool
ttlCronJobSchedule *duration.Duration
ttlCronJobSchedule *string
err error
}{
{
Expand All @@ -55,7 +55,7 @@ func Test_getTTLInfoInOptions(t *testing.T) {
IntervalExprStr: "5",
IntervalTimeUnit: int(ast.TimeUnitYear),
Enable: true,
JobInterval: duration.Duration{Hour: 1},
JobInterval: "1h",
},
nil,
nil,
Expand All @@ -79,7 +79,7 @@ func Test_getTTLInfoInOptions(t *testing.T) {
IntervalExprStr: "5",
IntervalTimeUnit: int(ast.TimeUnitYear),
Enable: false,
JobInterval: duration.Duration{Hour: 1},
JobInterval: "1h",
},
&falseValue,
nil,
Expand Down Expand Up @@ -107,7 +107,7 @@ func Test_getTTLInfoInOptions(t *testing.T) {
IntervalExprStr: "5",
IntervalTimeUnit: int(ast.TimeUnitYear),
Enable: true,
JobInterval: duration.Duration{Hour: 1},
JobInterval: "1h",
},
&trueValue,
nil,
Expand All @@ -131,10 +131,10 @@ func Test_getTTLInfoInOptions(t *testing.T) {
IntervalExprStr: "5",
IntervalTimeUnit: int(ast.TimeUnitYear),
Enable: true,
JobInterval: duration.Duration{Day: 1},
JobInterval: "24h",
},
nil,
&duration.Duration{Day: 1},
&twentyFourHours,
nil,
},
}
Expand Down
15 changes: 0 additions & 15 deletions distsql/request_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -782,20 +782,5 @@ func EncodeIndexKey(sc *stmtctx.StatementContext, ran *ranger.Range) ([]byte, []
if !ran.HighExclude {
high = kv.Key(high).PrefixNext()
}

var hasNull bool
for _, highVal := range ran.HighVal {
if highVal.IsNull() {
hasNull = true
break
}
}

// NOTE: this is a hard-code operation to avoid wrong results when accessing unique index with NULL;
// Please see https://github.com/pingcap/tidb/issues/29650 for more details
if hasNull {
// Append 0 to make unique-key range [null, null] to be a scan rather than point-get.
high = kv.Key(high).Next()
}
return low, high, nil
}
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -1095,6 +1095,7 @@ const (
ErrIngestFailed = 8247
ErrResourceGroupExists = 8248
ErrResourceGroupNotExists = 8249
ErrResourceGroupSupportDisabled = 8250

// TiKV/PD/TiFlash errors.
ErrPDServerTimeout = 9001
Expand Down
4 changes: 3 additions & 1 deletion errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1101,7 +1101,9 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrResourceGroupExists: mysql.Message("Resource group '%-.192s' already exists", nil),
ErrResourceGroupNotExists: mysql.Message("Unknown resource group '%-.192s'", nil),

ErrColumnInChange: mysql.Message("column %s id %d does not exist, this column may have been updated by other DDL ran in parallel", nil),
ErrColumnInChange: mysql.Message("column %s id %d does not exist, this column may have been updated by other DDL ran in parallel", nil),
ErrResourceGroupSupportDisabled: mysql.Message("Resource group feature is disabled", nil),

// TiKV/PD errors.
ErrPDServerTimeout: mysql.Message("PD server timeout: %s", nil),
ErrTiKVServerTimeout: mysql.Message("TiKV server timeout", nil),
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -2531,6 +2531,11 @@ error = '''
Unknown resource group '%-.192s'
'''

["schema:8250"]
error = '''
Resource group feature is disabled
'''

["session:8002"]
error = '''
[%d] can not retry select for update statement
Expand Down
9 changes: 9 additions & 0 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -743,13 +743,22 @@ func (e *DDLExec) executeAlterPlacementPolicy(s *ast.AlterPlacementPolicyStmt) e
}

func (e *DDLExec) executeCreateResourceGroup(s *ast.CreateResourceGroupStmt) error {
if !variable.EnableResourceControl.Load() {
return infoschema.ErrResourceGroupSupportDisabled
}
return domain.GetDomain(e.ctx).DDL().CreateResourceGroup(e.ctx, s)
}

func (e *DDLExec) executeAlterResourceGroup(s *ast.AlterResourceGroupStmt) error {
if !variable.EnableResourceControl.Load() {
return infoschema.ErrResourceGroupSupportDisabled
}
return domain.GetDomain(e.ctx).DDL().AlterResourceGroup(e.ctx, s)
}

func (e *DDLExec) executeDropResourceGroup(s *ast.DropResourceGroupStmt) error {
if !variable.EnableResourceControl.Load() {
return infoschema.ErrResourceGroupSupportDisabled
}
return domain.GetDomain(e.ctx).DDL().DropResourceGroup(e.ctx, s)
}
4 changes: 2 additions & 2 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1564,7 +1564,7 @@ func TestCreateTableWithTTL(t *testing.T) {

tk.MustGetErrMsg("CREATE TABLE t (id int) TTL_JOB_INTERVAL = '1h'", "[ddl:8150]Cannot set TTL_JOB_INTERVAL on a table without TTL config")

tk.MustExec("CREATE TABLE t (created_at datetime) TTL_ENABLE = 'ON' TTL = `created_at` + INTERVAL 1 DAY TTL_ENABLE = 'OFF' TTL_JOB_INTERVAL = '24h'")
tk.MustExec("CREATE TABLE t (created_at datetime) TTL_ENABLE = 'ON' TTL = `created_at` + INTERVAL 1 DAY TTL_ENABLE = 'OFF' TTL_JOB_INTERVAL = '1d'")
tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`created_at` + INTERVAL 1 DAY */ /*T![ttl] TTL_ENABLE='OFF' */ /*T![ttl] TTL_JOB_INTERVAL='1d' */"))
tk.MustExec("DROP TABLE t")

Expand All @@ -1586,7 +1586,7 @@ func TestAlterTTLInfo(t *testing.T) {
tk.MustExec("ALTER TABLE t TTL_ENABLE = 'OFF'")
tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`updated_at` + INTERVAL 2 YEAR */ /*T![ttl] TTL_ENABLE='OFF' */ /*T![ttl] TTL_JOB_INTERVAL='1h' */"))

tk.MustExec("ALTER TABLE t TTL_JOB_INTERVAL = '24h'")
tk.MustExec("ALTER TABLE t TTL_JOB_INTERVAL = '1d'")
tk.MustQuery("SHOW CREATE TABLE t").Check(testkit.Rows("t CREATE TABLE `t` (\n `created_at` datetime DEFAULT NULL,\n `updated_at` datetime DEFAULT NULL,\n `wrong_type` int(11) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![ttl] TTL=`updated_at` + INTERVAL 2 YEAR */ /*T![ttl] TTL_ENABLE='OFF' */ /*T![ttl] TTL_JOB_INTERVAL='1d' */"))

tk.MustGetErrMsg("ALTER TABLE t TTL = `not_exist` + INTERVAL 2 YEAR", "[ddl:1054]Unknown column 'not_exist' in 'TTL config'")
Expand Down
15 changes: 15 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1521,6 +1521,21 @@ func TestSetOperation(t *testing.T) {
tk.MustQuery("explain " + tt).Check(testkit.Rows(output[i].Plan...))
tk.MustQuery(tt).Sort().Check(testkit.Rows(output[i].Res...))
}

// from https://github.com/pingcap/tidb/issues/40279
tk.MustExec("CREATE TABLE `issue40279` (`a` char(155) NOT NULL DEFAULT 'on1unvbxp5sko6mbetn3ku26tuiyju7w3wc0olzto9ew7gsrx',`b` mediumint(9) NOT NULL DEFAULT '2525518',PRIMARY KEY (`b`,`a`) /*T![clustered_index] CLUSTERED */);")
tk.MustExec("insert into `issue40279` values ();")
tk.MustQuery("( select `issue40279`.`b` as r0 , from_base64( `issue40279`.`a` ) as r1 from `issue40279` ) " +
"except ( " +
"select `issue40279`.`a` as r0 , elt(2, `issue40279`.`a` , `issue40279`.`a` ) as r1 from `issue40279`);").
Check(testkit.Rows("2525518 <nil>"))
tk.MustExec("drop table if exists t2")

tk.MustExec("CREATE TABLE `t2` ( `a` varchar(20) CHARACTER SET gbk COLLATE gbk_chinese_ci DEFAULT NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")
tk.MustExec("insert into t2 values(0xCED2)")
result := tk.MustQuery("(select elt(2,t2.a,t2.a) from t2) except (select 0xCED2 from t2)")
rows := result.Rows()
require.Len(t, rows, 0)
}

func TestSetOperationOnDiffColType(t *testing.T) {
Expand Down
11 changes: 11 additions & 0 deletions executor/issuetest/executor_issue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1339,3 +1339,14 @@ PARTITION BY LIST COLUMNS(c_int)
tk.MustExec("delete t1, t2 from t1, t2 where t1.c_enum in ('blue');")
tk.MustExec("commit")
}

func TestIssue40158(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (_id int PRIMARY KEY, c1 char, index (c1));")
tk.MustExec("insert into t1 values (1, null);")
tk.MustQuery("select * from t1 where c1 is null and _id < 1;").Check(testkit.Rows())
}
2 changes: 1 addition & 1 deletion executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -1269,7 +1269,7 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T
err = restoreCtx.WriteWithSpecialComments(tidb.FeatureIDTTL, func() error {
restoreCtx.WriteKeyWord("TTL_JOB_INTERVAL")
restoreCtx.WritePlain("=")
restoreCtx.WriteString(tableInfo.TTLInfo.JobInterval.String())
restoreCtx.WriteString(tableInfo.TTLInfo.JobInterval)
return nil
})

Expand Down
6 changes: 6 additions & 0 deletions executor/simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -1092,6 +1092,9 @@ func (e *SimpleExec) executeCreateUser(ctx context.Context, s *ast.CreateUserStm
}
resourceGroupName := "default"
if s.ResourceGroupNameOption != nil {
if !variable.EnableResourceControl.Load() {
return infoschema.ErrResourceGroupSupportDisabled
}
if s.ResourceGroupNameOption.Type == ast.UserResourceGroupName {
resourceGroupName = s.ResourceGroupNameOption.Value
}
Expand Down Expand Up @@ -1894,6 +1897,9 @@ func (e *SimpleExec) executeAlterUser(ctx context.Context, s *ast.AlterUserStmt)
}
}
if s.ResourceGroupNameOption != nil && s.ResourceGroupNameOption.Type == ast.UserResourceGroupName {
if !variable.EnableResourceControl.Load() {
return infoschema.ErrResourceGroupSupportDisabled
}
newAttributes = append(newAttributes, fmt.Sprintf(`"resource_group": "%s"`, s.ResourceGroupNameOption.Value))
}
if passwordLockingStr != "" {
Expand Down
3 changes: 2 additions & 1 deletion executor/simple_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -128,6 +128,7 @@ func TestUserAttributes(t *testing.T) {
rootTK.MustExec("create user usr1@'%' identified by 'passord'")
rootTK.MustExec("alter user usr1 comment 'comment1'")
rootTK.MustQuery("select user_attributes from mysql.user where user = 'usr1'").Check(testkit.Rows(`{"metadata": {"comment": "comment1"}, "resource_group": "default"}`))
rootTK.MustExec("alter user usr1 resource group 'rg1'")
rootTK.MustExec("set global tidb_enable_resource_control = 'on'")
rootTK.MustExec("alter user usr1 resource group rg1")
rootTK.MustQuery("select user_attributes from mysql.user where user = 'usr1'").Check(testkit.Rows(`{"metadata": {"comment": "comment1"}, "resource_group": "rg1"}`))
}
Loading

0 comments on commit 0fff15a

Please sign in to comment.