From 7601ec32d98c545dad018c56abd68db19356a0ad Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 19 Nov 2019 10:05:38 +0800 Subject: [PATCH 1/5] ddl, util: fix uk/pk select overflows value --- executor/adapter.go | 1 + executor/point_get.go | 4 + executor/point_get_test.go | 53 ++++++++----- planner/core/point_get_plan.go | 132 ++++++++++++++++++++------------- util/testkit/testkit.go | 2 +- 5 files changed, 120 insertions(+), 72 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 2ba79d0b4638b..453c7e4c175c2 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -189,6 +189,7 @@ type ExecStmt struct { // PointGet short path for point exec directly from plan, keep only necessary steps func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*recordSet, error) { + logutil.BgLogger().Warn("xxx------------------------------------------ 00x") if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("ExecStmt.PointGet", opentracing.ChildOf(span.Context())) span1.LogKV("sql", a.OriginText()) diff --git a/executor/point_get.go b/executor/point_get.go index d858d0ae5a291..46f01412e6b1a 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -27,7 +27,9 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" + "go.uber.org/zap" ) func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { @@ -120,6 +122,7 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { tblID = e.tblInfo.ID } if e.idxInfo != nil { + logutil.BgLogger().Warn(" xxx get kv------------------------------------------ 001", zap.Reflect("val", e.idxVals)) idxKey, err1 := encodeIndexKey(e.base(), e.tblInfo, e.idxInfo, e.idxVals, tblID) if err1 != nil && !kv.ErrNotExist.Equal(err1) { return err1 @@ -154,6 +157,7 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { key := tablecodec.EncodeRowKeyWithHandle(tblID, e.handle) val, err := e.get(ctx, key) + logutil.BgLogger().Warn(" xxx get kv------------------------------------------ 002", zap.Reflect("val", key)) if err != nil && !kv.ErrNotExist.Equal(err) { return err } diff --git a/executor/point_get_test.go b/executor/point_get_test.go index 38de3e6665933..f2e187914ac75 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/testkit" ) @@ -137,10 +138,10 @@ func (s *testPointGetSuite) TestPointGetCharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "";`).Check(testkit.Rows(` `)) tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = " ";`).Check(testkit.Rows()) } @@ -153,7 +154,7 @@ func (s *testPointGetSuite) TestPointGetAliasTableCharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) tk.MustExec(`truncate table t;`) tk.MustExec(`insert into t values("a ", "b ");`) @@ -161,7 +162,7 @@ func (s *testPointGetSuite) TestPointGetAliasTableCharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t tmp where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) // Test CHAR BINARY. tk.MustExec(`drop table if exists t;`) @@ -172,10 +173,10 @@ func (s *testPointGetSuite) TestPointGetAliasTableCharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t tmp where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t tmp where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t tmp where a = "";`).Check(testkit.Rows(` `)) tk.MustPointGet(`select * from t tmp where a = " ";`).Check(testkit.Rows()) - tk.MustPointGet(`select * from t tmp where a = " ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t tmp where a = " ";`).Check(testkit.Rows()) // Test both wildcard and column name exist in select field list tk.MustExec(`set @@sql_mode="";`) @@ -188,9 +189,9 @@ func (s *testPointGetSuite) TestPointGetAliasTableCharPK(c *C) { tk.MustPointGet(`select tmp.* from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) tk.MustPointGet(`select tmp.a, tmp.b from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) tk.MustPointGet(`select tmp.*, tmp.a, tmp.b from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb aa bb`)) - tk.MustPointGet(`select tmp.* from t tmp where a = "aab";`).Check(testkit.Rows()) - tk.MustPointGet(`select tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) - tk.MustPointGet(`select tmp.*, tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select tmp.* from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select tmp.*, tmp.a, tmp.b from t tmp where a = "aab";`).Check(testkit.Rows()) // Test using table alias in where clause tk.MustPointGet(`select * from t tmp where tmp.a = "aa";`).Check(testkit.Rows(`aa bb`)) @@ -221,11 +222,11 @@ func (s *testPointGetSuite) TestIndexLookupChar(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustIndexLookup(`select * from t where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustIndexLookup(`select * from t where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "aab";`).Check(testkit.Rows()) // Test query with table alias tk.MustIndexLookup(`select * from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustIndexLookup(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) tk.MustExec(`truncate table t;`) tk.MustExec(`insert into t values("a ", "b ");`) @@ -233,7 +234,7 @@ func (s *testPointGetSuite) TestIndexLookupChar(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustIndexLookup(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) // Test CHAR BINARY. tk.MustExec(`drop table if exists t;`) @@ -244,11 +245,11 @@ func (s *testPointGetSuite) TestIndexLookupChar(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustIndexLookup(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustIndexLookup(`select * from t where a = "";`).Check(testkit.Rows(` `)) tk.MustIndexLookup(`select * from t where a = " ";`).Check(testkit.Rows()) tk.MustIndexLookup(`select * from t where a = " ";`).Check(testkit.Rows()) - tk.MustIndexLookup(`select * from t where a = " ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = " ";`).Check(testkit.Rows()) } @@ -261,7 +262,7 @@ func (s *testPointGetSuite) TestPointGetVarcharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustPointGet(`select * from t where a = "aab";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "aab";`).Check(testkit.Rows()) tk.MustExec(`truncate table t;`) tk.MustExec(`insert into t values("a ", "b ");`) @@ -269,7 +270,7 @@ func (s *testPointGetSuite) TestPointGetVarcharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows(`a b `)) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) // // Test VARCHAR BINARY. tk.MustExec(`drop table if exists t;`) @@ -280,10 +281,10 @@ func (s *testPointGetSuite) TestPointGetVarcharPK(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows(`a b `)) - tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows(` `)) - tk.MustPointGet(`select * from t where a = " ";`).Check(testkit.Rows()) + tk.MustTableDual(`select * from t where a = " ";`).Check(testkit.Rows()) } @@ -295,7 +296,9 @@ func (s *testPointGetSuite) TestPointGetBinaryPK(c *C) { tk.MustExec(`insert into t values("a", "b");`) tk.MustExec(`set @@sql_mode="";`) + logutil.BgLogger().Warn("yyy-------------------------------------------------------------------------------------------------------------- xxxxxxxxxxxx") tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows()) + logutil.BgLogger().Warn("yyy-------------------------------------------------------------------------------------------------------------- xxxxxxxxxxxx") tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "a\0";`).Check(testkit.Rows("a\x00 b\x00")) @@ -364,6 +367,20 @@ func (s *testPointGetSuite) TestIndexLookupBinary(c *C) { } +func (s *testPointGetSuite) TestOverflowOr(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("create table t6 (id bigint, a bigint, primary key(id), unique key(a));") + tk.MustExec("insert into t6 values(9223372036854775807, 9223372036854775807);") + tk.MustExec("insert into t6 values(1, 1);") + var nilVal []string + // for unique key + tk.MustQuery("select * from t6 where a = 9223372036854775808").Check(testkit.Rows(nilVal...)) + tk.MustQuery("select * from t6 where a = '1.123'").Check(testkit.Rows(nilVal...)) + // for primary key + tk.MustQuery("select * from t6 where id = 9223372036854775808").Check(testkit.Rows(nilVal...)) + tk.MustQuery("select * from t6 where id = '1.123'").Check(testkit.Rows(nilVal...)) +} + func (s *testPointGetSuite) TestIssue10448(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 63673b232a72a..35a1173e932f1 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/opcode" @@ -32,9 +33,11 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/parser_driver" + "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/math" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tipb/go-tipb" + "go.uber.org/zap" ) // PointGetPlan is a fast plan for simple point get. @@ -556,11 +559,15 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP if tbl == nil { return nil } + // Do not handle partitioned table. // Table partition implementation translates LogicalPlan from `DataSource` to // `Union -> DataSource` in the logical plan optimization pass, since PointGetPlan // bypass the logical plan optimization, it can't support partitioned table. pi := tbl.GetPartitionInfo() + if pi != nil && pi.Type != model.PartitionTypeHash { + return nil + } for _, col := range tbl.Columns { // Do not handle generated columns. if col.IsGenerated() { @@ -571,52 +578,48 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP return nil } } + schema, names := buildSchemaFromFields(tblName.Schema, tbl, tblAlias, selStmt.Fields.Fields) + if schema == nil { + return nil + } + dbName := tblName.Schema.L + if dbName == "" { + dbName = ctx.GetSessionVars().CurrentDB + } + pairs := make([]nameValuePair, 0, 4) - pairs = getNameValuePairs(pairs, tblAlias, selStmt.Where) + logutil.BgLogger().Warn("xxx------------------------------------------======================================overflows ----------------------- 0") + pairs, isTableDual := getNameValuePairs(ctx.GetSessionVars().StmtCtx, tbl, tblAlias, pairs, selStmt.Where) + logutil.BgLogger().Warn("xxx------------------------------------------======================================overflows ----------------------- 1", + zap.Bool("ok", isTableDual), zap.Bool("pairs is nil", pairs == nil)) + if isTableDual { + p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names) + p.IsTableDual = true + return p + } if pairs == nil { return nil } var partitionInfo *model.PartitionDefinition if pi != nil { - if pi.Type != model.PartitionTypeHash { - return nil - } partitionInfo = getPartitionInfo(ctx, tbl, pairs) + logutil.BgLogger().Warn("xxx------------------------------------------======================================overflows ----------------------- is part", + zap.Bool("has part info", partitionInfo != nil)) if partitionInfo == nil { return nil } } + handlePair, fieldType := findPKHandle(tbl, pairs) + logutil.BgLogger().Warn("xxx------------------------------------------ 001", + zap.Reflect("val", handlePair.value), zap.Reflect("p", handlePair.param), + zap.String("where", selStmt.Where.Text()), + zap.Bool("nil", handlePair.value.Kind() == types.KindNull), + zap.Int("len", len(pairs))) if handlePair.value.Kind() != types.KindNull && len(pairs) == 1 { - schema, names := buildSchemaFromFields(tblName.Schema, tbl, tblAlias, selStmt.Fields.Fields) - if schema == nil { - return nil - } - dbName := tblName.Schema.L - if dbName == "" { - dbName = ctx.GetSessionVars().CurrentDB - } p := newPointGetPlan(ctx, dbName, schema, tbl, names) - intDatum, err := handlePair.value.ConvertTo(ctx.GetSessionVars().StmtCtx, fieldType) - if err != nil { - if terror.ErrorEqual(types.ErrOverflow, err) { - p.IsTableDual = true - return p - } - // some scenarios cast to int with error, but we may use this value in point get - if !terror.ErrorEqual(types.ErrTruncatedWrongVal, err) { - return nil - } - } - cmp, err := intDatum.CompareDatum(ctx.GetSessionVars().StmtCtx, &handlePair.value) - if err != nil { - return nil - } else if cmp != 0 { - p.IsTableDual = true - return p - } - p.Handle = intDatum.GetInt64() + p.Handle = handlePair.value.GetInt64() p.UnsignedHandle = mysql.HasUnsignedFlag(fieldType.Flag) p.HandleParam = handlePair.param p.PartitionInfo = partitionInfo @@ -634,19 +637,12 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP if idxValues == nil { continue } - schema, names := buildSchemaFromFields(tblName.Schema, tbl, tblAlias, selStmt.Fields.Fields) - if schema == nil { - return nil - } - dbName := tblName.Schema.L - if dbName == "" { - dbName = ctx.GetSessionVars().CurrentDB - } p := newPointGetPlan(ctx, dbName, schema, tbl, names) p.IndexInfo = idxInfo p.IndexValues = idxValues p.IndexValueParams = idxValueParams p.PartitionInfo = partitionInfo + logutil.BgLogger().Warn("xxx------------------------------------------ 002", zap.Reflect("val", idxValues[0].GetInt64())) return p } return nil @@ -769,21 +765,22 @@ func getSingleTableNameAndAlias(tableRefs *ast.TableRefsClause) (tblName *ast.Ta } // getNameValuePairs extracts `column = constant/paramMarker` conditions from expr as name value pairs. -func getNameValuePairs(nvPairs []nameValuePair, tblName model.CIStr, expr ast.ExprNode) []nameValuePair { +func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, tblName model.CIStr, nvPairs []nameValuePair, expr ast.ExprNode) ( + pairs []nameValuePair, isTableDual bool) { binOp, ok := expr.(*ast.BinaryOperationExpr) if !ok { - return nil + return nil, false } if binOp.Op == opcode.LogicAnd { - nvPairs = getNameValuePairs(nvPairs, tblName, binOp.L) - if nvPairs == nil { - return nil + nvPairs, isTableDual = getNameValuePairs(stmtCtx, tbl, tblName, nvPairs, binOp.L) + if nvPairs == nil || isTableDual { + return nil, isTableDual } - nvPairs = getNameValuePairs(nvPairs, tblName, binOp.R) - if nvPairs == nil { - return nil + nvPairs, isTableDual = getNameValuePairs(stmtCtx, tbl, tblName, nvPairs, binOp.R) + if nvPairs == nil || isTableDual { + return nil, isTableDual } - return nvPairs + return nvPairs, isTableDual } else if binOp.Op == opcode.EQ { var d types.Datum var colName *ast.ColumnNameExpr @@ -806,17 +803,46 @@ func getNameValuePairs(nvPairs []nameValuePair, tblName model.CIStr, expr ast.Ex param = x } } else { - return nil + return nil, false } if d.IsNull() { - return nil + return nil, false } if colName.Name.Table.L != "" && colName.Name.Table.L != tblName.L { - return nil + return nil, false } - return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}) + col := model.FindColumnInfo(tbl.Cols(), colName.Name.Name.L) + if col == nil || // Handling the case when the column is _tidb_rowid. + (col.Tp == mysql.TypeString && col.Collate == charset.CollationBin) { // This type we needn't to pad `\0` in here. + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}), false + } + dVal, err := d.ConvertTo(stmtCtx, &col.FieldType) + logutil.BgLogger().Warn("yyy--------------------------------", + zap.String("col", colName.Name.String()), zap.Reflect("dVal", dVal.GetString()), + zap.String("d", d.GetString()), zap.Reflect("kind", d.Kind()), + zap.String("field", col.FieldType.String()), zap.Error(err)) + if err != nil { + if terror.ErrorEqual(types.ErrOverflow, err) { + return nil, true + } + // Some scenarios cast to int with error, but we may use this value in point get. + if !terror.ErrorEqual(types.ErrTruncatedWrongVal, err) { + return nil, false + } + } + // The converted result must be same as original datum. + cmp, err := d.CompareDatum(stmtCtx, &dVal) + logutil.BgLogger().Warn("yyy--------------------------------", zap.Bool("cmp is 0", cmp == 0), + zap.String("tp", col.FieldType.String()), zap.Int("tp", int(col.FieldType.Tp)), zap.String("collate", col.Collate)) + if err != nil { + return nil, false + } else if cmp != 0 { + return nil, true + } + + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), false } - return nil + return nil, false } func findPKHandle(tblInfo *model.TableInfo, pairs []nameValuePair) (handlePair nameValuePair, fieldType *types.FieldType) { diff --git a/util/testkit/testkit.go b/util/testkit/testkit.go index 20d31acfefc22..022fe73a127c2 100644 --- a/util/testkit/testkit.go +++ b/util/testkit/testkit.go @@ -224,7 +224,7 @@ func (tk *TestKit) MustTableDual(sql string, args ...interface{}) *Result { func (tk *TestKit) MustPointGet(sql string, args ...interface{}) *Result { rs := tk.MustQuery("explain "+sql, args...) tk.c.Assert(len(rs.rows), check.Equals, 1) - tk.c.Assert(strings.Contains(rs.rows[0][0], "Point_Get"), check.IsTrue) + tk.c.Assert(strings.Contains(rs.rows[0][0], "Point_Get"), check.IsTrue, check.Commentf("plan %v", rs.rows[0][0])) return tk.MustQuery(sql, args...) } From 2c02cb0c0f818963dc696fb0d1b7bc471733c3de Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 13 Feb 2020 15:03:50 +0800 Subject: [PATCH 2/5] *: remove logs --- executor/adapter.go | 1 - executor/point_get.go | 4 ---- executor/point_get_test.go | 4 +--- planner/core/point_get_plan.go | 19 ------------------- 4 files changed, 1 insertion(+), 27 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index 453c7e4c175c2..2ba79d0b4638b 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -189,7 +189,6 @@ type ExecStmt struct { // PointGet short path for point exec directly from plan, keep only necessary steps func (a *ExecStmt) PointGet(ctx context.Context, is infoschema.InfoSchema) (*recordSet, error) { - logutil.BgLogger().Warn("xxx------------------------------------------ 00x") if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("ExecStmt.PointGet", opentracing.ChildOf(span.Context())) span1.LogKV("sql", a.OriginText()) diff --git a/executor/point_get.go b/executor/point_get.go index 46f01412e6b1a..d858d0ae5a291 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -27,9 +27,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/codec" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" - "go.uber.org/zap" ) func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { @@ -122,7 +120,6 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { tblID = e.tblInfo.ID } if e.idxInfo != nil { - logutil.BgLogger().Warn(" xxx get kv------------------------------------------ 001", zap.Reflect("val", e.idxVals)) idxKey, err1 := encodeIndexKey(e.base(), e.tblInfo, e.idxInfo, e.idxVals, tblID) if err1 != nil && !kv.ErrNotExist.Equal(err1) { return err1 @@ -157,7 +154,6 @@ func (e *PointGetExecutor) Next(ctx context.Context, req *chunk.Chunk) error { key := tablecodec.EncodeRowKeyWithHandle(tblID, e.handle) val, err := e.get(ctx, key) - logutil.BgLogger().Warn(" xxx get kv------------------------------------------ 002", zap.Reflect("val", key)) if err != nil && !kv.ErrNotExist.Equal(err) { return err } diff --git a/executor/point_get_test.go b/executor/point_get_test.go index f2e187914ac75..d0e9bde5f88ae 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -296,9 +296,7 @@ func (s *testPointGetSuite) TestPointGetBinaryPK(c *C) { tk.MustExec(`insert into t values("a", "b");`) tk.MustExec(`set @@sql_mode="";`) - logutil.BgLogger().Warn("yyy-------------------------------------------------------------------------------------------------------------- xxxxxxxxxxxx") tk.MustPointGet(`select * from t where a = "a";`).Check(testkit.Rows()) - logutil.BgLogger().Warn("yyy-------------------------------------------------------------------------------------------------------------- xxxxxxxxxxxx") tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustPointGet(`select * from t where a = "a\0";`).Check(testkit.Rows("a\x00 b\x00")) @@ -367,7 +365,7 @@ func (s *testPointGetSuite) TestIndexLookupBinary(c *C) { } -func (s *testPointGetSuite) TestOverflowOr(c *C) { +func (s *testPointGetSuite) TestOverflowOrTruncated(c *C) { tk := testkit.NewTestKitWithInit(c, s.store) tk.MustExec("create table t6 (id bigint, a bigint, primary key(id), unique key(a));") tk.MustExec("insert into t6 values(9223372036854775807, 9223372036854775807);") diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 35a1173e932f1..eb1670e1adddf 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -33,11 +33,9 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/parser_driver" - "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/math" "github.com/pingcap/tidb/util/plancodec" "github.com/pingcap/tipb/go-tipb" - "go.uber.org/zap" ) // PointGetPlan is a fast plan for simple point get. @@ -588,10 +586,7 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP } pairs := make([]nameValuePair, 0, 4) - logutil.BgLogger().Warn("xxx------------------------------------------======================================overflows ----------------------- 0") pairs, isTableDual := getNameValuePairs(ctx.GetSessionVars().StmtCtx, tbl, tblAlias, pairs, selStmt.Where) - logutil.BgLogger().Warn("xxx------------------------------------------======================================overflows ----------------------- 1", - zap.Bool("ok", isTableDual), zap.Bool("pairs is nil", pairs == nil)) if isTableDual { p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names) p.IsTableDual = true @@ -604,19 +599,12 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP var partitionInfo *model.PartitionDefinition if pi != nil { partitionInfo = getPartitionInfo(ctx, tbl, pairs) - logutil.BgLogger().Warn("xxx------------------------------------------======================================overflows ----------------------- is part", - zap.Bool("has part info", partitionInfo != nil)) if partitionInfo == nil { return nil } } handlePair, fieldType := findPKHandle(tbl, pairs) - logutil.BgLogger().Warn("xxx------------------------------------------ 001", - zap.Reflect("val", handlePair.value), zap.Reflect("p", handlePair.param), - zap.String("where", selStmt.Where.Text()), - zap.Bool("nil", handlePair.value.Kind() == types.KindNull), - zap.Int("len", len(pairs))) if handlePair.value.Kind() != types.KindNull && len(pairs) == 1 { p := newPointGetPlan(ctx, dbName, schema, tbl, names) p.Handle = handlePair.value.GetInt64() @@ -642,7 +630,6 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP p.IndexValues = idxValues p.IndexValueParams = idxValueParams p.PartitionInfo = partitionInfo - logutil.BgLogger().Warn("xxx------------------------------------------ 002", zap.Reflect("val", idxValues[0].GetInt64())) return p } return nil @@ -817,10 +804,6 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}), false } dVal, err := d.ConvertTo(stmtCtx, &col.FieldType) - logutil.BgLogger().Warn("yyy--------------------------------", - zap.String("col", colName.Name.String()), zap.Reflect("dVal", dVal.GetString()), - zap.String("d", d.GetString()), zap.Reflect("kind", d.Kind()), - zap.String("field", col.FieldType.String()), zap.Error(err)) if err != nil { if terror.ErrorEqual(types.ErrOverflow, err) { return nil, true @@ -832,8 +815,6 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, } // The converted result must be same as original datum. cmp, err := d.CompareDatum(stmtCtx, &dVal) - logutil.BgLogger().Warn("yyy--------------------------------", zap.Bool("cmp is 0", cmp == 0), - zap.String("tp", col.FieldType.String()), zap.Int("tp", int(col.FieldType.Tp)), zap.String("collate", col.Collate)) if err != nil { return nil, false } else if cmp != 0 { From 328769e457e0c2f22455492e90561e59c1dabcd8 Mon Sep 17 00:00:00 2001 From: Lynn Date: Thu, 26 Mar 2020 11:27:57 +0800 Subject: [PATCH 3/5] *: pass TestDelete --- executor/point_get_test.go | 10 +++++----- planner/core/point_get_plan.go | 28 ++++++++++++++++++++-------- 2 files changed, 25 insertions(+), 13 deletions(-) diff --git a/executor/point_get_test.go b/executor/point_get_test.go index f0bf69cccd703..89a09d9eade62 100644 --- a/executor/point_get_test.go +++ b/executor/point_get_test.go @@ -225,11 +225,11 @@ func (s *testPointGetSuite) TestIndexLookupChar(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustIndexLookup(`select * from t where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustTableDual(`select * from t where a = "aab";`).Check(testkit.Rows()) + tk.MustIndexLookup(`select * from t where a = "aab";`).Check(testkit.Rows()) // Test query with table alias tk.MustIndexLookup(`select * from t tmp where a = "aa";`).Check(testkit.Rows(`aa bb`)) - tk.MustTableDual(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) + tk.MustIndexLookup(`select * from t tmp where a = "aab";`).Check(testkit.Rows()) tk.MustExec(`truncate table t;`) tk.MustExec(`insert into t values("a ", "b ");`) @@ -237,7 +237,7 @@ func (s *testPointGetSuite) TestIndexLookupChar(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustIndexLookup(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) // Test CHAR BINARY. tk.MustExec(`drop table if exists t;`) @@ -248,11 +248,11 @@ func (s *testPointGetSuite) TestIndexLookupChar(c *C) { tk.MustExec(`set @@sql_mode="";`) tk.MustIndexLookup(`select * from t where a = "a";`).Check(testkit.Rows(`a b`)) tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) - tk.MustTableDual(`select * from t where a = "a ";`).Check(testkit.Rows()) + tk.MustIndexLookup(`select * from t where a = "a ";`).Check(testkit.Rows()) tk.MustIndexLookup(`select * from t where a = "";`).Check(testkit.Rows(` `)) tk.MustIndexLookup(`select * from t where a = " ";`).Check(testkit.Rows()) tk.MustIndexLookup(`select * from t where a = " ";`).Check(testkit.Rows()) - tk.MustTableDual(`select * from t where a = " ";`).Check(testkit.Rows()) + tk.MustIndexLookup(`select * from t where a = " ";`).Check(testkit.Rows()) } diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 6874f106ac9fb..bf76e06ec4eb9 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -677,12 +677,7 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP pairs := make([]nameValuePair, 0, 4) pairs, isTableDual := getNameValuePairs(ctx.GetSessionVars().StmtCtx, tbl, tblAlias, pairs, selStmt.Where) - if isTableDual { - p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names) - p.IsTableDual = true - return p - } - if pairs == nil { + if pairs == nil && !isTableDual { return nil } @@ -697,6 +692,12 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP handlePair, fieldType := findPKHandle(tbl, pairs) if handlePair.value.Kind() != types.KindNull && len(pairs) == 1 { + if isTableDual { + p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names) + p.IsTableDual = true + return p + } + p := newPointGetPlan(ctx, dbName, schema, tbl, names) p.Handle = handlePair.value.GetInt64() p.UnsignedHandle = mysql.HasUnsignedFlag(fieldType.Flag) @@ -712,6 +713,12 @@ func tryPointGetPlan(ctx sessionctx.Context, selStmt *ast.SelectStmt) *PointGetP if idxInfo.State != model.StatePublic { continue } + if isTableDual { + p := newPointGetPlan(ctx, tblName.Schema.O, schema, tbl, names) + p.IsTableDual = true + return p + } + idxValues, idxValueParams := getIndexValues(idxInfo, pairs) if idxValues == nil { continue @@ -889,6 +896,10 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, if d.IsNull() { return nil, false } + // Views' columns have no FieldType. + if tbl.IsView() { + return nil, false + } if colName.Name.Table.L != "" && colName.Name.Table.L != tblName.L { return nil, false } @@ -900,7 +911,7 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, dVal, err := d.ConvertTo(stmtCtx, &col.FieldType) if err != nil { if terror.ErrorEqual(types.ErrOverflow, err) { - return nil, true + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), true } // Some scenarios cast to int with error, but we may use this value in point get. if !terror.ErrorEqual(types.ErrTruncatedWrongVal, err) { @@ -912,7 +923,8 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, if err != nil { return nil, false } else if cmp != 0 { - return nil, true + // return nil, true + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), true } return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), false From 181491a91b90db2b091e7495a1febdf74f3d334a Mon Sep 17 00:00:00 2001 From: Lynn Date: Tue, 14 Apr 2020 12:07:45 +0800 Subject: [PATCH 4/5] *: tiny update --- planner/core/point_get_plan.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index bf76e06ec4eb9..72f859a695480 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -911,7 +911,7 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, dVal, err := d.ConvertTo(stmtCtx, &col.FieldType) if err != nil { if terror.ErrorEqual(types.ErrOverflow, err) { - return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), true + return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: d, param: param}), true } // Some scenarios cast to int with error, but we may use this value in point get. if !terror.ErrorEqual(types.ErrTruncatedWrongVal, err) { From f24a5659d4a296d932590155de841c754543a5a6 Mon Sep 17 00:00:00 2001 From: Lynn Date: Wed, 15 Apr 2020 16:05:32 +0800 Subject: [PATCH 5/5] planner/core: remove useless comments --- planner/core/point_get_plan.go | 1 - 1 file changed, 1 deletion(-) diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 72f859a695480..0b2de9cdd9e9f 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -923,7 +923,6 @@ func getNameValuePairs(stmtCtx *stmtctx.StatementContext, tbl *model.TableInfo, if err != nil { return nil, false } else if cmp != 0 { - // return nil, true return append(nvPairs, nameValuePair{colName: colName.Name.Name.L, value: dVal, param: param}), true }