Skip to content

Commit

Permalink
Merge branch 'master' into resource-metering
Browse files Browse the repository at this point in the history
  • Loading branch information
mornyx authored Nov 23, 2021
2 parents 110a222 + 18381f9 commit 37a684d
Show file tree
Hide file tree
Showing 11 changed files with 45 additions and 21 deletions.
1 change: 0 additions & 1 deletion executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,6 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm
if err != nil {
return nil, err
}
stmtNode = plannercore.TryAddExtraLimit(c.Ctx, stmtNode)

finalPlan, names, err := planner.Optimize(ctx, c.Ctx, stmtNode, ret.InfoSchema)
if err != nil {
Expand Down
8 changes: 2 additions & 6 deletions executor/replace.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,12 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/parser/charset"
"github.com/pingcap/tidb/parser/mysql"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"go.uber.org/zap"
Expand Down Expand Up @@ -99,11 +99,7 @@ func (e *ReplaceExec) EqualDatumsAsBinary(sc *stmtctx.StatementContext, a []type
return false, nil
}
for i, ai := range a {
collation := ai.Collation()
// We should use binary collation to compare datum, otherwise the result will be incorrect
ai.SetCollation(charset.CollationBin)
v, err := ai.CompareDatum(sc, &b[i])
ai.SetCollation(collation)
v, err := ai.Compare(sc, &b[i], collate.GetBinaryCollator())
if err != nil {
return false, errors.Trace(err)
}
Expand Down
3 changes: 2 additions & 1 deletion expression/builtin_compare.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tipb/go-tipb"
)

Expand Down Expand Up @@ -1378,7 +1379,7 @@ func RefineComparedConstant(ctx sessionctx.Context, targetFieldType types.FieldT
}
return con, false
}
c, err := intDatum.CompareDatum(sc, &con.Value)
c, err := intDatum.Compare(sc, &con.Value, collate.GetBinaryCollator())
if err != nil {
return con, false
}
Expand Down
3 changes: 2 additions & 1 deletion expression/builtin_other_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -198,7 +199,7 @@ func TestValues(t *testing.T) {
ret, err = evalBuiltinFunc(sig, chunk.Row{})
require.NoError(t, err)

cmp, err := ret.CompareDatum(nil, &currInsertValues[1])
cmp, err := ret.Compare(nil, &currInsertValues[1], collate.GetBinaryCollator())
require.NoError(t, err)
require.Equal(t, 0, cmp)
}
Expand Down
3 changes: 2 additions & 1 deletion expression/distsql_builtin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tipb/go-tipb"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -787,7 +788,7 @@ func TestEval(t *testing.T) {
result, err := expr.Eval(row)
require.NoError(t, err)
require.Equal(t, tt.result.Kind(), result.Kind())
cmp, err := result.CompareDatum(sc, &tt.result)
cmp, err := result.Compare(sc, &tt.result, collate.GetCollator(fieldTps[0].Collate))
require.NoError(t, err)
require.Equal(t, 0, cmp)
}
Expand Down
20 changes: 20 additions & 0 deletions planner/core/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4719,6 +4719,26 @@ func (s *testIntegrationSuite) TestIssue27797(c *C) {
result.Check(testkit.Rows("<nil>"))
}

func (s *testIntegrationSuite) TestIssue27949(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t27949")
tk.MustExec("create table t27949 (a int, b int, key(b))")
tk.MustQuery("explain format = 'brief' select * from t27949 where b=1").Check(testkit.Rows("IndexLookUp 10.00 root ",
"├─IndexRangeScan(Build) 10.00 cop[tikv] table:t27949, index:b(b) range:[1,1], keep order:false, stats:pseudo",
"└─TableRowIDScan(Probe) 10.00 cop[tikv] table:t27949 keep order:false, stats:pseudo"))
tk.MustExec("create global binding for select * from t27949 where b=1 using select * from t27949 ignore index(b) where b=1")
tk.MustQuery("explain format = 'brief' select * from t27949 where b=1").Check(testkit.Rows("TableReader 10.00 root data:Selection",
"└─Selection 10.00 cop[tikv] eq(test.t27949.b, 1)",
" └─TableFullScan 10000.00 cop[tikv] table:t27949 keep order:false, stats:pseudo"))
tk.MustExec("set @@sql_select_limit=100")
tk.MustQuery("explain format = 'brief' select * from t27949 where b=1").Check(testkit.Rows("Limit 10.00 root offset:0, count:100",
"└─TableReader 10.00 root data:Limit",
" └─Limit 10.00 cop[tikv] offset:0, count:100",
" └─Selection 10.00 cop[tikv] eq(test.t27949.b, 1)",
" └─TableFullScan 10000.00 cop[tikv] table:t27949 keep order:false, stats:pseudo"))
}

