Skip to content

Commit

Permalink
Merge remote-tracking branch 'upstream/release-5.0' into release-5.0-…
Browse files Browse the repository at this point in the history
…4eeff54d85c6
  • Loading branch information
time-and-fate committed Sep 1, 2021
2 parents 410e223 + c6a143b commit b733655
Show file tree
Hide file tree
Showing 57 changed files with 695 additions and 621 deletions.
30 changes: 30 additions & 0 deletions docs/tidb_http_api.md
Original file line number Diff line number Diff line change
Expand Up @@ -289,6 +289,36 @@ timezone.*
$curl http://127.0.0.1:10080/mvcc/index/test(p1)/t1/idx/1\?a\=A
```

If the handle is clustered, also specify the primary key column values in the query string

```shell
$curl http://{TiDBIP}:10080/mvcc/index/{db}/{table}/{index}?${c1}={v1}&${c2}=${v2}
```

```shell
$curl http://127.0.0.1:10080/mvcc/index/test/t/idx\?a\=1.1\&b\=111\&c\=1
{
"key": "74800000000000003B5F69800000000000000203800000000000000105BFF199999999999A013131310000000000FA",
"region_id": 59,
"value": {
"info": {
"writes": [
{
"start_ts": 424752858505150464,
"commit_ts": 424752858506461184,
"short_value": "AH0B"
}
],
"values": [
{
"start_ts": 424752858505150464,
"value": "AH0B"
}
]
}
}
}
1. Scatter regions of the specified table, add a `scatter-range` scheduler for the PD and the range is same as the table range.
```shell
Expand Down
16 changes: 12 additions & 4 deletions executor/aggfuncs/func_count_distinct.go
Original file line number Diff line number Diff line change
Expand Up @@ -319,6 +319,10 @@ func (e *countOriginalWithDistinct) UpdatePartialResult(sctx sessionctx.Context,
p := (*partialResult4CountWithDistinct)(pr)

encodedBytes := make([]byte, 0)
collators := make([]collate.Collator, 0, len(e.args))
for _, arg := range e.args {
collators = append(collators, collate.GetCollator(arg.GetType().Collate))
}
// Decimal struct is the biggest type we will use.
buf := make([]byte, types.MyDecimalStructSize)

Expand All @@ -328,7 +332,7 @@ func (e *countOriginalWithDistinct) UpdatePartialResult(sctx sessionctx.Context,
encodedBytes = encodedBytes[:0]

for i := 0; i < len(e.args) && !hasNull; i++ {
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], row, buf, encodedBytes)
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], collators[i], row, buf, encodedBytes)
if err != nil {
return memDelta, err
}
Expand All @@ -349,7 +353,7 @@ func (e *countOriginalWithDistinct) UpdatePartialResult(sctx sessionctx.Context,

// evalAndEncode eval one row with an expression and encode value to bytes.
func evalAndEncode(
sctx sessionctx.Context, arg expression.Expression,
sctx sessionctx.Context, arg expression.Expression, collator collate.Collator,
row chunk.Row, buf, encodedBytes []byte,
) (_ []byte, isNull bool, err error) {
switch tp := arg.GetType().EvalType(); tp {
Expand Down Expand Up @@ -401,7 +405,7 @@ func evalAndEncode(
if err != nil || isNull {
break
}
encodedBytes = codec.EncodeCompactBytes(encodedBytes, hack.Slice(val))
encodedBytes = codec.EncodeCompactBytes(encodedBytes, collator.Key(val))
default:
return nil, false, errors.Errorf("unsupported column type for encode %d", tp)
}
Expand Down Expand Up @@ -784,14 +788,18 @@ func (e *approxCountDistinctOriginal) UpdatePartialResult(sctx sessionctx.Contex
encodedBytes := make([]byte, 0)
// Decimal struct is the biggest type we will use.
buf := make([]byte, types.MyDecimalStructSize)
collators := make([]collate.Collator, 0, len(e.args))
for _, arg := range e.args {
collators = append(collators, collate.GetCollator(arg.GetType().Collate))
}

for _, row := range rowsInGroup {
var err error
var hasNull, isNull bool
encodedBytes = encodedBytes[:0]

for i := 0; i < len(e.args) && !hasNull; i++ {
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], row, buf, encodedBytes)
encodedBytes, isNull, err = evalAndEncode(sctx, e.args[i], collators[i], row, buf, encodedBytes)
if err != nil {
return memDelta, err
}
Expand Down
35 changes: 35 additions & 0 deletions executor/batch_point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -354,6 +354,23 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
if err != nil {
return err
}
// Change the unique index LOCK into PUT record.
if len(indexKeys) > 0 {
if !e.txn.Valid() {
return kv.ErrInvalidTxn
}
membuf := e.txn.GetMemBuffer()
for _, idxKey := range indexKeys {
handleVal := handleVals[string(idxKey)]
if len(handleVal) == 0 {
continue
}
err = membuf.Set(idxKey, handleVal)
if err != nil {
return err
}
}
}
}
// Fetch all values.
values, err = batchGetter.BatchGet(ctx, keys)
Expand All @@ -365,6 +382,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
if e.lock && rc {
existKeys = make([]kv.Key, 0, 2*len(values))
}
changeLockToPutIdxKeys := make([]kv.Key, 0, len(indexKeys))
e.values = make([][]byte, 0, len(values))
for i, key := range keys {
val := values[string(key)]
Expand All @@ -384,6 +402,7 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
// lock primary key for clustered index table is redundant
if len(indexKeys) != 0 {
existKeys = append(existKeys, indexKeys[i])
changeLockToPutIdxKeys = append(changeLockToPutIdxKeys, indexKeys[i])
}
}
}
Expand All @@ -393,6 +412,22 @@ func (e *BatchPointGetExec) initialize(ctx context.Context) error {
if err != nil {
return err
}
if len(changeLockToPutIdxKeys) > 0 {
if !e.txn.Valid() {
return kv.ErrInvalidTxn
}
for _, idxKey := range changeLockToPutIdxKeys {
membuf := e.txn.GetMemBuffer()
handleVal := handleVals[string(idxKey)]
if len(handleVal) == 0 {
return kv.ErrNotExist
}
err = membuf.Set(idxKey, handleVal)
if err != nil {
return err
}
}
}
}
e.handles = handles
return nil
Expand Down
17 changes: 17 additions & 0 deletions executor/join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2592,3 +2592,20 @@ func (s *testSuiteJoinSerial) TestIssue20219(c *C) {
tk.MustQuery("select /*+ inl_join(s)*/ t.a from t left join s on t.a = s.a;").Check(testkit.Rows("i", "j"))
tk.MustQuery("show warnings").Check(testkit.Rows())
}

func (s *testSuiteJoinSerial) TestIssue25902(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
tk.MustExec("drop table if exists tt1,tt2,tt3; ")
tk.MustExec("create table tt1 (ts timestamp);")
tk.MustExec("create table tt2 (ts varchar(32));")
tk.MustExec("create table tt3 (ts datetime);")
tk.MustExec("insert into tt1 values (\"2001-01-01 00:00:00\");")
tk.MustExec("insert into tt2 values (\"2001-01-01 00:00:00\");")
tk.MustExec("insert into tt3 values (\"2001-01-01 00:00:00\");")
tk.MustQuery("select * from tt1 where ts in (select ts from tt2);").Check(testkit.Rows("2001-01-01 00:00:00"))
tk.MustQuery("select * from tt1 where ts in (select ts from tt3);").Check(testkit.Rows("2001-01-01 00:00:00"))
tk.MustExec("set @tmp=(select @@session.time_zone);")
tk.MustExec("set @@session.time_zone = '+10:00';")
tk.MustQuery("select * from tt1 where ts in (select ts from tt2);").Check(testkit.Rows())
tk.MustExec("set @@session.time_zone = @tmp;")
}
3 changes: 3 additions & 0 deletions executor/memtable_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,6 +191,9 @@ func fetchClusterConfig(sctx sessionctx.Context, nodeTypes, nodeAddrs set.String
url = fmt.Sprintf("%s://%s%s", util.InternalHTTPSchema(), statusAddr, pdapi.Config)
case "tikv", "tidb":
url = fmt.Sprintf("%s://%s/config", util.InternalHTTPSchema(), statusAddr)
case "tiflash":
// TODO: support show tiflash config once tiflash supports it
return
default:
ch <- result{err: errors.Errorf("unknown node type: %s(%s)", typ, address)}
return
Expand Down
2 changes: 1 addition & 1 deletion executor/memtable_reader_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -170,7 +170,7 @@ func (s *testMemTableReaderSuite) TestTiDBClusterConfig(c *C) {

// mock servers
servers := []string{}
for _, typ := range []string{"tidb", "tikv", "pd"} {
for _, typ := range []string{"tidb", "tikv", "tiflash", "pd"} {
for _, server := range testServers {
servers = append(servers, strings.Join([]string{typ, server.address, server.address}, ","))
}
Expand Down
11 changes: 11 additions & 0 deletions executor/point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,17 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error {
if err != nil {
return err
}
// Change the unique index LOCK into PUT record.
if e.lock && len(e.handleVal) > 0 {
if !e.txn.Valid() {
return kv.ErrInvalidTxn
}
memBuffer := e.txn.GetMemBuffer()
err = memBuffer.Set(e.idxKey, e.handleVal)
if err != nil {
return err
}
}
}
if len(e.handleVal) == 0 {
return nil
Expand Down
2 changes: 1 addition & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ func (e *PrepareExec) Next(ctx context.Context, req *chunk.Chunk) error {
)
if sqlParser, ok := e.ctx.(sqlexec.SQLParser); ok {
// FIXME: ok... yet another parse API, may need some api interface clean.
stmts, err = sqlParser.ParseSQL(e.sqlText, charset, collation)
stmts, _, err = sqlParser.ParseSQL(ctx, e.sqlText, charset, collation)
} else {
p := parser.New()
p.SetParserConfig(vars.BuildParserConfig())
Expand Down
2 changes: 1 addition & 1 deletion executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -193,7 +193,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, old
if sc.DupKeyAsWarning {
// For `UPDATE IGNORE`/`INSERT IGNORE ON DUPLICATE KEY UPDATE`
// If the new handle or unique index exists, this will avoid to remove the record.
err = tables.CheckHandleOrUniqueKeyExistForUpdateIgnoreOrInsertOnDupIgnore(ctx, sctx, t, newHandle, newData, modified)
err = tables.CheckHandleOrUniqueKeyExistForUpdateIgnoreOrInsertOnDupIgnore(ctx, sctx, t, newHandle, newData, oldData, modified)
if err != nil {
if terr, ok := errors.Cause(err).(*terror.Error); sctx.GetSessionVars().StmtCtx.IgnoreNoPartition && ok && terr.Code() == errno.ErrNoPartitionForGivenValue {
return false, nil
Expand Down
6 changes: 6 additions & 0 deletions executor/write_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -812,6 +812,12 @@ func (s *testSuite4) TestInsertIgnoreOnDup(c *C) {
tk.MustExec("insert into t7(col_335, col_336) values(7685969, 'alice'),(2002468, 'bob')")
tk.MustExec("insert ignore into t7(col_335, col_336) values(2002468, 'david') on duplicate key update col_335 = 7685969")
tk.MustQuery("select * from t7").Check(testkit.Rows("-3217641 7685969 alice", "-3217641 2002468 bob"))

tk.MustExec("drop table if exists t8")
tk.MustExec("CREATE TABLE `t8` (`col_70` varbinary(444) NOT NULL DEFAULT 'bezhs', PRIMARY KEY (`col_70`) clustered, UNIQUE KEY `idx_22` (`col_70`(1)))")
tk.MustExec("insert into t8 values('lldcxiyfjrqzgj')")
tk.MustExec("insert ignore into t8 values ( 'lalozlkdosasfklmflo' ) on duplicate key update col_70 = 'lyhohxtby'")
tk.MustQuery("select * from t8").Check(testkit.Rows("lyhohxtby"))
}

func (s *testSuite4) TestInsertSetWithDefault(c *C) {
Expand Down
2 changes: 1 addition & 1 deletion expression/aggregation/base_func.go
Original file line number Diff line number Diff line change
Expand Up @@ -431,7 +431,7 @@ func (a *baseFuncDesc) WrapCastForAggArgs(ctx sessionctx.Context) {
if col, ok := a.Args[i].(*expression.Column); ok {
col.RetType = types.NewFieldType(col.RetType.Tp)
}
// originTp is used when the the `Tp` of column is TypeFloat32 while
// originTp is used when the `Tp` of column is TypeFloat32 while
// the type of the aggregation function is TypeFloat64.
originTp := a.Args[i].GetType().Tp
*(a.Args[i].GetType()) = *(a.RetTp)
Expand Down
2 changes: 1 addition & 1 deletion expression/builtin_control_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@ func (s *testEvaluatorSuite) TestIfNull(c *C) {
{tm, nil, tm, false, false},
{nil, duration, duration, false, false},
{nil, types.NewDecFromFloatForTest(123.123), types.NewDecFromFloatForTest(123.123), false, false},
{nil, types.NewBinaryLiteralFromUint(0x01, -1), uint64(1), false, false},
{nil, types.NewBinaryLiteralFromUint(0x01, -1), "\x01", false, false},
{nil, types.Set{Value: 1, Name: "abc"}, "abc", false, false},
{nil, jsonInt.GetMysqlJSON(), jsonInt.GetMysqlJSON(), false, false},
{"abc", nil, "abc", false, false},
Expand Down
2 changes: 1 addition & 1 deletion expression/builtin_string_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1149,7 +1149,7 @@ func (s *testEvaluatorSuite) TestHexFunc(c *C) {
{-1, false, false, "FFFFFFFFFFFFFFFF"},
{-12.3, false, false, "FFFFFFFFFFFFFFF4"},
{-12.8, false, false, "FFFFFFFFFFFFFFF3"},
{types.NewBinaryLiteralFromUint(0xC, -1), false, false, "C"},
{types.NewBinaryLiteralFromUint(0xC, -1), false, false, "0C"},
{0x12, false, false, "12"},
{nil, true, false, ""},
{errors.New("must err"), false, true, ""},
Expand Down
2 changes: 1 addition & 1 deletion expression/constant_propagation.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ func tryToReplaceCond(ctx sessionctx.Context, src *Column, tgt *Column, cond Exp
sf.FuncName.L == ast.If ||
sf.FuncName.L == ast.Case ||
sf.FuncName.L == ast.NullEQ) {
return false, false, cond
return false, true, cond
}
for idx, expr := range sf.GetArgs() {
if src.Equal(nil, expr) {
Expand Down
4 changes: 2 additions & 2 deletions expression/constant_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -254,8 +254,8 @@ func (*testExpressionSuite) TestDeferredParamNotNull(c *C) {
c.Assert(mysql.TypeTimestamp, Equals, cstTime.GetType().Tp)
c.Assert(mysql.TypeDuration, Equals, cstDuration.GetType().Tp)
c.Assert(mysql.TypeBlob, Equals, cstBytes.GetType().Tp)
c.Assert(mysql.TypeBit, Equals, cstBinary.GetType().Tp)
c.Assert(mysql.TypeBit, Equals, cstBit.GetType().Tp)
c.Assert(mysql.TypeVarString, Equals, cstBinary.GetType().Tp)
c.Assert(mysql.TypeVarString, Equals, cstBit.GetType().Tp)
c.Assert(mysql.TypeFloat, Equals, cstFloat32.GetType().Tp)
c.Assert(mysql.TypeDouble, Equals, cstFloat64.GetType().Tp)
c.Assert(mysql.TypeEnum, Equals, cstEnum.GetType().Tp)
Expand Down
34 changes: 34 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6814,6 +6814,22 @@ func (s *testIntegrationSerialSuite) TestIssue16668(c *C) {
tk.MustQuery("select count(distinct(b)) from tx").Check(testkit.Rows("4"))
}

func (s *testIntegrationSerialSuite) TestIssue27091(c *C) {
collate.SetNewCollationEnabledForTest(true)
defer collate.SetNewCollationEnabledForTest(false)
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists tx")
tk.MustExec("CREATE TABLE `tx` ( `a` int(11) NOT NULL,`b` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci DEFAULT NULL, `c` varchar(5) CHARACTER SET utf8mb4 COLLATE utf8mb4_bin DEFAULT NULL)")
tk.MustExec("insert into tx values (1, 'a', 'a'), (2, 'A ', 'a '), (3, 'A', 'A'), (4, 'a ', 'A ')")
tk.MustQuery("select count(distinct b) from tx").Check(testkit.Rows("1"))
tk.MustQuery("select count(distinct c) from tx").Check(testkit.Rows("2"))
tk.MustQuery("select count(distinct b, c) from tx where a < 3").Check(testkit.Rows("1"))
tk.MustQuery("select approx_count_distinct(b) from tx").Check(testkit.Rows("1"))
tk.MustQuery("select approx_count_distinct(c) from tx").Check(testkit.Rows("2"))
tk.MustQuery("select approx_count_distinct(b, c) from tx where a < 3").Check(testkit.Rows("1"))
}

func (s *testIntegrationSerialSuite) TestCollateStringFunction(c *C) {
collate.SetNewCollationEnabledForTest(true)
defer collate.SetNewCollationEnabledForTest(false)
Expand Down Expand Up @@ -9167,3 +9183,21 @@ func (s *testIntegrationSuite) TestRefineArgNullValues(c *C) {
"<nil>",
))
}

func (s *testIntegrationSuite) TestConstPropNullFunctions(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (a integer)")
tk.MustExec("insert into t1 values (0), (1), (2), (3)")
tk.MustExec("create table t2 (a integer, b integer)")
tk.MustExec("insert into t2 values (0,1), (1,1), (2,1), (3,1)")
tk.MustQuery("select t1.* from t1 left join t2 on t2.a = t1.a where t1.a = ifnull(t2.b, 0)").Check(testkit.Rows("1"))

tk.MustExec("drop table if exists t1, t2")
tk.MustExec("create table t1 (i1 integer, c1 char)")
tk.MustExec("insert into t1 values (2, 'a'), (1, 'b'), (3, 'c'), (0, null);")
tk.MustExec("create table t2 (i2 integer, c2 char, f2 float)")
tk.MustExec("insert into t2 values (0, 'c', null), (1, null, 0.1), (3, 'b', 0.01), (2, 'q', 0.12), (null, 'a', -0.1), (null, null, null)")
tk.MustQuery("select * from t2 where t2.i2=((select count(1) from t1 where t1.i1=t2.i2))").Check(testkit.Rows("1 <nil> 0.1"))
}
22 changes: 22 additions & 0 deletions planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -187,6 +187,21 @@ type Execute struct {
Plan Plan
}

// Check if result of GetVar expr is BinaryLiteral
// Because GetVar use String to represent BinaryLiteral, here we need to convert string back to BinaryLiteral.
func isGetVarBinaryLiteral(sctx sessionctx.Context, expr expression.Expression) (res bool) {
scalarFunc, ok := expr.(*expression.ScalarFunction)
if ok && scalarFunc.FuncName.L == ast.GetVar {
name, isNull, err := scalarFunc.GetArgs()[0].EvalString(sctx, chunk.Row{})
if err != nil || isNull {
res = false
} else if dt, ok2 := sctx.GetSessionVars().Users[name]; ok2 {
res = (dt.Kind() == types.KindBinaryLiteral)
}
}
return res
}

// OptimizePreparedPlan optimizes the prepared statement.
func (e *Execute) OptimizePreparedPlan(ctx context.Context, sctx sessionctx.Context, is infoschema.InfoSchema) error {
vars := sctx.GetSessionVars()
Expand Down Expand Up @@ -228,6 +243,13 @@ func (e *Execute) OptimizePreparedPlan(ctx context.Context, sctx sessionctx.Cont
return err
}
param := prepared.Params[i].(*driver.ParamMarkerExpr)
if isGetVarBinaryLiteral(sctx, usingVar) {
binVal, convErr := val.ToBytes()
if convErr != nil {
return convErr
}
val.SetBinaryLiteral(types.BinaryLiteral(binVal))
}
param.Datum = val
param.InExecute = true
vars.PreparedParams = append(vars.PreparedParams, val)
Expand Down
Loading

0 comments on commit b733655

Please sign in to comment.