Skip to content

Commit

Permalink
Merge branch 'master' of https://github.com/pingcap/tidb into m/pessi…
Browse files Browse the repository at this point in the history
…mistic-lock-optimize
  • Loading branch information
MyonKeminta committed Jan 13, 2023
2 parents 4aa76a7 + bd7fb08 commit 3e8604d
Show file tree
Hide file tree
Showing 28 changed files with 208 additions and 223 deletions.
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
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)
}
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())
}
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
1 change: 1 addition & 0 deletions 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("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"}`))
}
38 changes: 23 additions & 15 deletions expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -433,11 +433,14 @@ func (c *castAsArrayFunctionClass) getFunction(ctx sessionctx.Context, args []Ex
}
arrayType := c.tp.ArrayType()
switch arrayType.GetType() {
case mysql.TypeYear, mysql.TypeJSON:
case mysql.TypeYear, mysql.TypeJSON, mysql.TypeDouble, mysql.TypeFloat, mysql.TypeNewDecimal:
return nil, ErrNotSupportedYet.GenWithStackByArgs(fmt.Sprintf("CAST-ing data to array of %s", arrayType.String()))
}
if arrayType.EvalType() == types.ETString && arrayType.GetCharset() != charset.CharsetUTF8MB4 && arrayType.GetCharset() != charset.CharsetBin {
return nil, ErrNotSupportedYet.GenWithStackByArgs("specifying charset for multi-valued index", arrayType.String())
return nil, ErrNotSupportedYet.GenWithStackByArgs("specifying charset for multi-valued index")
}
if arrayType.EvalType() == types.ETString && arrayType.GetFlen() == types.UnspecifiedLength {
return nil, ErrNotSupportedYet.GenWithStackByArgs("CAST-ing data to array of char/binary BLOBs")
}

bf, err := newBaseBuiltinFunc(ctx, c.funcName, args, c.tp)
Expand Down Expand Up @@ -467,22 +470,30 @@ func (b *castJSONAsArrayFunctionSig) evalJSON(row chunk.Row) (res types.BinaryJS
return res, isNull, err
}

if val.TypeCode != types.JSONTypeCodeArray {
return types.BinaryJSON{}, false, ErrNotSupportedYet.GenWithStackByArgs("CAST-ing Non-JSON Array type to array")
if val.TypeCode == types.JSONTypeCodeObject {
return types.BinaryJSON{}, false, ErrNotSupportedYet.GenWithStackByArgs("CAST-ing JSON OBJECT type to array")
}

arrayVals := make([]any, 0, len(b.args))
arrayVals := make([]any, 0, 8)
ft := b.tp.ArrayType()
f := convertJSON2Tp(ft.EvalType())
if f == nil {
return types.BinaryJSON{}, false, ErrNotSupportedYet.GenWithStackByArgs("CAS-ing JSON to the target type")
return types.BinaryJSON{}, false, ErrNotSupportedYet.GenWithStackByArgs(fmt.Sprintf("CAS-ing data to array of %s", ft.String()))
}
for i := 0; i < val.GetElemCount(); i++ {
item, err := f(fakeSctx, val.ArrayGetElem(i), ft)
if val.TypeCode != types.JSONTypeCodeArray {
item, err := f(fakeSctx, val, ft)
if err != nil {
return types.BinaryJSON{}, false, err
}
arrayVals = append(arrayVals, item)
} else {
for i := 0; i < val.GetElemCount(); i++ {
item, err := f(fakeSctx, val.ArrayGetElem(i), ft)
if err != nil {
return types.BinaryJSON{}, false, err
}
arrayVals = append(arrayVals, item)
}
}
return types.CreateBinaryJSON(arrayVals), false, nil
}
Expand Down Expand Up @@ -510,14 +521,11 @@ func convertJSON2Tp(evalType types.EvalType) func(*stmtctx.StatementContext, typ
if item.TypeCode != types.JSONTypeCodeInt64 && item.TypeCode != types.JSONTypeCodeUint64 {
return nil, ErrInvalidJSONForFuncIndex
}
return types.ConvertJSONToInt(sc, item, mysql.HasUnsignedFlag(tp.GetFlag()), tp.GetType())
}
case types.ETReal, types.ETDecimal:
return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) {
if item.TypeCode != types.JSONTypeCodeInt64 && item.TypeCode != types.JSONTypeCodeUint64 && item.TypeCode != types.JSONTypeCodeFloat64 {
return nil, ErrInvalidJSONForFuncIndex
jsonToInt, err := types.ConvertJSONToInt(sc, item, mysql.HasUnsignedFlag(tp.GetFlag()), tp.GetType())
if mysql.HasUnsignedFlag(tp.GetFlag()) {
return uint64(jsonToInt), err
}
return types.ConvertJSONToFloat(sc, item)
return jsonToInt, err
}
case types.ETDatetime:
return func(sc *stmtctx.StatementContext, item types.BinaryJSON, tp *types.FieldType) (any, error) {
Expand Down
14 changes: 0 additions & 14 deletions expression/builtin_cast_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1650,20 +1650,6 @@ func TestCastArrayFunc(t *testing.T) {
false,
true,
},
{
[]interface{}{"1", "2"},
nil,
types.NewFieldTypeBuilder().SetType(mysql.TypeDouble).SetCharset(charset.CharsetBin).SetCollate(charset.CharsetBin).SetArray(true).BuildP(),
false,
true,
},
{
[]interface{}{int64(-1), 2.1, int64(3)},
[]interface{}{int64(-1), 2.1, int64(3)},
types.NewFieldTypeBuilder().SetType(mysql.TypeDouble).SetCharset(charset.CharsetBin).SetCollate(charset.CharsetBin).SetArray(true).BuildP(),
true,
true,
},
}
for _, tt := range tbl {
f, err := BuildCastFunctionWithCheck(ctx, datumsToConstants(types.MakeDatums(types.CreateBinaryJSON(tt.input)))[0], tt.tp)
Expand Down
Loading

0 comments on commit 3e8604d

Please sign in to comment.