func (s *testIntegrationSuite) TestIssue28154(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down
10 changes: 6 additions & 4 deletions planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,9 +160,10 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
useBinding = false
}
}
if useBinding && sessVars.SelectLimit != math.MaxUint64 {
sessVars.StmtCtx.AppendWarning(errors.New("sql_select_limit is set, ignore SQL bindings"))
useBinding = false
if ok {
// add the extra Limit after matching the bind record
stmtNode = plannercore.TryAddExtraLimit(sctx, stmtNode)
node = stmtNode
}

var names types.NameSlice
Expand Down Expand Up @@ -234,7 +235,8 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
defer func() {
sessVars.StmtCtx.StmtHints = savedStmtHints
}()
if sessVars.EvolvePlanBaselines && bestPlanFromBind != nil {
if sessVars.EvolvePlanBaselines && bestPlanFromBind != nil &&
sessVars.SelectLimit == math.MaxUint64 { // do not evolve this query if sql_select_limit is enabled
// Check bestPlanFromBind firstly to avoid nil stmtNode.
if _, ok := stmtNode.(*ast.SelectStmt); ok && !bindRecord.Bindings[0].Hint.ContainTableHint(plannercore.HintReadFromStorage) {
sessVars.StmtCtx.StmtHints = originStmtHints
Expand Down
5 changes: 3 additions & 2 deletions store/mockstore/unistore/cophandler/cop_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,7 @@ import (
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/rowcodec"
"github.com/pingcap/tipb/go-tipb"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -339,10 +340,10 @@ func TestPointGet(t *testing.T) {

// verify the returned rows value as input
expectedRow := data.rows[handle]
eq, err := returnedRow[0].CompareDatum(nil, &expectedRow[0])
eq, err := returnedRow[0].Compare(nil, &expectedRow[0], collate.GetBinaryCollator())
require.NoError(t, err)
require.Equal(t, 0, eq)
eq, err = returnedRow[1].CompareDatum(nil, &expectedRow[1])
eq, err = returnedRow[1].Compare(nil, &expectedRow[1], collate.GetBinaryCollator())
require.NoError(t, err)
require.Equal(t, 0, eq)
}
Expand Down
5 changes: 3 additions & 2 deletions table/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/mock"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -126,7 +127,7 @@ func TestHandleBadNull(t *testing.T) {
d := types.Datum{}
err := col.HandleBadNull(&d, sc)
require.NoError(t, err)
cmp, err := d.CompareDatum(sc, &types.Datum{})
cmp, err := d.Compare(sc, &types.Datum{}, collate.GetBinaryCollator())
require.NoError(t, err)
require.Equal(t, 0, cmp)

Expand Down Expand Up @@ -255,7 +256,7 @@ func TestGetZeroValue(t *testing.T) {
colInfo := &model.ColumnInfo{FieldType: *tt.ft}
zv := GetZeroValue(colInfo)
require.Equal(t, tt.value.Kind(), zv.Kind())
cmp, err := zv.CompareDatum(sc, &tt.value)
cmp, err := zv.Compare(sc, &tt.value, collate.GetCollator(tt.ft.Collate))
require.NoError(t, err)
require.Equal(t, 0, cmp)
})
Expand Down
5 changes: 3 additions & 2 deletions util/codec/codec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/json"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/collate"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -987,7 +988,7 @@ func TestDecodeOneToChunk(t *testing.T) {
require.True(t, expect.IsNull())
} else {
if got.Kind() != types.KindMysqlDecimal {
cmp, err := got.CompareDatum(sc, &expect)
cmp, err := got.Compare(sc, &expect, collate.GetCollator(tp.Collate))
require.NoError(t, err)
require.Equalf(t, 0, cmp, "expect: %v, got %v", expect, got)
} else {
Expand Down Expand Up @@ -1110,7 +1111,7 @@ func TestDecodeRange(t *testing.T) {
datums1, _, err := DecodeRange(rowData, len(datums), nil, nil)
require.NoError(t, err)
for i, datum := range datums1 {
cmp, err := datum.CompareDatum(nil, &datums[i])
cmp, err := datum.Compare(nil, &datums[i], collate.GetBinaryCollator())
require.NoError(t, err)
require.Equal(t, 0, cmp)
}
Expand Down
3 changes: 2 additions & 1 deletion util/profile/flamegraph_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@ import (
"testing"

"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/collate"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -91,7 +92,7 @@ func TestProfileToDatum(t *testing.T) {
comment = fmt.Sprintf("row %2d, actual (%s), expected (%s)", i, rowStr, expectStr)
equal := true
for j, r := range row {
v, err := r.CompareDatum(nil, &datums[i][j])
v, err := r.Compare(nil, &datums[i][j], collate.GetBinaryCollator())
if v != 0 || err != nil {
equal = false
break
Expand Down

0 comments on commit 37a684d

Please sign in to comment.