Skip to content

Commit

Permalink
*: fix datum's collation behavior to build correct logical ranges (pi…
Browse files Browse the repository at this point in the history
  • Loading branch information
sre-bot authored Mar 5, 2020
1 parent 5dbe82b commit 0ddfe07
Show file tree
Hide file tree
Showing 29 changed files with 210 additions and 76 deletions.
3 changes: 0 additions & 3 deletions cmd/explaintest/config.toml
Original file line number Diff line number Diff line change
Expand Up @@ -2,9 +2,6 @@ port = 4001
lease = "0"
mem-quota-query = 34359738368

[log]
level = "error"

[status]
status-port = 10081

Expand Down
4 changes: 2 additions & 2 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -765,7 +765,7 @@ func setSetDefaultValue(v types.Datum, col *table.Column) (string, error) {
if err != nil {
return "", errors.Trace(err)
}
v.SetMysqlSet(setVal)
v.SetMysqlSet(setVal, col.Collate)
return v.ToString()
}

Expand Down Expand Up @@ -799,7 +799,7 @@ func setSetDefaultValue(v types.Datum, col *table.Column) (string, error) {
if err != nil {
return "", ErrInvalidDefaultValue.GenWithStackByArgs(col.Name.O)
}
v.SetMysqlSet(setVal)
v.SetMysqlSet(setVal, col.Collate)

return v.ToString()
}
Expand Down
4 changes: 2 additions & 2 deletions executor/aggfuncs/aggfunc_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,13 +274,13 @@ func getDataGenFunc(ft *types.FieldType) func(i int) types.Datum {
elems := []string{"a", "b", "c", "d", "e"}
return func(i int) types.Datum {
e, _ := types.ParseEnumValue(elems, uint64(i+1))
return types.NewMysqlEnumDatum(e)
return types.NewCollateMysqlEnumDatum(e, ft.Collate)
}
case mysql.TypeSet:
elems := []string{"a", "b", "c", "d", "e"}
return func(i int) types.Datum {
e, _ := types.ParseSetValue(elems, uint64(i+1))
return types.NewMysqlSetDatum(e)
return types.NewMysqlSetDatum(e, ft.Collate)
}
}
return nil
Expand Down
2 changes: 1 addition & 1 deletion executor/aggfuncs/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -280,7 +280,7 @@ func buildMaxMin(aggFuncDesc *aggregation.AggFuncDesc, ordinal int, isMax bool)
case types.ETDecimal:
return &maxMin4Decimal{base}
case types.ETString:
return &maxMin4String{base}
return &maxMin4String{baseMaxMinAggFunc: base, retTp: aggFuncDesc.RetTp}
case types.ETDatetime, types.ETTimestamp:
return &maxMin4Time{base}
case types.ETDuration:
Expand Down
1 change: 1 addition & 0 deletions executor/aggfuncs/func_max_min.go
Original file line number Diff line number Diff line change
Expand Up @@ -396,6 +396,7 @@ func (e *maxMin4Decimal) MergePartialResult(sctx sessionctx.Context, src, dst Pa

type maxMin4String struct {
baseMaxMinAggFunc
retTp *types.FieldType
}

func (e *maxMin4String) AllocPartialResult() PartialResult {
Expand Down
4 changes: 4 additions & 0 deletions executor/mem_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -173,6 +173,8 @@ func buildMemTableReader(us *UnionScanExec, tblReader *TableReaderExecutor) *mem
Tp: int32(col.Tp),
Flag: int32(col.Flag),
IsPKHandle: us.table.Meta().PKIsHandle && mysql.HasPriKeyFlag(col.Flag),
Collate: col.Collate,
Flen: col.Flen,
})
}

Expand Down Expand Up @@ -423,6 +425,8 @@ func (m *memIndexLookUpReader) getMemRows() ([][]types.Datum, error) {
Tp: int32(col.Tp),
Flag: int32(col.Flag),
IsPKHandle: m.table.Meta().PKIsHandle && mysql.HasPriKeyFlag(col.Flag),
Collate: col.Collate,
Flen: col.Flen,
})
}
rd := rowcodec.NewByteDecoder(colInfos, -1, nil, nil)
Expand Down
20 changes: 17 additions & 3 deletions expression/builtin_encryption_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,13 @@ import (

. "github.com/pingcap/check"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/charset"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/hack"
)

