Skip to content

Commit 5445e17

Browse files
authored
*: fix in-compatible behavior when modify value from Navicat GUI (#6105)
1 parent 546b5ac commit 5445e17

File tree

10 files changed

+56
-55
lines changed

10 files changed

+56
-55
lines changed

ddl/ddl_api.go

Lines changed: 1 addition & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -336,8 +336,7 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef) (
336336
// Set `NoDefaultValueFlag` if this field doesn't have a default value and
337337
// it is `not null` and not an `AUTO_INCREMENT` field or `TIMESTAMP` field.
338338
setNoDefaultValueFlag(col, hasDefaultValue)
339-
340-
if col.Charset == charset.CharsetBin {
339+
if col.FieldType.EvalType().IsStringKind() && col.Charset == charset.CharsetBin {
341340
col.Flag |= mysql.BinaryFlag
342341
}
343342
if col.Tp == mysql.TypeBit {

expression/builtin_control.go

Lines changed: 13 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -53,10 +53,6 @@ var (
5353
_ builtinFunc = &builtinIfJSONSig{}
5454
)
5555

56-
type caseWhenFunctionClass struct {
57-
baseFunctionClass
58-
}
59-
6056
// Infer result type for builtin IF, IFNULL && NULLIF.
6157
func inferType4ControlFuncs(lhs, rhs *types.FieldType) *types.FieldType {
6258
resultFieldType := &types.FieldType{}
@@ -85,12 +81,12 @@ func inferType4ControlFuncs(lhs, rhs *types.FieldType) *types.FieldType {
8581
}
8682
if types.IsNonBinaryStr(lhs) && !types.IsBinaryStr(rhs) {
8783
resultFieldType.Charset, resultFieldType.Collate, resultFieldType.Flag = charset.CharsetUTF8, charset.CollationUTF8, 0
88-
if mysql.HasBinaryFlag(lhs.Flag) {
84+
if mysql.HasBinaryFlag(lhs.Flag) || !types.IsNonBinaryStr(rhs) {
8985
resultFieldType.Flag |= mysql.BinaryFlag
9086
}
9187
} else if types.IsNonBinaryStr(rhs) && !types.IsBinaryStr(lhs) {
9288
resultFieldType.Charset, resultFieldType.Collate, resultFieldType.Flag = charset.CharsetUTF8, charset.CollationUTF8, 0
93-
if mysql.HasBinaryFlag(rhs.Flag) {
89+
if mysql.HasBinaryFlag(rhs.Flag) || !types.IsNonBinaryStr(lhs) {
9490
resultFieldType.Flag |= mysql.BinaryFlag
9591
}
9692
} else if types.IsBinaryStr(lhs) || types.IsBinaryStr(rhs) || !evalType.IsStringKind() {
@@ -132,25 +128,31 @@ func inferType4ControlFuncs(lhs, rhs *types.FieldType) *types.FieldType {
132128
return resultFieldType
133129
}
134130

131+
type caseWhenFunctionClass struct {
132+
baseFunctionClass
133+
}
134+
135135
func (c *caseWhenFunctionClass) getFunction(ctx sessionctx.Context, args []Expression) (sig builtinFunc, err error) {
136136
if err = c.verifyArgs(args); err != nil {
137137
return nil, errors.Trace(err)
138138
}
139139
l := len(args)
140140
// Fill in each 'THEN' clause parameter type.
141141
fieldTps := make([]*types.FieldType, 0, (l+1)/2)
142-
decimal, flen, isBinaryStr := args[1].GetType().Decimal, 0, false
142+
decimal, flen, isBinaryStr, isBinaryFlag := args[1].GetType().Decimal, 0, false, false
143143
for i := 1; i < l; i += 2 {
144144
fieldTps = append(fieldTps, args[i].GetType())
145145
decimal = mathutil.Max(decimal, args[i].GetType().Decimal)
146146
flen = mathutil.Max(flen, args[i].GetType().Flen)
147147
isBinaryStr = isBinaryStr || types.IsBinaryStr(args[i].GetType())
148+
isBinaryFlag = isBinaryFlag || !types.IsNonBinaryStr(args[i].GetType())
148149
}
149150
if l%2 == 1 {
150151
fieldTps = append(fieldTps, args[l-1].GetType())
151152
decimal = mathutil.Max(decimal, args[l-1].GetType().Decimal)
152153
flen = mathutil.Max(flen, args[l-1].GetType().Flen)
153154
isBinaryStr = isBinaryStr || types.IsBinaryStr(args[l-1].GetType())
155+
isBinaryFlag = isBinaryFlag || !types.IsNonBinaryStr(args[l-1].GetType())
154156
}
155157

156158
fieldTp := types.AggFieldType(fieldTps)
@@ -163,6 +165,9 @@ func (c *caseWhenFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
163165
if fieldTp.EvalType().IsStringKind() && !isBinaryStr {
164166
fieldTp.Charset, fieldTp.Collate = mysql.DefaultCharset, mysql.DefaultCollationName
165167
}
168+
if isBinaryFlag {
169+
fieldTp.Flag |= mysql.BinaryFlag
170+
}
166171
// Set retType to BINARY(0) if all arguments are of type NULL.
167172
if fieldTp.Tp == mysql.TypeNull {
168173
fieldTp.Flen, fieldTp.Decimal = 0, -1
@@ -395,6 +400,7 @@ func (c *ifFunctionClass) getFunction(ctx sessionctx.Context, args []Expression)
395400
retTp := inferType4ControlFuncs(args[1].GetType(), args[2].GetType())
396401
evalTps := retTp.EvalType()
397402
bf := newBaseBuiltinFuncWithTp(ctx, args, evalTps, types.ETInt, evalTps, evalTps)
403+
retTp.Flag |= bf.tp.Flag
398404
bf.tp = retTp
399405
switch evalTps {
400406
case types.ETInt:

expression/builtin_miscellaneous.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -189,6 +189,7 @@ func (c *anyValueFunctionClass) getFunction(ctx sessionctx.Context, args []Expre
189189
}
190190
argTp := args[0].GetType().EvalType()
191191
bf := newBaseBuiltinFuncWithTp(ctx, args, argTp, argTp)
192+
args[0].GetType().Flag |= bf.tp.Flag
192193
*bf.tp = *args[0].GetType()
193194
var sig builtinFunc
194195
switch argTp {

expression/builtin_string.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -167,7 +167,7 @@ func reverseRunes(origin []rune) []rune {
167167
func SetBinFlagOrBinStr(argTp *types.FieldType, resTp *types.FieldType) {
168168
if types.IsBinaryStr(argTp) {
169169
types.SetBinChsClnFlag(resTp)
170-
} else if mysql.HasBinaryFlag(argTp.Flag) {
170+
} else if mysql.HasBinaryFlag(argTp.Flag) || !types.IsNonBinaryStr(argTp) {
171171
resTp.Flag |= mysql.BinaryFlag
172172
}
173173
}

expression/typeinfer_test.go

Lines changed: 27 additions & 33 deletions
Original file line numberDiff line numberDiff line change
@@ -129,7 +129,6 @@ func (s *testInferTypeSuite) TestInferType(c *C) {
129129
tests = append(tests, s.createTestCase4Literals()...)
130130
tests = append(tests, s.createTestCase4JSONFuncs()...)
131131
tests = append(tests, s.createTestCase4MiscellaneousFunc()...)
132-
tests = append(tests, s.createTestCase4AggregationFunc()...)
133132

134133
for _, tt := range tests {
135134
ctx := testKit.Se.(sessionctx.Context)
@@ -205,35 +204,35 @@ func (s *testInferTypeSuite) createTestCase4Columns() []typeInferTestCase {
205204
return []typeInferTestCase{
206205
{"c_bit ", mysql.TypeBit, charset.CharsetBin, mysql.UnsignedFlag, 10, 0},
207206
{"c_year ", mysql.TypeYear, charset.CharsetBin, mysql.UnsignedFlag | mysql.ZerofillFlag, 4, 0},
208-
{"c_int_d ", mysql.TypeLong, charset.CharsetBin, mysql.BinaryFlag, 11, 0},
209-
{"c_uint_d ", mysql.TypeLong, charset.CharsetBin, mysql.BinaryFlag | mysql.UnsignedFlag, 10, 0},
210-
{"c_bigint_d ", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0},
211-
{"c_ubigint_d ", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag | mysql.UnsignedFlag, 20, 0},
212-
{"c_float_d ", mysql.TypeFloat, charset.CharsetBin, mysql.BinaryFlag, 12, types.UnspecifiedLength},
213-
{"c_ufloat_d ", mysql.TypeFloat, charset.CharsetBin, mysql.BinaryFlag | mysql.UnsignedFlag, 12, types.UnspecifiedLength},
214-
{"c_double_d ", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, types.UnspecifiedLength},
215-
{"c_udouble_d ", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag | mysql.UnsignedFlag, 22, types.UnspecifiedLength},
216-
{"c_decimal ", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 6, 3}, // TODO: Flen should be 8
217-
{"c_udecimal ", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag | mysql.UnsignedFlag, 10, 3}, // TODO: Flen should be 11
218-
{"c_decimal_d ", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 11, 0},
219-
{"c_udecimal_d ", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag | mysql.UnsignedFlag, 11, 0}, // TODO: Flen should be 10
207+
{"c_int_d ", mysql.TypeLong, charset.CharsetBin, 0, 11, 0},
208+
{"c_uint_d ", mysql.TypeLong, charset.CharsetBin, mysql.UnsignedFlag, 10, 0},
209+
{"c_bigint_d ", mysql.TypeLonglong, charset.CharsetBin, 0, 20, 0},
210+
{"c_ubigint_d ", mysql.TypeLonglong, charset.CharsetBin, mysql.UnsignedFlag, 20, 0},
211+
{"c_float_d ", mysql.TypeFloat, charset.CharsetBin, 0, 12, types.UnspecifiedLength},
212+
{"c_ufloat_d ", mysql.TypeFloat, charset.CharsetBin, mysql.UnsignedFlag, 12, types.UnspecifiedLength},
213+
{"c_double_d ", mysql.TypeDouble, charset.CharsetBin, 0, 22, types.UnspecifiedLength},
214+
{"c_udouble_d ", mysql.TypeDouble, charset.CharsetBin, mysql.UnsignedFlag, 22, types.UnspecifiedLength},
215+
{"c_decimal ", mysql.TypeNewDecimal, charset.CharsetBin, 0, 6, 3}, // TODO: Flen should be 8
216+
{"c_udecimal ", mysql.TypeNewDecimal, charset.CharsetBin, mysql.UnsignedFlag, 10, 3}, // TODO: Flen should be 11
217+
{"c_decimal_d ", mysql.TypeNewDecimal, charset.CharsetBin, 0, 11, 0},
218+
{"c_udecimal_d ", mysql.TypeNewDecimal, charset.CharsetBin, mysql.UnsignedFlag, 11, 0}, // TODO: Flen should be 10
220219
{"c_datetime ", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 22, 2},
221220
{"c_datetime_d ", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, 19, 0},
222221
{"c_time ", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 14, 3},
223222
{"c_time_d ", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0},
224223
{"c_timestamp ", mysql.TypeTimestamp, charset.CharsetBin, mysql.NotNullFlag | mysql.BinaryFlag | mysql.TimestampFlag, 24, 4},
225224
{"c_timestamp_d", mysql.TypeTimestamp, charset.CharsetBin, mysql.NotNullFlag | mysql.BinaryFlag | mysql.TimestampFlag, 19, 0},
226-
{"c_char ", mysql.TypeString, charset.CharsetUTF8, 0, 20, 0},
225+
{"c_char ", mysql.TypeString, charset.CharsetUTF8, 0, 20, 0}, // TODO: flag should be BinaryFlag
227226
{"c_bchar ", mysql.TypeString, charset.CharsetUTF8, mysql.BinaryFlag, 20, 0},
228-
{"c_varchar ", mysql.TypeVarchar, charset.CharsetUTF8, 0, 20, 0}, // TODO: tp should be TypeVarString
229-
{"c_bvarchar ", mysql.TypeVarchar, charset.CharsetUTF8, mysql.BinaryFlag, 20, 0}, // TODO: tp should be TypeVarString
230-
{"c_text_d ", mysql.TypeBlob, charset.CharsetUTF8, 0, 65535, 0}, // TODO: BlobFlag
231-
{"c_btext_d ", mysql.TypeBlob, charset.CharsetUTF8, mysql.BinaryFlag, 65535, 0}, // TODO: BlobFlag
232-
{"c_binary ", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, 0},
233-
{"c_varbinary ", mysql.TypeVarchar, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, // TODO: tp should be TypeVarString
234-
{"c_blob_d ", mysql.TypeBlob, charset.CharsetBin, mysql.BinaryFlag, 65535, 0}, // TODO: BlobFlag
235-
{"c_set ", mysql.TypeSet, charset.CharsetUTF8, 0, types.UnspecifiedLength, 0}, // TODO: SetFlag, Flen should be 5
236-
{"c_enum ", mysql.TypeEnum, charset.CharsetUTF8, 0, types.UnspecifiedLength, 0}, // TODO: EnumFlag, Flen should be 1
227+
{"c_varchar ", mysql.TypeVarchar, charset.CharsetUTF8, 0, 20, 0}, // TODO: BinaryFlag, tp should be TypeVarString
228+
{"c_bvarchar ", mysql.TypeVarchar, charset.CharsetUTF8, mysql.BinaryFlag, 20, 0}, // TODO: BinaryFlag, tp should be TypeVarString
229+
{"c_text_d ", mysql.TypeBlob, charset.CharsetUTF8, 0, 65535, 0}, // TODO: BlobFlag, BinaryFlag
230+
{"c_btext_d ", mysql.TypeBlob, charset.CharsetUTF8, mysql.BinaryFlag, 65535, 0}, // TODO: BlobFlag, BinaryFlag
231+
{"c_binary ", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, // TODO: BinaryFlag
232+
{"c_varbinary ", mysql.TypeVarchar, charset.CharsetBin, mysql.BinaryFlag, 20, 0}, // TODO: BinaryFlag, tp should be TypeVarString
233+
{"c_blob_d ", mysql.TypeBlob, charset.CharsetBin, mysql.BinaryFlag, 65535, 0}, // TODO: BlobFlag, BinaryFlag
234+
{"c_set ", mysql.TypeSet, charset.CharsetUTF8, 0, types.UnspecifiedLength, 0}, // TODO: SetFlag, BinaryFlag, Flen should be 5
235+
{"c_enum ", mysql.TypeEnum, charset.CharsetUTF8, 0, types.UnspecifiedLength, 0}, // TODO: EnumFlag, BinaryFlag, Flen should be 1
237236
}
238237
}
239238

@@ -795,20 +794,20 @@ func (s *testInferTypeSuite) createTestCase4ControlFuncs() []typeInferTestCase {
795794
return []typeInferTestCase{
796795
{"ifnull(c_int_d, c_int_d)", mysql.TypeLong, charset.CharsetBin, mysql.BinaryFlag, 11, 0},
797796
{"ifnull(c_int_d, c_decimal)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 14, 3},
798-
{"ifnull(c_int_d, c_char)", mysql.TypeString, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength},
797+
{"ifnull(c_int_d, c_char)", mysql.TypeString, charset.CharsetUTF8, mysql.BinaryFlag, 20, types.UnspecifiedLength},
799798
{"ifnull(c_int_d, c_binary)", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength},
800799
{"ifnull(c_char, c_binary)", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength},
801800
{"ifnull(null, null)", mysql.TypeNull, charset.CharsetBin, mysql.BinaryFlag, 0, types.UnspecifiedLength},
802801
{"ifnull(c_double_d, c_timestamp_d)", mysql.TypeVarchar, charset.CharsetUTF8, mysql.NotNullFlag, 22, types.UnspecifiedLength},
803802
{"ifnull(c_json, c_decimal)", mysql.TypeLongBlob, charset.CharsetUTF8, 0, math.MaxUint32, types.UnspecifiedLength},
804803
{"if(c_int_d, c_decimal, c_int_d)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 14, 3},
805-
{"if(c_int_d, c_char, c_int_d)", mysql.TypeString, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength},
804+
{"if(c_int_d, c_char, c_int_d)", mysql.TypeString, charset.CharsetUTF8, mysql.BinaryFlag, 20, types.UnspecifiedLength},
806805
{"if(c_int_d, c_binary, c_int_d)", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 20, types.UnspecifiedLength},
807806
{"if(c_int_d, c_bchar, c_int_d)", mysql.TypeString, charset.CharsetUTF8, mysql.BinaryFlag, 20, types.UnspecifiedLength},
808-
{"if(c_int_d, c_char, c_decimal)", mysql.TypeString, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength},
807+
{"if(c_int_d, c_char, c_decimal)", mysql.TypeString, charset.CharsetUTF8, mysql.BinaryFlag, 20, types.UnspecifiedLength},
809808
{"if(c_int_d, c_datetime, c_int_d)", mysql.TypeVarchar, charset.CharsetUTF8, 0, 22, types.UnspecifiedLength},
810809
{"if(c_int_d, c_int_d, c_double_d)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, 22, types.UnspecifiedLength},
811-
{"if(c_int_d, c_time_d, c_datetime)", mysql.TypeDatetime, charset.CharsetUTF8, 0, 22, 2},
810+
{"if(c_int_d, c_time_d, c_datetime)", mysql.TypeDatetime, charset.CharsetUTF8, mysql.BinaryFlag, 22, 2}, // TODO: should not be BinaryFlag
812811
{"if(c_int_d, c_time, c_json)", mysql.TypeLongBlob, charset.CharsetUTF8, 0, math.MaxUint32, types.UnspecifiedLength},
813812
{"if(null, null, null)", mysql.TypeString, charset.CharsetBin, mysql.BinaryFlag, 0, 0},
814813
{"case when c_int_d then c_char else c_varchar end", mysql.TypeVarchar, charset.CharsetUTF8, 0, 20, types.UnspecifiedLength},
@@ -835,6 +834,7 @@ func (s *testInferTypeSuite) createTestCase4Aggregations() []typeInferTestCase {
835834
{"avg(1.0)", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDecimalWidth, 5},
836835
{"avg(1.2e2)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, types.UnspecifiedLength},
837836
{"avg(c_char)", mysql.TypeDouble, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxRealWidth, 0},
837+
{"group_concat(c_int_d)", mysql.TypeVarString, charset.CharsetUTF8, 0, mysql.MaxBlobWidth, 0},
838838
}
839839
}
840840

@@ -1951,9 +1951,3 @@ func (s *testInferTypeSuite) createTestCase4MiscellaneousFunc() []typeInferTestC
19511951
{"release_lock(c_text_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0},
19521952
}
19531953
}
1954-
1955-
func (s *testInferTypeSuite) createTestCase4AggregationFunc() []typeInferTestCase {
1956-
return []typeInferTestCase{
1957-
{"group_concat(c_int_d)", mysql.TypeVarString, charset.CharsetUTF8, 0, mysql.MaxBlobWidth, 0},
1958-
}
1959-
}

parser/parser.y

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3485,10 +3485,10 @@ SumExpr:
34853485
args := []ast.ExprNode{ast.NewValueExpr(1)}
34863486
$$ = &ast.AggregateFuncExpr{F: $1, Args: args}
34873487
}
3488-
| builtinGroupConcat '(' BuggyDefaultFalseDistinctOpt ExpressionList OptGConcatSeparator ')'
3488+
| builtinGroupConcat '(' BuggyDefaultFalseDistinctOpt ExpressionList OrderByOptional OptGConcatSeparator ')'
34893489
{
34903490
args := $4.([]ast.ExprNode)
3491-
args = append(args, $5.(ast.ExprNode))
3491+
args = append(args, $6.(ast.ExprNode))
34923492
$$ = &ast.AggregateFuncExpr{F: $1, Args: args, Distinct: $3.(bool)}
34933493
}
34943494
| builtinMax '(' BuggyDefaultFalseDistinctOpt Expression ')'

parser/parser_test.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1214,6 +1214,7 @@ func (s *testParserSuite) TestBuiltin(c *C) {
12141214
{`select group_concat(c2,c1 SEPARATOR ';') from t group by c1;`, true},
12151215
{`select group_concat(distinct c2,c1) from t group by c1;`, true},
12161216
{`select group_concat(distinctrow c2,c1) from t group by c1;`, true},
1217+
{`SELECT student_name, GROUP_CONCAT(DISTINCT test_score ORDER BY test_score DESC SEPARATOR ' ') FROM student GROUP BY student_name;`, true},
12171218

12181219
// for encryption and compression functions
12191220
{`select AES_ENCRYPT('text',UNHEX('F3229A0B371ED2D9441B830D21A390C3'))`, true},

plan/logical_plan_test.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -371,7 +371,7 @@ func (s *testPlanSuite) TestPredicatePushDown(c *C) {
371371
},
372372
{
373373
sql: "select a, d from (select * from t union all select * from t union all select * from t) z where a < 10",
374-
best: "UnionAll{DataScan(t)->Sel([lt(cast(test.t.a), 10)])->Projection->Projection->DataScan(t)->Sel([lt(cast(test.t.a), 10)])->Projection->Projection->DataScan(t)->Sel([lt(cast(test.t.a), 10)])->Projection->Projection}->Projection",
374+
best: "UnionAll{DataScan(t)->Projection->DataScan(t)->Projection->DataScan(t)->Projection}->Projection",
375375
},
376376
{
377377
sql: "select (select count(*) from t where t.a = k.a) from t k",
@@ -729,7 +729,7 @@ func (s *testPlanSuite) TestEagerAggregation(c *C) {
729729
},
730730
{
731731
sql: "select sum(c1) from (select c c1, d c2 from t a union all select a c1, b c2 from t b union all select b c1, e c2 from t c) x group by c2",
732-
best: "UnionAll{DataScan(a)->Projection->Aggr(sum(cast(a.c1)),firstrow(cast(a.c2)))->DataScan(b)->Projection->Aggr(sum(cast(b.c1)),firstrow(cast(b.c2)))->DataScan(c)->Projection->Aggr(sum(cast(c.c1)),firstrow(c.c2))}->Aggr(sum(join_agg_0))->Projection",
732+
best: "UnionAll{DataScan(a)->Aggr(sum(a.c),firstrow(a.d))->DataScan(b)->Aggr(sum(b.a),firstrow(b.b))->DataScan(c)->Aggr(sum(c.b),firstrow(c.e))}->Aggr(sum(join_agg_0))->Projection",
733733
},
734734
{
735735
sql: "select max(a.b), max(b.b) from t a join t b on a.c = b.c group by a.a",
@@ -741,7 +741,7 @@ func (s *testPlanSuite) TestEagerAggregation(c *C) {
741741
},
742742
{
743743
sql: "select max(c.b) from (select * from t a union all select * from t b) c group by c.a",
744-
best: "UnionAll{DataScan(a)->Projection->Aggr(max(cast(a.b)),firstrow(cast(a.a)))->DataScan(b)->Projection->Aggr(max(cast(b.b)),firstrow(cast(b.a)))}->Aggr(max(join_agg_0))->Projection",
744+
best: "UnionAll{DataScan(a)->Projection->DataScan(b)->Projection}->Projection->Projection",
745745
},
746746
{
747747
sql: "select max(a.c) from t a join t b on a.a=b.a and a.b=b.b group by a.b",
@@ -1505,12 +1505,12 @@ func (s *testPlanSuite) TestTopNPushDown(c *C) {
15051505
// Test TopN + UA + Proj.
15061506
{
15071507
sql: "select * from t union all (select * from t s) order by a,b limit 5",
1508-
best: "UnionAll{DataScan(t)->TopN([cast(test.t.a) cast(test.t.b)],0,5)->Projection->DataScan(s)->TopN([cast(s.a) cast(s.b)],0,5)->Projection}->TopN([t.a t.b],0,5)",
1508+
best: "UnionAll{DataScan(t)->TopN([test.t.a test.t.b],0,5)->Projection->DataScan(s)->TopN([s.a s.b],0,5)->Projection}->TopN([t.a t.b],0,5)",
15091509
},
15101510
// Test TopN + UA + Proj.
15111511
{
15121512
sql: "select * from t union all (select * from t s) order by a,b limit 5, 5",
1513-
best: "UnionAll{DataScan(t)->TopN([cast(test.t.a) cast(test.t.b)],0,10)->Projection->DataScan(s)->TopN([cast(s.a) cast(s.b)],0,10)->Projection}->TopN([t.a t.b],5,5)",
1513+
best: "UnionAll{DataScan(t)->TopN([test.t.a test.t.b],0,10)->Projection->DataScan(s)->TopN([s.a s.b],0,10)->Projection}->TopN([t.a t.b],5,5)",
15141514
},
15151515
// Test Limit + UA + Proj + Sort.
15161516
{

plan/physical_plan_test.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -616,22 +616,22 @@ func (s *testPlanSuite) TestDAGPlanBuilderUnion(c *C) {
616616
// Test simple union.
617617
{
618618
sql: "select * from t union all select * from t",
619-
best: "UnionAll{TableReader(Table(t))->Projection->TableReader(Table(t))->Projection}",
619+
best: "UnionAll{TableReader(Table(t))->TableReader(Table(t))}",
620620
},
621621
// Test Order by + Union.
622622
{
623623
sql: "select * from t union all (select * from t) order by a ",
624-
best: "UnionAll{TableReader(Table(t))->Projection->TableReader(Table(t))->Projection}->Sort",
624+
best: "UnionAll{TableReader(Table(t))->TableReader(Table(t))}->Sort",
625625
},
626626
// Test Limit + Union.
627627
{
628628
sql: "select * from t union all (select * from t) limit 1",
629-
best: "UnionAll{TableReader(Table(t)->Limit)->Projection->TableReader(Table(t)->Limit)->Projection}->Limit",
629+
best: "UnionAll{TableReader(Table(t)->Limit)->TableReader(Table(t)->Limit)}->Limit",
630630
},
631631
// Test TopN + Union.
632632
{
633633
sql: "select a from t union all (select c from t) order by a limit 1",
634-
best: "UnionAll{TableReader(Table(t))->Projection->TableReader(Table(t))->Projection}->TopN([t.a],0,1)",
634+
best: "UnionAll{TableReader(Table(t)->Limit)->IndexReader(Index(t.c_d_e)[[<nil>,+inf]]->Limit)}->TopN([t.a],0,1)",
635635
},
636636
}
637637
for _, tt := range tests {

types/field_type.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -54,8 +54,8 @@ func NewFieldType(tp byte) *FieldType {
5454

5555
// Equal checks whether two FieldType objects are equal.
5656
func (ft *FieldType) Equal(other *FieldType) bool {
57+
// We do not need to compare `ft.Flag == other.Flag` when wrapping cast upon an Expression.
5758
partialEqual := ft.Tp == other.Tp &&
58-
ft.Flag == other.Flag &&
5959
ft.Flen == other.Flen &&
6060
ft.Decimal == other.Decimal &&
6161
ft.Charset == other.Charset &&

0 commit comments

Comments
 (0)