Skip to content

Commit

Permalink
Merge branch 'master' into fix_37255
Browse files Browse the repository at this point in the history
  • Loading branch information
Yisaer authored Aug 24, 2022
2 parents 9326a22 + 25dda97 commit 64acda6
Show file tree
Hide file tree
Showing 73 changed files with 545 additions and 300 deletions.
1 change: 1 addition & 0 deletions build/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,7 @@ nogo(
"@org_golang_x_tools//go/analysis/passes/ctrlflow:go_default_library",
"@org_golang_x_tools//go/analysis/passes/deepequalerrors:go_default_library",
"@org_golang_x_tools//go/analysis/passes/errorsas:go_default_library",
"@org_golang_x_tools//go/analysis/passes/fieldalignment:go_default_library",
"@org_golang_x_tools//go/analysis/passes/findcall:go_default_library",
"@org_golang_x_tools//go/analysis/passes/httpresponse:go_default_library",
"@org_golang_x_tools//go/analysis/passes/ifaceassert:go_default_library",
Expand Down
43 changes: 43 additions & 0 deletions build/nogo_config.json
Original file line number Diff line number Diff line change
Expand Up @@ -142,6 +142,49 @@
".*_/testmain\\.go$": "ignore code"
}
},
"fieldalignment": {
"exclude_files": {
"/external/": "no need to vet third party code",
".*_generated\\.go$": "ignore generated code",
".*_/testmain\\.go$": "ignore code",
".*_test\\.go$": "ignore test code"
},
"only_files": {
"util/checksum": "util/checksum code",
"util/processinfo.go": "util/processinfo.go code",
"util/cpuprofile/": "util/cpuprofile/ code",
"util/cteutil/": "util/cteutil/ code",
"util/dbutil/": "util/dbutil/ code",
"util/deadlockhistory/": "util/deadlockhistory/ code",
"util/domainutil/": "util/domainutil/ code",
"util/encrypt/": "util/encrypt/ code",
"util/etcd/": "util/etcd/ code",
"util/expensivequery/": "util/expensivequery/ code",
"util/filter/": "util/filter/ code",
"util/importer/": "util/importer/ code",
"util/keydecoder/": "util/keydecoder/ code",
"util/kvcache/": "util/kvcache/ code",
"util/localpool/": "util/localpool/ code",
"util/mathutil/": "util/mathutil/ code",
"util/memory/": "util/memory/ code",
"util/mock/": "util/mock/ code",
"util/mvmap/": "util/mvmap/ code",
"util/profile/": "util/profile/ code",
"util/ranger/": "util/ranger/ code",
"util/regexpr-router/": "util/regexpr-router/ code",
"util/schemacmp/": "util/schemacmp/ code",
"util/sqlexec/": "util/sqlexec/ code",
"util/stringutil/": "util/stringutil/ code",
"util/table-router/": "util/table-router/ code",
"util/timeutil/": "util/timeutil/ code",
"util/topsql/": "util/topsql/ code",
"util/tracing/": "util/tracing/ code",
"util/trxevents/": "util/trxevents/ code",
"util/watcher/": "util/watcher/ code",
"store/mockstore/unistore/util": "store/mockstore/unistore/util code",
"ddl/util/": "ddl/util code"
}
},
"findcall": {
"exclude_files": {
"/external/": "no need to vet third party code",
Expand Down
28 changes: 28 additions & 0 deletions cmd/explaintest/r/subquery.result
Original file line number Diff line number Diff line change
Expand Up @@ -46,3 +46,31 @@ create table t1(a int(11));
create table t2(a decimal(40,20) unsigned, b decimal(40,20));
select count(*) as x from t1 group by a having x not in (select a from t2 where x = t2.b);
x
drop table if exists stu;
drop table if exists exam;
create table stu(id int, name varchar(100));
insert into stu values(1, null);
create table exam(stu_id int, course varchar(100), grade int);
insert into exam values(1, 'math', 100);
set names utf8 collate utf8_general_ci;
explain format = 'brief' select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
id estRows task access object operator info
Apply 10000.00 root CARTESIAN anti semi join, other cond:eq(test.stu.name, Column#8)
├─TableReader(Build) 10000.00 root data:TableFullScan
│ └─TableFullScan 10000.00 cop[tikv] table:stu keep order:false, stats:pseudo
└─Projection(Probe) 10.00 root guo->Column#8
└─TableReader 10.00 root data:Selection
└─Selection 10.00 cop[tikv] eq(test.exam.stu_id, test.stu.id)
└─TableFullScan 10000.00 cop[tikv] table:exam keep order:false, stats:pseudo
select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
id name
set names utf8mb4;
explain format = 'brief' select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
id estRows task access object operator info
HashJoin 8000.00 root anti semi join, equal:[eq(test.stu.id, test.exam.stu_id)], other cond:eq(test.stu.name, "guo")
├─TableReader(Build) 10000.00 root data:TableFullScan
│ └─TableFullScan 10000.00 cop[tikv] table:exam keep order:false, stats:pseudo
└─TableReader(Probe) 10000.00 root data:TableFullScan
└─TableFullScan 10000.00 cop[tikv] table:stu keep order:false, stats:pseudo
select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
id name
13 changes: 13 additions & 0 deletions cmd/explaintest/t/subquery.test
Original file line number Diff line number Diff line change
Expand Up @@ -20,3 +20,16 @@ drop table if exists t1, t2;
create table t1(a int(11));
create table t2(a decimal(40,20) unsigned, b decimal(40,20));
select count(*) as x from t1 group by a having x not in (select a from t2 where x = t2.b);

drop table if exists stu;
drop table if exists exam;
create table stu(id int, name varchar(100));
insert into stu values(1, null);
create table exam(stu_id int, course varchar(100), grade int);
insert into exam values(1, 'math', 100);
set names utf8 collate utf8_general_ci;
explain format = 'brief' select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
set names utf8mb4;
explain format = 'brief' select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
select * from stu where stu.name not in (select 'guo' from exam where exam.stu_id = stu.id);
4 changes: 2 additions & 2 deletions ddl/util/event.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,11 @@ import (

// Event is an event that a ddl operation happened.
type Event struct {
Tp model.ActionType
TableInfo *model.TableInfo
PartInfo *model.PartitionInfo
ColumnInfos []*model.ColumnInfo
IndexInfo *model.IndexInfo
ColumnInfos []*model.ColumnInfo
Tp model.ActionType
}

// String implements fmt.Stringer interface.
Expand Down
6 changes: 4 additions & 2 deletions ddl/util/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -57,8 +57,10 @@ const (

// DelRangeTask is for run delete-range command in gc_worker.
type DelRangeTask struct {
JobID, ElementID int64
StartKey, EndKey kv.Key
StartKey kv.Key
EndKey kv.Key
JobID int64
ElementID int64
}

// Range returns the range [start, end) to delete.
Expand Down
65 changes: 65 additions & 0 deletions executor/executor_issue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
Expand Down Expand Up @@ -392,6 +393,70 @@ func TestIssue30971(t *testing.T) {
}
}

func TestIssue31678(t *testing.T) {
// The issue31678 is mainly about type conversion in UNION
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("USE test")
tk.MustExec("DROP TABLE IF EXISTS t1, t2;")

// https://github.com/pingcap/tidb/issues/31678
tk.MustExec("CREATE TABLE t1 (c VARCHAR(11)) CHARACTER SET utf8mb4")
tk.MustExec("CREATE TABLE t2 (b CHAR(1) CHARACTER SET binary, i INT)")
tk.MustExec("INSERT INTO t1 (c) VALUES ('н1234567890')")
tk.MustExec("INSERT INTO t2 (b, i) VALUES ('1', 1)")
var tests = []struct {
query string
expectedFlen int
expectedCharset string
result []string
}{
{"SELECT c FROM t1 UNION SELECT b FROM t2", 44, "binary", []string{"1", "н1234567890"}},
{"SELECT c FROM t1 UNION SELECT i FROM t2", 20, "utf8mb4", []string{"1", "н1234567890"}},
{"SELECT i FROM t2 UNION SELECT c FROM t1", 20, "utf8mb4", []string{"1", "н1234567890"}},
{"SELECT b FROM t2 UNION SELECT c FROM t1", 44, "binary", []string{"1", "н1234567890"}},
}
for _, test := range tests {
tk.MustQuery(test.query).Sort().Check(testkit.Rows(test.result...))
rs, err := tk.Exec(test.query)
require.NoError(t, err)
resultFields := rs.Fields()
require.Equal(t, 1, len(resultFields), test.query)
require.Equal(t, test.expectedFlen, resultFields[0].Column.FieldType.GetFlen(), test.query)
require.Equal(t, test.expectedCharset, resultFields[0].Column.FieldType.GetCharset(), test.query)
}
tk.MustExec("DROP TABLE t1, t2;")

// test some other charset
tk.MustExec("CREATE TABLE t1 (c1 VARCHAR(5) CHARACTER SET utf8mb4, c2 VARCHAR(1) CHARACTER SET binary)")
tk.MustExec("CREATE TABLE t2 (c1 CHAR(10) CHARACTER SET GBK, c2 VARCHAR(50) CHARACTER SET binary)")
tk.MustExec("INSERT INTO t1 VALUES ('一二三四五', '1')")
tk.MustExec("INSERT INTO t2 VALUES ('一二三四五六七八九十', '1234567890')")
gbkResult, err := charset.NewCustomGBKEncoder().String("一二三四五六七八九十")
require.NoError(t, err)
tests = []struct {
query string
expectedFlen int
expectedCharset string
result []string
}{
{"SELECT c1 FROM t1 UNION SELECT c1 FROM t2", 10, "utf8mb4", []string{"一二三四五", "一二三四五六七八九十"}},
{"SELECT c1 FROM t1 UNION SELECT c2 FROM t2", 50, "binary", []string{"1234567890", "一二三四五"}},
{"SELECT c2 FROM t1 UNION SELECT c1 FROM t2", 20, "binary", []string{"1", gbkResult}},
{"SELECT c2 FROM t1 UNION SELECT c2 FROM t2", 50, "binary", []string{"1", "1234567890"}},
}
for _, test := range tests {
tk.MustQuery(test.query).Sort().Check(testkit.Rows(test.result...))
rs, err := tk.Exec(test.query)
require.NoError(t, err)
resultFields := rs.Fields()
require.Equal(t, 1, len(resultFields), test.query)
require.Equal(t, test.expectedFlen, resultFields[0].Column.FieldType.GetFlen(), test.query)
require.Equal(t, test.expectedCharset, resultFields[0].Column.FieldType.GetCharset(), test.query)
}
tk.MustExec("DROP TABLE t1, t2;")
}

func TestIndexJoin31494(t *testing.T) {
store, dom := testkit.CreateMockStoreAndDomain(t)
tk := testkit.NewTestKit(t, store)
Expand Down
8 changes: 7 additions & 1 deletion expression/builtin_json.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,6 +109,7 @@ func (c *jsonTypeFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
bf.tp.SetCharset(charset)
bf.tp.SetCollate(collate)
bf.tp.SetFlen(51) // flen of JSON_TYPE is length of UNSIGNED INTEGER.
bf.tp.AddFlag(mysql.BinaryFlag)
sig := &builtinJSONTypeSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_JsonTypeSig)
return sig, nil
Expand Down Expand Up @@ -212,7 +213,8 @@ func (c *jsonUnquoteFunctionClass) getFunction(ctx sessionctx.Context, args []Ex
if err != nil {
return nil, err
}
bf.tp.SetFlen(mysql.MaxFieldVarCharLength)
bf.tp.SetFlen(args[0].GetType().GetFlen())
bf.tp.AddFlag(mysql.BinaryFlag)
DisableParseJSONFlag4Expr(args[0])
sig := &builtinJSONUnquoteSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_JsonUnquoteSig)
Expand Down Expand Up @@ -1145,6 +1147,8 @@ func (c *jsonPrettyFunctionClass) getFunction(ctx sessionctx.Context, args []Exp
if err != nil {
return nil, err
}
bf.tp.AddFlag(mysql.BinaryFlag)
bf.tp.SetFlen(mysql.MaxBlobWidth * 4)
sig := &builtinJSONSPrettySig{bf}
sig.setPbCode(tipb.ScalarFuncSig_JsonPrettySig)
return sig, nil
Expand Down Expand Up @@ -1200,6 +1204,8 @@ func (c *jsonQuoteFunctionClass) getFunction(ctx sessionctx.Context, args []Expr
return nil, err
}
DisableParseJSONFlag4Expr(args[0])
bf.tp.AddFlag(mysql.BinaryFlag)
bf.tp.SetFlen(args[0].GetType().GetFlen()*6 + 2)
sig := &builtinJSONQuoteSig{bf}
sig.setPbCode(tipb.ScalarFuncSig_JsonQuoteSig)
return sig, nil
Expand Down
13 changes: 10 additions & 3 deletions expression/typeinfer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2965,9 +2965,8 @@ func (s *InferTypeSuite) createTestCase4Literals() []typeInferTestCase {

func (s *InferTypeSuite) createTestCase4JSONFuncs() []typeInferTestCase {
return []typeInferTestCase{
{"json_type(c_json)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, 51, types.UnspecifiedLength},
// TODO: flen of json_unquote doesn't follow MySQL now.
{"json_unquote(c_json)", mysql.TypeVarString, charset.CharsetUTF8MB4, 0, mysql.MaxFieldVarCharLength, types.UnspecifiedLength},
{"json_type(c_json)", mysql.TypeVarString, charset.CharsetUTF8MB4, mysql.BinaryFlag, 51, types.UnspecifiedLength},
{"json_unquote(c_json)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxLongBlobWidth, types.UnspecifiedLength},
{"json_extract(c_json, '')", mysql.TypeJSON, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxBlobWidth, 0},
{"json_set(c_json, '', 0)", mysql.TypeJSON, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxBlobWidth, 0},
{"json_insert(c_json, '', 0)", mysql.TypeJSON, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxBlobWidth, 0},
Expand All @@ -2976,6 +2975,14 @@ func (s *InferTypeSuite) createTestCase4JSONFuncs() []typeInferTestCase {
{"json_merge(c_json, c_json)", mysql.TypeJSON, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxBlobWidth, 0},
{"json_object('k', 'v')", mysql.TypeJSON, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxBlobWidth, 0},
{"json_array('k', 'v')", mysql.TypeJSON, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxBlobWidth, 0},
{"json_pretty(c_json)", mysql.TypeLongBlob, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxBlobWidth * 4, types.UnspecifiedLength},
{"json_contains(c_json, 'a')", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0},
{"json_contains_path(c_json, 'one', '$.a')", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0},
{"json_quote('k')", mysql.TypeVarString, charset.CharsetUTF8MB4, mysql.BinaryFlag | mysql.NotNullFlag, 8, types.UnspecifiedLength},
{"json_valid(c_json)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0},
{`json_search(c_json, 'one', '"a"')`, mysql.TypeJSON, charset.CharsetUTF8MB4, mysql.BinaryFlag, mysql.MaxBlobWidth, 0},
{`json_depth(c_json)`, mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0},
{`json_length(c_json)`, mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxIntWidth, 0},
}
}

Expand Down
56 changes: 43 additions & 13 deletions expression/util.go
Original file line number Diff line number Diff line change
Expand Up @@ -366,7 +366,8 @@ func extractColumnSet(expr Expression, set *intsets.Sparse) {
}
}

func setExprColumnInOperand(expr Expression) Expression {
// SetExprColumnInOperand is used to set columns in expr as InOperand.
func SetExprColumnInOperand(expr Expression) Expression {
switch v := expr.(type) {
case *Column:
col := v.Clone().(*Column)
Expand All @@ -375,7 +376,7 @@ func setExprColumnInOperand(expr Expression) Expression {
case *ScalarFunction:
args := v.GetArgs()
for i, arg := range args {
args[i] = setExprColumnInOperand(arg)
args[i] = SetExprColumnInOperand(arg)
}
}
return expr
Expand All @@ -384,44 +385,65 @@ func setExprColumnInOperand(expr Expression) Expression {
// ColumnSubstitute substitutes the columns in filter to expressions in select fields.
// e.g. select * from (select b as a from t) k where a < 10 => select * from (select b as a from t where b < 10) k.
func ColumnSubstitute(expr Expression, schema *Schema, newExprs []Expression) Expression {
_, resExpr := ColumnSubstituteImpl(expr, schema, newExprs)
_, _, resExpr := ColumnSubstituteImpl(expr, schema, newExprs, false)
return resExpr
}

// ColumnSubstituteAll substitutes the columns just like ColumnSubstitute, but we don't accept partial substitution.
// Only accept:
//
// 1: substitute them all once find col in schema.
// 2: nothing in expr can be substituted.
func ColumnSubstituteAll(expr Expression, schema *Schema, newExprs []Expression) (bool, Expression) {
_, hasFail, resExpr := ColumnSubstituteImpl(expr, schema, newExprs, true)
return hasFail, resExpr
}

// ColumnSubstituteImpl tries to substitute column expr using newExprs,
// the newFunctionInternal is only called if its child is substituted
func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression) (bool, Expression) {
// @return bool means whether the expr has changed.
// @return bool means whether the expr should change (has the dependency in schema, while the corresponding expr has some compatibility), but finally fallback.
// @return Expression, the original expr or the changed expr, it depends on the first @return bool.
func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression, fail1Return bool) (bool, bool, Expression) {
switch v := expr.(type) {
case *Column:
id := schema.ColumnIndex(v)
if id == -1 {
return false, v
return false, false, v
}
newExpr := newExprs[id]
if v.InOperand {
newExpr = setExprColumnInOperand(newExpr)
newExpr = SetExprColumnInOperand(newExpr)
}
newExpr.SetCoercibility(v.Coercibility())
return true, newExpr
return true, false, newExpr
case *ScalarFunction:
substituted := false
hasFail := false
if v.FuncName.L == ast.Cast {
newFunc := v.Clone().(*ScalarFunction)
substituted, newFunc.GetArgs()[0] = ColumnSubstituteImpl(newFunc.GetArgs()[0], schema, newExprs)
substituted, hasFail, newFunc.GetArgs()[0] = ColumnSubstituteImpl(newFunc.GetArgs()[0], schema, newExprs, fail1Return)
if fail1Return && hasFail {
return substituted, hasFail, newFunc
}
if substituted {
// Workaround for issue https://github.com/pingcap/tidb/issues/28804
e := NewFunctionInternal(v.GetCtx(), v.FuncName.L, v.RetType, newFunc.GetArgs()...)
e.SetCoercibility(v.Coercibility())
return true, e
return true, false, e
}
return false, newFunc
return false, false, newFunc
}
// cowExprRef is a copy-on-write util, args array allocation happens only
// when expr in args is changed
refExprArr := cowExprRef{v.GetArgs(), nil}
_, coll := DeriveCollationFromExprs(v.GetCtx(), v.GetArgs()...)
for idx, arg := range v.GetArgs() {
changed, newFuncExpr := ColumnSubstituteImpl(arg, schema, newExprs)
changed, hasFail, newFuncExpr := ColumnSubstituteImpl(arg, schema, newExprs, fail1Return)
if fail1Return && hasFail {
return changed, hasFail, v
}
oldChanged := changed
if collate.NewCollationEnabled() {
// Make sure the collation used by the ScalarFunction isn't changed and its result collation is not weaker than the collation used by the ScalarFunction.
if changed {
Expand All @@ -434,16 +456,24 @@ func ColumnSubstituteImpl(expr Expression, schema *Schema, newExprs []Expression
}
}
}
if fail1Return && oldChanged != changed {
// Only when the oldChanged is true and changed is false, we will get here.
// And this means there some dependency in this arg can be substituted with
// given expressions, while it has some collation compatibility, finally we
// fall back to use the origin args. (commonly used in projection elimination
// in which fallback usage is unacceptable)
return changed, true, v
}
refExprArr.Set(idx, changed, newFuncExpr)
if changed {
substituted = true
}
}
if substituted {
return true, NewFunctionInternal(v.GetCtx(), v.FuncName.L, v.RetType, refExprArr.Result()...)
return true, false, NewFunctionInternal(v.GetCtx(), v.FuncName.L, v.RetType, refExprArr.Result()...)
}
}
return false, expr
return false, false, expr
}

// checkCollationStrictness check collation strictness-ship between `coll` and `newFuncColl`
Expand Down
Loading

0 comments on commit 64acda6

Please sign in to comment.