Skip to content

Commit

Permalink
*: revert #10124 and remove sql mode PadCharToFullLength (#14007) (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
winoros authored and sre-bot committed Dec 20, 2019
1 parent 9efbf48 commit b52cccd
Show file tree
Hide file tree
Showing 10 changed files with 29 additions and 272 deletions.
3 changes: 0 additions & 3 deletions executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,9 +154,6 @@ func statementContextToFlags(sc *stmtctx.StatementContext) uint64 {
if sc.DividedByZeroAsWarning {
flags |= model.FlagDividedByZeroAsWarning
}
if sc.PadCharToFullLength {
flags |= model.FlagPadCharToFullLength
}
return flags
}

Expand Down
1 change: 0 additions & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1499,7 +1499,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
5 changes: 3 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 @@ -157,7 +156,9 @@ func (e *PointGetExecutor) encodeIndexKey() (_ []byte, err error) {
for i := range e.idxVals {
colInfo := e.tblInfo.Columns[e.idxInfo.Columns[i].Offset]
if colInfo.Tp == mysql.TypeString || colInfo.Tp == mysql.TypeVarString || colInfo.Tp == mysql.TypeVarchar {
e.idxVals[i], err = ranger.HandlePadCharToFullLength(sc, &colInfo.FieldType, e.idxVals[i])
var str string
str, err = e.idxVals[i].ToString()
e.idxVals[i].SetString(str)
} else {
e.idxVals[i], err = table.CastValue(e.ctx, e.idxVals[i], colInfo)
}
Expand Down
180 changes: 25 additions & 155 deletions executor/point_get_test.go

Large diffs are not rendered by default.

11 changes: 0 additions & 11 deletions expression/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ package expression

import (
"fmt"
"strings"

"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
Expand Down Expand Up @@ -71,10 +70,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 @@ -249,12 +244,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
1 change: 1 addition & 0 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -181,6 +181,7 @@ github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d h1:GoAlyOgbOEIFd
github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
github.com/remyoudompheng/bigfft v0.0.0-20190512091148-babf20351dd7 h1:FUL3b97ZY2EPqg2NbXKuMHs5pXJB9hjj1fDHnF2vl28=
github.com/remyoudompheng/bigfft v0.0.0-20190512091148-babf20351dd7/go.mod h1:qqbHyh8v60DhA7CoWK5oRCqLrMHRGoxYCSS9EjAz6Eo=
github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc=
github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo=
github.com/shirou/gopsutil v2.18.10+incompatible h1:cy84jW6EVRPa5g9HAHrlbxMSIjBhDSX0OFYyMYminYs=
github.com/shirou/gopsutil v2.18.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA=
Expand Down
1 change: 0 additions & 1 deletion sessionctx/stmtctx/stmtctx.go
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,6 @@ type StatementContext struct {
OverflowAsWarning bool
InShowWarning bool
UseCache bool
PadCharToFullLength bool
BatchCheck bool
InNullRejectCheck bool
AllowInvalidDate bool
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 @@ -417,7 +417,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
return sc
}

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
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 @@ -245,11 +242,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 @@ -286,64 +278,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 b52cccd

Please sign in to comment.