Skip to content

Commit

Permalink
*: revert pingcap#10124 and remove sql mode PadCharToFullLength (pi…
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored and XiaTianliang committed Dec 21, 2019
1 parent 5b3fb73 commit f3a7af9
Show file tree
Hide file tree
Showing 14 changed files with 43 additions and 320 deletions.
1 change: 0 additions & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1591,7 +1591,6 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc.Priority = opts.Priority
sc.NotFillCache = !opts.SQLCache
}
sc.PadCharToFullLength = ctx.GetSessionVars().SQLMode.HasPadCharToFullLengthMode()
sc.CastStrToIntStrict = true
case *ast.ShowStmt:
sc.IgnoreTruncate = true
Expand Down
8 changes: 6 additions & 2 deletions executor/point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/ranger"
)

func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor {
Expand Down Expand Up @@ -185,8 +184,13 @@ func encodeIndexKey(e *baseExecutor, tblInfo *model.TableInfo, idxInfo *model.In
sc := e.ctx.GetSessionVars().StmtCtx
for i := range idxVals {
colInfo := tblInfo.Columns[idxInfo.Columns[i].Offset]
// table.CastValue will append 0x0 if the string value's length is smaller than the BINARY column's length.
// So we don't use CastValue for string value for now.
// TODO: merge two if branch.
if colInfo.Tp == mysql.TypeString || colInfo.Tp == mysql.TypeVarString || colInfo.Tp == mysql.TypeVarchar {
idxVals[i], err = ranger.HandlePadCharToFullLength(sc, &colInfo.FieldType, idxVals[i])
var str string
str, err = idxVals[i].ToString()
idxVals[i].SetString(str)
} else {
idxVals[i], err = table.CastValue(e.ctx, idxVals[i], colInfo)
}
Expand Down
180 changes: 25 additions & 155 deletions executor/point_get_test.go

Large diffs are not rendered by default.

12 changes: 1 addition & 11 deletions expression/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -106,10 +106,6 @@ func (col *CorrelatedColumn) EvalString(ctx sessionctx.Context, row chunk.Row) (
return "", true, nil
}
res, err := col.Data.ToString()
resLen := len([]rune(res))
if resLen < col.RetType.Flen && ctx.GetSessionVars().StmtCtx.PadCharToFullLength {
res = res + strings.Repeat(" ", col.RetType.Flen-resLen)
}
return res, err != nil, err
}

Expand Down Expand Up @@ -268,7 +264,7 @@ func (col *Column) VecEvalReal(ctx sessionctx.Context, input *chunk.Chunk, resul

// VecEvalString evaluates this expression in a vectorized manner.
func (col *Column) VecEvalString(ctx sessionctx.Context, input *chunk.Chunk, result *chunk.Column) error {
if col.RetType.Hybrid() || ctx.GetSessionVars().StmtCtx.PadCharToFullLength {
if col.RetType.Hybrid() {
it := chunk.NewIterator4Chunk(input)
result.ReserveString(input.NumRows())
for row := it.Begin(); row != it.End(); row = it.Next() {
Expand Down Expand Up @@ -380,12 +376,6 @@ func (col *Column) EvalString(ctx sessionctx.Context, row chunk.Row) (string, bo
}

val := row.GetString(col.Index)
if ctx.GetSessionVars().StmtCtx.PadCharToFullLength && col.GetType().Tp == mysql.TypeString {
valLen := len([]rune(val))
if valLen < col.RetType.Flen {
val = val + strings.Repeat(" ", col.RetType.Flen-valLen)
}
}
return val, false, nil
}

Expand Down
24 changes: 0 additions & 24 deletions expression/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -229,27 +229,3 @@ func (s *testEvaluatorSuite) TestColHybird(c *C) {
c.Assert(v, Equals, result.GetString(i))
}
}

func (s *testEvaluatorSuite) TestPadCharToFullLength(c *C) {
ctx := mock.NewContext()
ctx.GetSessionVars().StmtCtx.PadCharToFullLength = true

ft := types.NewFieldType(mysql.TypeString)
ft.Flen = 10
col := &Column{RetType: ft, Index: 0}
input := chunk.New([]*types.FieldType{ft}, 1024, 1024)
for i := 0; i < 1024; i++ {
input.AppendString(0, "xy")
}
result, err := newBuffer(types.ETString, 1024)
c.Assert(err, IsNil)
c.Assert(col.VecEvalString(ctx, input, result), IsNil)

it := chunk.NewIterator4Chunk(input)
for row, i := it.Begin(), 0; row != it.End(); row, i = it.Next(), i+1 {
v, _, err := col.EvalString(ctx, row)
c.Assert(err, IsNil)
c.Assert(len(v), Equals, ft.Flen)
c.Assert(v, Equals, result.GetString(i))
}
}
5 changes: 2 additions & 3 deletions planner/core/task.go
Original file line number Diff line number Diff line change
Expand Up @@ -410,14 +410,13 @@ func (p *PhysicalIndexJoin) GetCost(outerTask, innerTask task) float64 {
}

func (p *PhysicalHashJoin) avgRowSize(inner PhysicalPlan) (size float64) {
padChar := p.ctx.GetSessionVars().StmtCtx.PadCharToFullLength
if inner.statsInfo().HistColl != nil {
size = inner.statsInfo().HistColl.GetAvgRowSizeListInDisk(inner.Schema().Columns, padChar)
size = inner.statsInfo().HistColl.GetAvgRowSizeListInDisk(inner.Schema().Columns)
} else {
// Estimate using just the type info.
cols := inner.Schema().Columns
for _, col := range cols {
size += float64(chunk.EstimateTypeWidth(padChar, col.GetType()))
size += float64(chunk.EstimateTypeWidth(col.GetType()))
}
}
return
Expand Down
5 changes: 0 additions & 5 deletions sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,6 @@ type StatementContext struct {
OverflowAsWarning bool
InShowWarning bool
UseCache bool
PadCharToFullLength bool
BatchCheck bool
InNullRejectCheck bool
AllowInvalidDate bool
Expand Down Expand Up @@ -502,9 +501,6 @@ func (sc *StatementContext) PushDownFlags() uint64 {
if sc.DividedByZeroAsWarning {
flags |= model.FlagDividedByZeroAsWarning
}
if sc.PadCharToFullLength {
flags |= model.FlagPadCharToFullLength
}
if sc.InLoadDataStmt {
flags |= model.FlagInLoadDataStmt
}
Expand Down Expand Up @@ -590,7 +586,6 @@ func (sc *StatementContext) CopTasksDetails() *CopTasksDetails {
func (sc *StatementContext) SetFlagsFromPBFlag(flags uint64) {
sc.IgnoreTruncate = (flags & model.FlagIgnoreTruncate) > 0
sc.TruncateAsWarning = (flags & model.FlagTruncateAsWarning) > 0
sc.PadCharToFullLength = (flags & model.FlagPadCharToFullLength) > 0
sc.InInsertStmt = (flags & model.FlagInInsertStmt) > 0
sc.InSelectStmt = (flags & model.FlagInSelectStmt) > 0
sc.OverflowAsWarning = (flags & model.FlagOverflowAsWarning) > 0
Expand Down
1 change: 0 additions & 1 deletion sessionctx/stmtctx/stmtctx_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -84,7 +84,6 @@ func (s *stmtctxSuit) TestStatementContextPushDownFLags(c *C) {
{&stmtctx.StatementContext{OverflowAsWarning: true}, 64},
{&stmtctx.StatementContext{IgnoreZeroInDate: true}, 128},
{&stmtctx.StatementContext{DividedByZeroAsWarning: true}, 256},
{&stmtctx.StatementContext{PadCharToFullLength: true}, 4},
{&stmtctx.StatementContext{InLoadDataStmt: true}, 1024},
{&stmtctx.StatementContext{InSelectStmt: true, TruncateAsWarning: true}, 34},
{&stmtctx.StatementContext{DividedByZeroAsWarning: true, IgnoreTruncate: true}, 257},
Expand Down
6 changes: 3 additions & 3 deletions statistics/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -696,18 +696,18 @@ func (coll *HistColl) GetAvgRowSize(cols []*expression.Column, isEncodedKey bool
}

// GetAvgRowSizeListInDisk computes average row size for given columns.
func (coll *HistColl) GetAvgRowSizeListInDisk(cols []*expression.Column, padChar bool) (size float64) {
func (coll *HistColl) GetAvgRowSizeListInDisk(cols []*expression.Column) (size float64) {
if coll.Pseudo || len(coll.Columns) == 0 || coll.Count == 0 {
for _, col := range cols {
size += float64(chunk.EstimateTypeWidth(padChar, col.GetType()))
size += float64(chunk.EstimateTypeWidth(col.GetType()))
}
} else {
for _, col := range cols {
colHist, ok := coll.Columns[col.UniqueID]
// Normally this would not happen, it is for compatibility with old version stats which
// does not include TotColSize.
if !ok || (!colHist.IsHandle && colHist.TotColSize == 0 && (colHist.NullCount != coll.Count)) {
size += float64(chunk.EstimateTypeWidth(padChar, col.GetType()))
size += float64(chunk.EstimateTypeWidth(col.GetType()))
continue
}
size += colHist.AvgColSizeListInDisk(coll.Count)
Expand Down
1 change: 0 additions & 1 deletion store/mockstore/mocktikv/cop_handler_dag.go
Original file line number Diff line number Diff line change
Expand Up @@ -426,7 +426,6 @@ func flagsToStatementContext(flags uint64) *stmtctx.StatementContext {
sc := new(stmtctx.StatementContext)
sc.IgnoreTruncate = (flags & model.FlagIgnoreTruncate) > 0
sc.TruncateAsWarning = (flags & model.FlagTruncateAsWarning) > 0
sc.PadCharToFullLength = (flags & model.FlagPadCharToFullLength) > 0
sc.InInsertStmt = (flags & model.FlagInInsertStmt) > 0
sc.InSelectStmt = (flags & model.FlagInSelectStmt) > 0
sc.OverflowAsWarning = (flags & model.FlagOverflowAsWarning) > 0
Expand Down
32 changes: 0 additions & 32 deletions types/const_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -255,38 +255,6 @@ func (s *testMySQLConstSuite) TestIgnoreSpaceMode(c *C) {

}

func (s *testMySQLConstSuite) TestPadCharToFullLengthMode(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
// test type `CHAR(n)`
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (a char(10));")
tk.MustExec("insert into t1 values ('xy');")
tk.MustExec("set sql_mode='';")
r := tk.MustQuery(`SELECT a='xy ', char_length(a) FROM t1;`)
r.Check(testkit.Rows("0 2"))
r = tk.MustQuery(`SELECT count(*) FROM t1 WHERE a='xy ';`)
r.Check(testkit.Rows("0"))
tk.MustExec("set sql_mode='PAD_CHAR_TO_FULL_LENGTH';")
r = tk.MustQuery(`SELECT a='xy ', char_length(a) FROM t1;`)
r.Check(testkit.Rows("1 10"))
r = tk.MustQuery(`SELECT count(*) FROM t1 WHERE a='xy ';`)
r.Check(testkit.Rows("1"))

// test type `VARCHAR(n)`
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (a varchar(10));")
tk.MustExec("insert into t1 values ('xy');")
tk.MustExec("set sql_mode='';")
r = tk.MustQuery(`SELECT a='xy ', char_length(a) FROM t1;`)
r.Check(testkit.Rows("0 2"))
r = tk.MustQuery(`SELECT count(*) FROM t1 WHERE a='xy ';`)
r.Check(testkit.Rows("0"))
tk.MustExec("set sql_mode='PAD_CHAR_TO_FULL_LENGTH';")
r = tk.MustQuery(`SELECT a='xy ', char_length(a) FROM t1;`)
r.Check(testkit.Rows("0 2"))
}

func (s *testMySQLConstSuite) TestNoBackslashEscapesMode(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("set sql_mode=''")
Expand Down
9 changes: 1 addition & 8 deletions util/chunk/codec.go
Original file line number Diff line number Diff line change
Expand Up @@ -195,7 +195,7 @@ func GetFixedLen(colType *types.FieldType) int {
// it's OK (and expected) to guess if we don't know for sure.
//
// mostly study from https://github.com/postgres/postgres/blob/REL_12_STABLE/src/backend/utils/cache/lsyscache.c#L2356
func EstimateTypeWidth(padChar bool, colType *types.FieldType) int {
func EstimateTypeWidth(colType *types.FieldType) int {
colLen := getFixedLen(colType)
// Easy if it's a fixed-width type
if colLen != varElemLen {
Expand All @@ -204,13 +204,6 @@ func EstimateTypeWidth(padChar bool, colType *types.FieldType) int {

colLen = colType.Flen
if colLen > 0 {
/*
* If PAD_CHAR_TO_FULL_LENGTH is enabled, and type is CHAR,
* the colType.Flen is also the only width.
*/
if padChar && colType.Tp == mysql.TypeString {
return colLen
}
if colLen <= 32 {
return colLen
}
Expand Down
13 changes: 5 additions & 8 deletions util/chunk/codec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,22 +81,19 @@ func (s *testCodecSuite) TestEstimateTypeWidth(c *check.C) {
var colType *types.FieldType

colType = &types.FieldType{Tp: mysql.TypeLonglong}
c.Assert(EstimateTypeWidth(false, colType), check.Equals, 8) // fixed-witch type

colType = &types.FieldType{Tp: mysql.TypeString, Flen: 100000}
c.Assert(EstimateTypeWidth(true, colType), check.Equals, 100000) // PAD_CHAR_TO_FULL_LENGTH
c.Assert(EstimateTypeWidth(colType), check.Equals, 8) // fixed-witch type

colType = &types.FieldType{Tp: mysql.TypeString, Flen: 31}
c.Assert(EstimateTypeWidth(false, colType), check.Equals, 31) // colLen <= 32
c.Assert(EstimateTypeWidth(colType), check.Equals, 31) // colLen <= 32

colType = &types.FieldType{Tp: mysql.TypeString, Flen: 999}
c.Assert(EstimateTypeWidth(false, colType), check.Equals, 515) // colLen < 1000
c.Assert(EstimateTypeWidth(colType), check.Equals, 515) // colLen < 1000

colType = &types.FieldType{Tp: mysql.TypeString, Flen: 2000}
c.Assert(EstimateTypeWidth(false, colType), check.Equals, 516) // colLen < 1000
c.Assert(EstimateTypeWidth(colType), check.Equals, 516) // colLen < 1000

colType = &types.FieldType{Tp: mysql.TypeString}
c.Assert(EstimateTypeWidth(false, colType), check.Equals, 32) // value after guessing
c.Assert(EstimateTypeWidth(colType), check.Equals, 32) // value after guessing
}

func BenchmarkEncodeChunk(b *testing.B) {
Expand Down
66 changes: 0 additions & 66 deletions util/ranger/points.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,12 @@ import (
"fmt"
"math"
"sort"
"strings"

"github.com/pingcap/errors"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/charset"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -240,11 +237,6 @@ func (r *builder) buildFormBinOp(expr *expression.ScalarFunction) []point {
return nil
}

value, err = HandlePadCharToFullLength(r.sc, ft, value)
if err != nil {
return nil
}

value, op, isValidRange := handleUnsignedIntCol(ft, value, op)
if !isValidRange {
return nil
Expand Down Expand Up @@ -281,64 +273,6 @@ func (r *builder) buildFormBinOp(expr *expression.ScalarFunction) []point {
return nil
}

// HandlePadCharToFullLength handles the "PAD_CHAR_TO_FULL_LENGTH" sql mode for
// CHAR[N] index columns.
// NOTE: kv.ErrNotExist is returned to indicate that this value can not match
// any (key, value) pair in tikv storage. This error should be handled by
// the caller.
func HandlePadCharToFullLength(sc *stmtctx.StatementContext, ft *types.FieldType, val types.Datum) (types.Datum, error) {
isChar := (ft.Tp == mysql.TypeString)
isBinary := (isChar && ft.Collate == charset.CollationBin)
isVarchar := (ft.Tp == mysql.TypeVarString || ft.Tp == mysql.TypeVarchar)
isVarBinary := (isVarchar && ft.Collate == charset.CollationBin)

if !isChar && !isVarchar && !isBinary && !isVarBinary {
return val, nil
}

hasBinaryFlag := mysql.HasBinaryFlag(ft.Flag)
targetStr, err := val.ToString()
if err != nil {
return val, err
}

switch {
case isBinary || isVarBinary:
val.SetString(targetStr)
return val, nil
case isVarchar && hasBinaryFlag:
noTrailingSpace := strings.TrimRight(targetStr, " ")
if numSpacesToFill := ft.Flen - len(noTrailingSpace); numSpacesToFill > 0 {
noTrailingSpace += strings.Repeat(" ", numSpacesToFill)
}
val.SetString(noTrailingSpace)
return val, nil
case isVarchar && !hasBinaryFlag:
val.SetString(targetStr)
return val, nil
case isChar && hasBinaryFlag:
noTrailingSpace := strings.TrimRight(targetStr, " ")
val.SetString(noTrailingSpace)
return val, nil
case isChar && !hasBinaryFlag && !sc.PadCharToFullLength:
val.SetString(targetStr)
return val, nil
case isChar && !hasBinaryFlag && sc.PadCharToFullLength:
if len(targetStr) != ft.Flen {
// return kv.ErrNotExist to indicate that this value can not match any
// (key, value) pair in tikv storage.
return val, kv.ErrNotExist
}
// Trailing spaces of data typed "CHAR[N]" is trimed in the storage, we
// need to trim these trailing spaces as well.
noTrailingSpace := strings.TrimRight(targetStr, " ")
val.SetString(noTrailingSpace)
return val, nil
default:
return val, nil
}
}

// handleUnsignedIntCol handles the case when unsigned column meets negative integer value.
// The three returned values are: fixed constant value, fixed operator, and a boolean
// which indicates whether the range is valid or not.
Expand Down

0 comments on commit f3a7af9

Please sign in to comment.