Expand Down Expand Up @@ -146,7 +148,11 @@ func (s *testEvaluatorSuite) TestAESDecrypt(c *C) {
c.Assert(err, IsNil)
str, err := evalBuiltinFunc(f, chunk.Row{})
c.Assert(err, IsNil)
c.Assert(str, DeepEquals, types.NewDatum(tt.origin))
if tt.origin == nil {
c.Assert(str.IsNull(), IsTrue)
continue
}
c.Assert(str, DeepEquals, types.NewCollationStringDatum(tt.origin.(string), charset.CollationBin, collate.DefaultLen))
}
variable.SetSessionSystemVar(s.ctx.GetSessionVars(), variable.BlockEncryptionMode, types.NewDatum("aes-128-ecb"))
s.testNullInput(c, ast.AesDecrypt)
Expand Down Expand Up @@ -373,7 +379,11 @@ func (s *testEvaluatorSuite) TestCompress(c *C) {
c.Assert(err, IsNil, Commentf("%v", test))
out, err := evalBuiltinFunc(f, chunk.Row{})
c.Assert(err, IsNil, Commentf("%v", test))
c.Assert(out, DeepEquals, types.NewDatum(test.expect), Commentf("%v", test))
if test.expect == nil {
c.Assert(out.IsNull(), IsTrue, Commentf("%v", test))
continue
}
c.Assert(out, DeepEquals, types.NewCollationStringDatum(test.expect.(string), charset.CollationBin, collate.DefaultLen), Commentf("%v", test))
}
}

Expand Down Expand Up @@ -403,7 +413,11 @@ func (s *testEvaluatorSuite) TestUncompress(c *C) {
c.Assert(err, IsNil, Commentf("%v", test))
out, err := evalBuiltinFunc(f, chunk.Row{})
c.Assert(err, IsNil, Commentf("%v", test))
c.Assert(out, DeepEquals, types.NewDatum(test.expect), Commentf("%v", test))
if test.expect == nil {
c.Assert(out.IsNull(), IsTrue, Commentf("%v", test))
continue
}
c.Assert(out, DeepEquals, types.NewCollationStringDatum(test.expect.(string), charset.CollationBin, collate.DefaultLen), Commentf("%v", test))
}
}

Expand Down
4 changes: 2 additions & 2 deletions expression/builtin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -55,10 +55,10 @@ func evalBuiltinFunc(f builtinFunc, row chunk.Row) (d types.Datum, err error) {
}

if isNull || err != nil {
d.SetValue(nil)
d.SetNull()
return d, err
}
d.SetValue(res)
d.SetValue(res, f.getRetTp())
return
}

Expand Down
4 changes: 2 additions & 2 deletions expression/constant.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,14 +81,14 @@ func (d *ParamMarker) GetUserVar() types.Datum {
func (c *Constant) String() string {
if c.ParamMarker != nil {
dt := c.ParamMarker.GetUserVar()
c.Value.SetValue(dt.GetValue())
c.Value.SetValue(dt.GetValue(), c.RetType)
} else if c.DeferredExpr != nil {
dt, err := c.Eval(chunk.Row{})
if err != nil {
logutil.BgLogger().Error("eval constant failed", zap.Error(err))
return ""
}
c.Value.SetValue(dt.GetValue())
c.Value.SetValue(dt.GetValue(), c.RetType)
}
return fmt.Sprintf("%v", c.Value.GetValue())
}
Expand Down
4 changes: 2 additions & 2 deletions expression/scalar_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -334,10 +334,10 @@ func (sf *ScalarFunction) Eval(row chunk.Row) (d types.Datum, err error) {
}

if isNull || err != nil {
d.SetValue(nil)
d.SetNull()
return d, err
}
d.SetValue(res)
d.SetValue(res, sf.RetType)
return
}

