Skip to content

Commit

Permalink
*: refine CharsetAndCollation (pingcap#29561)
Browse files Browse the repository at this point in the history
  • Loading branch information
wjhuang2016 authored Nov 12, 2021
1 parent 5bece67 commit bdfd3f6
Show file tree
Hide file tree
Showing 19 changed files with 38 additions and 37 deletions.
5 changes: 2 additions & 3 deletions bindinfo/capture_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -332,9 +332,8 @@ func TestCapturedBindingCharset(t *testing.T) {
require.Len(t, rows, 1)
require.Equal(t, "update `test` . `t` set `name` = ? where `name` <= ?", rows[0][0])
require.Equal(t, "UPDATE /*+ use_index(@`upd_1` `test`.`t` `idx`)*/ `test`.`t` SET `name`='hello' WHERE `name` <= 'abc'", rows[0][1])
// Charset and Collation are empty now, they are not used currently.
require.Equal(t, "", rows[0][6])
require.Equal(t, "", rows[0][7])
require.Equal(t, "utf8mb4", rows[0][6])
require.Equal(t, "utf8mb4_bin", rows[0][7])
}

func TestConcurrentCapture(t *testing.T) {
Expand Down
2 changes: 1 addition & 1 deletion executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1264,7 +1264,7 @@ func (b *executorBuilder) buildHashJoin(v *plannercore.PhysicalHashJoin) Executo

// consider collations
for i := range v.EqualConditions {
chs, coll := v.EqualConditions[i].CharsetAndCollation(e.ctx)
chs, coll := v.EqualConditions[i].CharsetAndCollation()
leftTypes[i].Charset, leftTypes[i].Collate = chs, coll
rightTypes[i].Charset, rightTypes[i].Collate = chs, coll
}
Expand Down
2 changes: 1 addition & 1 deletion expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -1931,7 +1931,7 @@ func WrapWithCastAsString(ctx sessionctx.Context, expr Expression) Expression {
}
tp := types.NewFieldType(mysql.TypeVarString)
if expr.Coercibility() == CoercibilityExplicit {
tp.Charset, tp.Collate = expr.CharsetAndCollation(ctx)
tp.Charset, tp.Collate = expr.CharsetAndCollation()
} else {
tp.Charset, tp.Collate = ctx.GetSessionVars().GetCharsetInfo()
}
Expand Down
18 changes: 4 additions & 14 deletions expression/collation.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,17 +68,7 @@ func (c *collationInfo) SetCharsetAndCollation(chs, coll string) {
c.charset, c.collation = chs, coll
}

func (c *collationInfo) CharsetAndCollation(ctx sessionctx.Context) (string, string) {
if c.charset != "" || c.collation != "" {
return c.charset, c.collation
}

if ctx != nil && ctx.GetSessionVars() != nil {
c.charset, c.collation = ctx.GetSessionVars().GetCharsetInfo()
}
if c.charset == "" || c.collation == "" {
c.charset, c.collation = charset.GetDefaultCharsetAndCollate()
}
func (c *collationInfo) CharsetAndCollation() (string, string) {
return c.charset, c.collation
}

Expand All @@ -99,10 +89,10 @@ type CollationInfo interface {
// SetRepertoire sets a specified repertoire for this expression.
SetRepertoire(r Repertoire)

// CharsetAndCollation ...
CharsetAndCollation(ctx sessionctx.Context) (string, string)
// CharsetAndCollation gets charset and collation.
CharsetAndCollation() (string, string)

// SetCharsetAndCollation ...
// SetCharsetAndCollation sets charset and collation.
SetCharsetAndCollation(chs, coll string)
}

Expand Down
2 changes: 1 addition & 1 deletion expression/constant_propagation.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,7 +152,7 @@ func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Exp
}
for idx, expr := range sf.GetArgs() {
if src.Equal(nil, expr) {
_, coll := cond.CharsetAndCollation(ctx)
_, coll := cond.CharsetAndCollation()
if tgt.GetType().Collate != coll {
continue
}
Expand Down
2 changes: 1 addition & 1 deletion expression/distsql_builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -967,7 +967,7 @@ func getSignatureByPB(ctx sessionctx.Context, sigCode tipb.ScalarFuncSig, tp *ti
case tipb.ScalarFuncSig_HexStrArg:
chs, args := "utf-8", base.getArgs()
if len(args) == 1 {
chs, _ = args[0].CharsetAndCollation(ctx)
chs, _ = args[0].CharsetAndCollation()
}
f = &builtinHexStrArgSig{base, charset.NewEncoding(chs)}
case tipb.ScalarFuncSig_InsertUTF8:
Expand Down
2 changes: 1 addition & 1 deletion expression/expr_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -272,7 +272,7 @@ func (pc PbConverter) scalarFuncToPBExpr(expr *ScalarFunction) *tipb.Expr {
// put collation information into the RetType enforcedly and push it down to TiKV/MockTiKV
tp := *expr.RetType
if collate.NewCollationEnabled() {
_, tp.Collate = expr.CharsetAndCollation(expr.GetCtx())
_, tp.Collate = expr.CharsetAndCollation()
}

// Construct expression ProtoBuf.
Expand Down
2 changes: 1 addition & 1 deletion expression/expr_to_pb_serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,7 +50,7 @@ func TestPushCollationDown(t *testing.T) {
require.NoError(t, err)
expr, err := PBToExpr(pbExpr[0], tps, sc)
require.NoError(t, err)
_, eColl := expr.CharsetAndCollation(nil)
_, eColl := expr.CharsetAndCollation()
require.Equal(t, coll, eColl)
}
}
Expand Down
10 changes: 5 additions & 5 deletions expression/scalar_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -280,7 +280,7 @@ func (sf *ScalarFunction) Clone() Expression {
Function: sf.Function.Clone(),
hashcode: sf.hashcode,
}
c.SetCharsetAndCollation(sf.CharsetAndCollation(sf.GetCtx()))
c.SetCharsetAndCollation(sf.CharsetAndCollation())
c.SetCoercibility(sf.Coercibility())
return c
}
Expand Down Expand Up @@ -552,12 +552,12 @@ func (sf *ScalarFunction) SetCoercibility(val Coercibility) {
sf.Function.SetCoercibility(val)
}

// CharsetAndCollation ...
func (sf *ScalarFunction) CharsetAndCollation(ctx sessionctx.Context) (string, string) {
return sf.Function.CharsetAndCollation(ctx)
// CharsetAndCollation gets charset and collation.
func (sf *ScalarFunction) CharsetAndCollation() (string, string) {
return sf.Function.CharsetAndCollation()
}

// SetCharsetAndCollation ...
// SetCharsetAndCollation sets charset and collation.
func (sf *ScalarFunction) SetCharsetAndCollation(chs, coll string) {
sf.Function.SetCharsetAndCollation(chs, coll)
}
Expand Down
2 changes: 1 addition & 1 deletion expression/util_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -608,7 +608,7 @@ func (m *MockExpr) SetCoercibility(Coercibility)
func (m *MockExpr) Repertoire() Repertoire { return UNICODE }
func (m *MockExpr) SetRepertoire(Repertoire) {}

func (m *MockExpr) CharsetAndCollation(ctx sessionctx.Context) (string, string) {
func (m *MockExpr) CharsetAndCollation() (string, string) {
return "", ""
}
func (m *MockExpr) SetCharsetAndCollation(chs, coll string) {}
2 changes: 1 addition & 1 deletion planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -179,7 +179,7 @@ func (p *LogicalJoin) GetJoinKeys() (leftKeys, rightKeys []*expression.Column, i
// the join keys of EqualConditions
func (p *LogicalJoin) GetPotentialPartitionKeys() (leftKeys, rightKeys []*property.MPPPartitionColumn) {
for _, expr := range p.EqualConditions {
_, coll := expr.CharsetAndCollation(p.ctx)
_, coll := expr.CharsetAndCollation()
collateID := property.GetCollateIDByNameForPartition(coll)
leftKeys = append(leftKeys, &property.MPPPartitionColumn{Col: expr.GetArgs()[0].(*expression.Column), CollateID: collateID})
rightKeys = append(rightKeys, &property.MPPPartitionColumn{Col: expr.GetArgs()[1].(*expression.Column), CollateID: collateID})
Expand Down
2 changes: 1 addition & 1 deletion planner/core/plan_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -417,7 +417,7 @@ func (p *PhysicalHashJoin) ToPB(ctx sessionctx.Context, storeType kv.StoreType)
buildFiledTypes := make([]*tipb.FieldType, 0, len(p.EqualConditions))
for _, equalCondition := range p.EqualConditions {
retType := equalCondition.RetType.Clone()
chs, coll := equalCondition.CharsetAndCollation(ctx)
chs, coll := equalCondition.CharsetAndCollation()
retType.Charset = chs
retType.Collate = coll
probeFiledTypes = append(probeFiledTypes, expression.ToPBFieldType(retType))
Expand Down
2 changes: 1 addition & 1 deletion planner/core/rule_partition_processor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1534,7 +1534,7 @@ func (p *rangeColumnsPruner) pruneUseBinarySearch(sctx sessionctx.Context, op st
}
var expr expression.Expression
expr, err = expression.NewFunctionBase(sctx, op, types.NewFieldType(mysql.TypeLonglong), p.data[ith], v)
expr.SetCharsetAndCollation(f.CharsetAndCollation(sctx))
expr.SetCharsetAndCollation(f.CharsetAndCollation())
var val int64
val, isNull, err = expr.EvalInt(sctx, chunk.Row{})
return val > 0
Expand Down
2 changes: 1 addition & 1 deletion planner/util/path.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func isColEqCorColOrConstant(ctx sessionctx.Context, filter expression.Expressio
if !ok || f.FuncName.L != ast.EQ {
return false
}
_, collation := f.CharsetAndCollation(ctx)
_, collation := f.CharsetAndCollation()
if c, ok := f.GetArgs()[0].(*expression.Column); ok {
if c.RetType.EvalType() == types.ETString && !collate.CompatibleCollate(collation, c.RetType.Collate) {
return false
Expand Down
2 changes: 2 additions & 0 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/parser/ast"
"github.com/pingcap/tidb/parser/auth"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/model"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/parser/terror"
Expand Down Expand Up @@ -1258,6 +1259,7 @@ func NewSessionVars() *SessionVars {
if !EnableLocalTxn.Load() {
vars.TxnScope = kv.NewGlobalTxnScopeVar()
}
vars.systems[CharacterSetConnection], vars.systems[CollationConnection] = charset.GetDefaultCharsetAndCollate()
return vars
}

Expand Down
10 changes: 10 additions & 0 deletions sessionctx/variable/sysvar_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -807,3 +807,13 @@ func TestDDLWorkers(t *testing.T) {
require.NoError(t, err)
require.Equal(t, val, "100") // unchanged
}

func TestDefaultCharsetAndCollation(t *testing.T) {
vars := NewSessionVars()
val, err := GetSessionOrGlobalSystemVar(vars, CharacterSetConnection)
require.NoError(t, err)
require.Equal(t, val, mysql.DefaultCharset)
val, err = GetSessionOrGlobalSystemVar(vars, CollationConnection)
require.NoError(t, err)
require.Equal(t, val, mysql.DefaultCollationName)
}
4 changes: 2 additions & 2 deletions util/ranger/checker.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ func (c *conditionChecker) check(condition expression.Expression) bool {
}

func (c *conditionChecker) checkScalarFunction(scalar *expression.ScalarFunction) bool {
_, collation := scalar.CharsetAndCollation(scalar.GetCtx())
_, collation := scalar.CharsetAndCollation()
switch scalar.FuncName.L {
case ast.LogicOr, ast.LogicAnd:
return c.check(scalar.GetArgs()[0]) && c.check(scalar.GetArgs()[1])
Expand Down Expand Up @@ -111,7 +111,7 @@ func (c *conditionChecker) checkScalarFunction(scalar *expression.ScalarFunction
}

func (c *conditionChecker) checkLikeFunc(scalar *expression.ScalarFunction) bool {
_, collation := scalar.CharsetAndCollation(scalar.GetCtx())
_, collation := scalar.CharsetAndCollation()
if !collate.CompatibleCollate(scalar.GetArgs()[0].GetType().Collate, collation) {
return false
}
Expand Down
2 changes: 1 addition & 1 deletion util/ranger/detacher.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ func getPotentialEqOrInColOffset(expr expression.Expression, cols []*expression.
if !ok {
return -1
}
_, collation := expr.CharsetAndCollation(f.GetCtx())
_, collation := expr.CharsetAndCollation()
switch f.FuncName.L {
case ast.LogicOr:
dnfItems := expression.FlattenDNFConditions(f)
Expand Down
2 changes: 1 addition & 1 deletion util/ranger/points.go
Original file line number Diff line number Diff line change
Expand Up @@ -627,7 +627,7 @@ func (r *builder) buildFromIn(expr *expression.ScalarFunction) ([]*point, bool)
}

func (r *builder) newBuildFromPatternLike(expr *expression.ScalarFunction) []*point {
_, collation := expr.CharsetAndCollation(expr.GetCtx())
_, collation := expr.CharsetAndCollation()
if !collate.CompatibleCollate(expr.GetArgs()[0].GetType().Collate, collation) {
return getFullRange()
}
Expand Down

0 comments on commit bdfd3f6

Please sign in to comment.