Expand Down
6 changes: 3 additions & 3 deletions planner/core/exhaust_physical_plans_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) {
})
dsSchema.Append(&expression.Column{
UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(),
RetType: types.NewFieldType(mysql.TypeVarchar),
RetType: types.NewFieldTypeWithCollation(mysql.TypeVarchar, mysql.DefaultCollationName, types.UnspecifiedLength),
})
dsNames = append(dsNames, &types.FieldName{
ColName: model.NewCIStr("c"),
Expand Down Expand Up @@ -104,7 +104,7 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) {
})
outerChildSchema.Append(&expression.Column{
UniqueID: s.ctx.GetSessionVars().AllocPlanColumnID(),
RetType: types.NewFieldType(mysql.TypeVarchar),
RetType: types.NewFieldTypeWithCollation(mysql.TypeVarchar, mysql.DefaultCollationName, types.UnspecifiedLength),
})
outerChildNames = append(outerChildNames, &types.FieldName{
ColName: model.NewCIStr("g"),
Expand Down Expand Up @@ -247,9 +247,9 @@ func (s *testUnitTestSuit) TestIndexJoinAnalyzeLookUpFilters(c *C) {
helper := &indexJoinBuildHelper{join: joinNode, lastColManager: nil}
_, err = helper.analyzeLookUpFilters(path, dataSourceNode, tt.innerKeys)
c.Assert(err, IsNil)
c.Assert(fmt.Sprintf("%v", helper.chosenAccess), Equals, tt.accesses)
c.Assert(fmt.Sprintf("%v", helper.chosenRanges), Equals, tt.ranges, Commentf("test case: #%v", i))
c.Assert(fmt.Sprintf("%v", helper.idxOff2KeyOff), Equals, tt.idxOff2KeyOff)
c.Assert(fmt.Sprintf("%v", helper.chosenAccess), Equals, tt.accesses)
c.Assert(fmt.Sprintf("%v", helper.chosenRemained), Equals, tt.remained)
c.Assert(fmt.Sprintf("%v", helper.lastColManager), Equals, tt.compareFilters)
}
Expand Down
1 change: 1 addition & 0 deletions planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -909,6 +909,7 @@ func (er *expressionRewriter) Leave(originInNode ast.Node) (retNode ast.Node, ok
case *ast.AggregateFuncExpr, *ast.ColumnNameExpr, *ast.ParenthesesExpr, *ast.WhenClause,
*ast.SubqueryExpr, *ast.ExistsSubqueryExpr, *ast.CompareSubqueryExpr, *ast.ValuesExpr, *ast.WindowFuncExpr, *ast.TableNameExpr:
case *driver.ValueExpr:
v.Datum.SetValue(v.Datum.GetValue(), &v.Type)
value := &expression.Constant{Value: v.Datum, RetType: &v.Type}
er.ctxStackAppend(value, types.EmptyName)
case *driver.ParamMarkerExpr:
Expand Down
2 changes: 1 addition & 1 deletion server/util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,7 +208,7 @@ func (s *testUtilSuite) TestDumpTextValue(c *C) {
c.Assert(mustDecodeStr(c, bs), Equals, "ename")

set := types.Datum{}
set.SetMysqlSet(types.Set{Name: "sname", Value: 0})
set.SetMysqlSet(types.Set{Name: "sname", Value: 0}, mysql.DefaultCollationName)
columns[0].Type = mysql.TypeSet
bs, err = dumpTextRow(nil, columns, chunk.MutRowFromDatums([]types.Datum{set}).ToRow())
c.Assert(err, IsNil)
Expand Down
6 changes: 3 additions & 3 deletions table/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -454,7 +454,7 @@ func getColDefaultValueFromNil(ctx sessionctx.Context, col *model.ColumnInfo) (t
if err != nil {
return types.Datum{}, err
}
return types.NewMysqlEnumDatum(defEnum), nil
return types.NewCollateMysqlEnumDatum(defEnum, col.Collate), nil
}
if mysql.HasAutoIncrementFlag(col.Flag) {
// Auto increment column doesn't has default value and we should not return error.
Expand Down Expand Up @@ -510,9 +510,9 @@ func GetZeroValue(col *model.ColumnInfo) types.Datum {
case mysql.TypeBit:
d.SetMysqlBit(types.ZeroBinaryLiteral)
case mysql.TypeSet:
d.SetMysqlSet(types.Set{})
d.SetMysqlSet(types.Set{}, col.Collate)
case mysql.TypeEnum:
d.SetMysqlEnum(types.Enum{})
d.SetMysqlEnum(types.Enum{}, col.Collate)
case mysql.TypeJSON:
d.SetMysqlJSON(json.CreateBinary(nil))
}
Expand Down
7 changes: 4 additions & 3 deletions table/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -338,9 +338,10 @@ func (t *testTableSuite) TestGetDefaultValue(c *C) {
{
&model.ColumnInfo{
FieldType: types.FieldType{
Tp: mysql.TypeEnum,
Flag: mysql.NotNullFlag,
Elems: []string{"abc", "def"},
Tp: mysql.TypeEnum,
Flag: mysql.NotNullFlag,
Elems: []string{"abc", "def"},
Collate: mysql.DefaultCollationName,
},
},
false,
Expand Down
6 changes: 3 additions & 3 deletions tablecodec/tablecodec.go
Original file line number Diff line number Diff line change
Expand Up @@ -530,22 +530,22 @@ func unflatten(datum types.Datum, ft *types.FieldType, loc *time.Location) (type
return datum, nil
case mysql.TypeDuration: //duration should read fsp from column meta data
dur := types.Duration{Duration: time.Duration(datum.GetInt64()), Fsp: int8(ft.Decimal)}
datum.SetValue(dur)
datum.SetMysqlDuration(dur)
return datum, nil
case mysql.TypeEnum:
// ignore error deliberately, to read empty enum value.
enum, err := types.ParseEnumValue(ft.Elems, datum.GetUint64())
if err != nil {
enum = types.Enum{}
}
datum.SetValue(enum)
datum.SetMysqlEnum(enum, ft.Collate)
return datum, nil
case mysql.TypeSet:
set, err := types.ParseSetValue(ft.Elems, datum.GetUint64())
if err != nil {
return datum, errors.Trace(err)
}
datum.SetValue(set)
datum.SetMysqlSet(set, ft.Collate)
return datum, nil
case mysql.TypeBit:
val := datum.GetUint64()
Expand Down
Loading

0 comments on commit 0ddfe07

Please sign in to comment.